You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by kr...@apache.org on 2016/10/20 19:31:24 UTC

[15/50] [abbrv] lucene-solr:jira/solr-8593: LUCENE-7494: Give points a per-field API.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
index a75290a..7228f37 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
@@ -503,8 +503,8 @@ public class TestPointValues extends LuceneTestCase {
     IndexReader r = DirectoryReader.open(w);
 
     for(LeafReaderContext ctx : r.leaves()) {
-      PointValues points = ctx.reader().getPointValues();
-      points.intersect("int",
+      PointValues points = ctx.reader().getPointValues("int");
+      points.intersect(
                        new IntersectVisitor() {
 
                          int lastDocID = -1;
@@ -553,8 +553,7 @@ public class TestPointValues extends LuceneTestCase {
     
     w.forceMerge(1);
     DirectoryReader r = w.getReader();
-    assertEquals(0, r.leaves().get(0).reader().getPointValues().size("int"));
-    assertEquals(0, r.leaves().get(0).reader().getPointValues().getDocCount("int"));
+    assertNull(r.leaves().get(0).reader().getPointValues("int"));
     w.close();
     r.close();
     dir.close();
@@ -611,10 +610,10 @@ public class TestPointValues extends LuceneTestCase {
       int size = 0;
       String fieldName = "int" + field;
       for(LeafReaderContext ctx : r.leaves()) {
-        PointValues points = ctx.reader().getPointValues();
-        if (ctx.reader().getFieldInfos().fieldInfo(fieldName) != null) {
-          docCount += points.getDocCount(fieldName);
-          size += points.size(fieldName);
+        PointValues points = ctx.reader().getPointValues(fieldName);
+        if (points != null) {
+          docCount += points.getDocCount();
+          size += points.size();
         }
       }
       assertEquals(fieldDocCounts[field], docCount);
@@ -738,7 +737,7 @@ public class TestPointValues extends LuceneTestCase {
     final IndexReader reader1 = DirectoryReader.open(w);
     w.forceMerge(1);
     final IndexReader reader2 = DirectoryReader.open(w);
-    final PointValues expected = getOnlyLeafReader(reader2).getPointValues();
+    final PointValues expected = getOnlyLeafReader(reader2).getPointValues("field");
     if (expected == null) {
       assertNull(PointValues.getMinPackedValue(reader1, "field"));
       assertNull(PointValues.getMaxPackedValue(reader1, "field"));
@@ -746,13 +745,13 @@ public class TestPointValues extends LuceneTestCase {
       assertEquals(0, PointValues.size(reader1, "field"));
     } else {
       assertArrayEquals(
-          expected.getMinPackedValue("field"),
+          expected.getMinPackedValue(),
           PointValues.getMinPackedValue(reader1, "field"));
       assertArrayEquals(
-          expected.getMaxPackedValue("field"),
+          expected.getMaxPackedValue(),
           PointValues.getMaxPackedValue(reader1, "field"));
-      assertEquals(expected.getDocCount("field"), PointValues.getDocCount(reader1, "field"));
-      assertEquals(expected.size("field"),  PointValues.size(reader1, "field"));
+      assertEquals(expected.getDocCount(), PointValues.getDocCount(reader1, "field"));
+      assertEquals(expected.size(),  PointValues.size(reader1, "field"));
     }
     IOUtils.close(w, reader1, reader2, dir);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java b/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
index 62e85fe..625b8c2 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestDocIdSetBuilder.java
@@ -307,37 +307,37 @@ public class TestDocIdSetBuilder extends LuceneTestCase {
     }
 
     @Override
-    public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
+    public void intersect(IntersectVisitor visitor) throws IOException {
       throw new UnsupportedOperationException();
     }
 
     @Override
-    public byte[] getMinPackedValue(String fieldName) throws IOException {
+    public byte[] getMinPackedValue() throws IOException {
       throw new UnsupportedOperationException();
     }
 
     @Override
-    public byte[] getMaxPackedValue(String fieldName) throws IOException {
+    public byte[] getMaxPackedValue() throws IOException {
       throw new UnsupportedOperationException();
     }
 
     @Override
-    public int getNumDimensions(String fieldName) throws IOException {
+    public int getNumDimensions() throws IOException {
       throw new UnsupportedOperationException();
     }
 
     @Override
-    public int getBytesPerDimension(String fieldName) throws IOException {
+    public int getBytesPerDimension() throws IOException {
       throw new UnsupportedOperationException();
     }
 
     @Override
-    public long size(String fieldName) {
+    public long size() {
       return numPoints;
     }
 
     @Override
-    public int getDocCount(String fieldName) {
+    public int getDocCount() {
       return docCount;
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java b/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
index 4616ce3..8d2ea3e 100644
--- a/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
+++ b/lucene/core/src/test/org/apache/lucene/util/bkd/TestMutablePointsReaderUtils.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Comparator;
 
-import org.apache.lucene.codecs.MutablePointsReader;
+import org.apache.lucene.codecs.MutablePointValues;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
@@ -184,7 +184,7 @@ public class TestMutablePointsReaderUtils extends LuceneTestCase {
     }
   }
 
-  private static class DummyPointsReader extends MutablePointsReader {
+  private static class DummyPointsReader extends MutablePointValues {
 
     private final Point[] points;
 
@@ -193,16 +193,6 @@ public class TestMutablePointsReaderUtils extends LuceneTestCase {
     }
 
     @Override
-    public void close() throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public long ramBytesUsed() {
-      return 0;
-    }
-
-    @Override
     public void getValue(int i, BytesRef packedValue) {
       packedValue.bytes = points[i].packedValue.bytes;
       packedValue.offset = points[i].packedValue.offset;
@@ -226,42 +216,37 @@ public class TestMutablePointsReaderUtils extends LuceneTestCase {
     }
 
     @Override
-    public void checkIntegrity() throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
+    public void intersect(IntersectVisitor visitor) throws IOException {
       throw new UnsupportedOperationException();
     }
 
     @Override
-    public byte[] getMinPackedValue(String fieldName) throws IOException {
+    public byte[] getMinPackedValue() throws IOException {
       throw new UnsupportedOperationException();
     }
 
     @Override
-    public byte[] getMaxPackedValue(String fieldName) throws IOException {
+    public byte[] getMaxPackedValue() throws IOException {
       throw new UnsupportedOperationException();
     }
 
     @Override
-    public int getNumDimensions(String fieldName) throws IOException {
+    public int getNumDimensions() throws IOException {
       throw new UnsupportedOperationException();
     }
 
     @Override
-    public int getBytesPerDimension(String fieldName) throws IOException {
+    public int getBytesPerDimension() throws IOException {
       throw new UnsupportedOperationException();
     }
 
     @Override
-    public long size(String fieldName) {
+    public long size() {
       throw new UnsupportedOperationException();
     }
 
     @Override
-    public int getDocCount(String fieldName) {
+    public int getDocCount() {
       throw new UnsupportedOperationException();
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
index 07506f2..608e3d4 100644
--- a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
@@ -144,7 +144,7 @@ public class TermVectorLeafReader extends LeafReader {
   }
 
   @Override
-  public PointValues getPointValues() {
+  public PointValues getPointValues(String fieldName) {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/join/src/java/org/apache/lucene/search/join/PointInSetIncludingScoreQuery.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/PointInSetIncludingScoreQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/PointInSetIncludingScoreQuery.java
index f99f318..70c28d5 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/PointInSetIncludingScoreQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/PointInSetIncludingScoreQuery.java
@@ -140,10 +140,6 @@ abstract class PointInSetIncludingScoreQuery extends Query {
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
         LeafReader reader = context.reader();
-        PointValues values = reader.getPointValues();
-        if (values == null) {
-          return null;
-        }
         FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field);
         if (fieldInfo == null) {
           return null;
@@ -154,10 +150,14 @@ abstract class PointInSetIncludingScoreQuery extends Query {
         if (fieldInfo.getPointNumBytes() != bytesPerDim) {
           throw new IllegalArgumentException("field=\"" + field + "\" was indexed with bytesPerDim=" + fieldInfo.getPointNumBytes() + " but this query has bytesPerDim=" + bytesPerDim);
         }
+        PointValues values = reader.getPointValues(field);
+        if (values == null) {
+          return null;
+        }
 
         FixedBitSet result = new FixedBitSet(reader.maxDoc());
         float[] scores = new float[reader.maxDoc()];
-        values.intersect(field, new MergePointVisitor(sortedPackedPoints, result, scores));
+        values.intersect(new MergePointVisitor(sortedPackedPoints, result, scores));
         return new Scorer(this) {
 
           DocIdSetIterator disi = new BitSetIterator(result, 10L);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
index 8db9c9e..ccbbf24 100644
--- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
+++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.Objects;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
@@ -1065,22 +1066,12 @@ public class MemoryIndex {
    */
   private final class MemoryIndexReader extends LeafReader {
 
-    private final PointValues pointValues;
     private Fields memoryFields = new MemoryFields(fields);
 
     private MemoryIndexReader() {
       super(); // avoid as much superclass baggage as possible
-      boolean hasPointValues = false;
       for (Info info : fields.values()) {
         info.prepareDocValuesAndPointValues();
-        if (info.pointValues != null) {
-          hasPointValues = true;
-        }
-      }
-      if (hasPointValues) {
-        pointValues = new MemoryIndexPointValues();
-      } else {
-        pointValues = null;
       }
     }
 
@@ -1198,8 +1189,12 @@ public class MemoryIndex {
     }
 
     @Override
-    public PointValues getPointValues() {
-      return pointValues;
+    public PointValues getPointValues(String fieldName) {
+      Info info = fields.get(fieldName);
+      if (info.pointValues == null) {
+        return null;
+      }
+      return new MemoryIndexPointValues(info);
     }
 
     @Override
@@ -1504,16 +1499,15 @@ public class MemoryIndex {
 
     private class MemoryIndexPointValues extends PointValues {
 
+      final Info info;
+
+      MemoryIndexPointValues(Info info) {
+        this.info = Objects.requireNonNull(info);
+      }
+
       @Override
-      public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
-        Info info = fields.get(fieldName);
-        if (info == null) {
-          return;
-        }
+      public void intersect(IntersectVisitor visitor) throws IOException {
         BytesRef[] values = info.pointValues;
-        if (values == null) {
-          return;
-        }
 
         visitor.grow(info.pointValuesCount);
         for (int i = 0; i < info.pointValuesCount; i++) {
@@ -1522,11 +1516,7 @@ public class MemoryIndex {
       }
 
       @Override
-      public byte[] getMinPackedValue(String fieldName) throws IOException {
-        Info info = fields.get(fieldName);
-        if (info == null) {
-          return null;
-        }
+      public byte[] getMinPackedValue() throws IOException {
         BytesRef[] values = info.pointValues;
         if (values != null) {
           return info.minPackedValue;
@@ -1536,63 +1526,28 @@ public class MemoryIndex {
       }
 
       @Override
-      public byte[] getMaxPackedValue(String fieldName) throws IOException {
-        Info info = fields.get(fieldName);
-        if (info == null) {
-          return null;
-        }
-        BytesRef[] values = info.pointValues;
-        if (values != null) {
-          return info.maxPackedValue;
-        } else {
-          return null;
-        }
+      public byte[] getMaxPackedValue() throws IOException {
+        return info.maxPackedValue;
       }
 
       @Override
-      public int getNumDimensions(String fieldName) throws IOException {
-        Info info = fields.get(fieldName);
-        if (info == null){
-          return 0;
-        }
+      public int getNumDimensions() throws IOException {
         return info.fieldInfo.getPointDimensionCount();
       }
 
       @Override
-      public int getBytesPerDimension(String fieldName) throws IOException {
-        Info info = fields.get(fieldName);
-        if (info == null){
-          return 0;
-        }
+      public int getBytesPerDimension() throws IOException {
         return info.fieldInfo.getPointNumBytes();
       }
 
       @Override
-      public long size(String fieldName) {
-        Info info = fields.get(fieldName);
-        if (info == null) {
-          return 0;
-        }
-        BytesRef[] values = info.pointValues;
-        if (values != null) {
-          return info.pointValuesCount;
-        } else {
-          return 0;
-        }
+      public long size() {
+        return info.pointValuesCount;
       }
 
       @Override
-      public int getDocCount(String fieldName) {
-        Info info = fields.get(fieldName);
-        if (info == null) {
-          return 0;
-        }
-        BytesRef[] values = info.pointValues;
-        if (values != null) {
-          return 1;
-        } else {
-          return 0;
-        }
+      public int getDocCount() {
+        return 1;
       }
 
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
index da3dd4c..ab4fe21 100644
--- a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
+++ b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
@@ -479,9 +479,9 @@ public class TestMemoryIndex extends LuceneTestCase {
     MemoryIndex mi = MemoryIndex.fromDocument(doc, analyzer);
     LeafReader leafReader = mi.createSearcher().getIndexReader().leaves().get(0).reader();
 
-    assertEquals(1, leafReader.getPointValues().size("field"));
-    assertArrayEquals(packedPoint, leafReader.getPointValues().getMinPackedValue("field"));
-    assertArrayEquals(packedPoint, leafReader.getPointValues().getMaxPackedValue("field"));
+    assertEquals(1, leafReader.getPointValues("field").size());
+    assertArrayEquals(packedPoint, leafReader.getPointValues("field").getMinPackedValue());
+    assertArrayEquals(packedPoint, leafReader.getPointValues("field").getMaxPackedValue());
 
     BinaryDocValues dvs = leafReader.getBinaryDocValues("field");
     assertEquals(0, dvs.nextDoc());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
index 19f80ab..1d3146e 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
-import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
 import org.apache.lucene.geo.GeoUtils;
 import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.index.FieldInfo;
@@ -307,13 +306,13 @@ public class LatLonPoint extends Field {
     List<Bits> liveDocs = new ArrayList<>();
     int totalHits = 0;
     for(LeafReaderContext leaf : searcher.getIndexReader().leaves()) {
-      PointValues points = leaf.reader().getPointValues();
+      PointValues points = leaf.reader().getPointValues(field);
       if (points != null) {
-        if (points instanceof Lucene60PointsReader == false) {
+        if (points instanceof BKDReader == false) {
           throw new IllegalArgumentException("can only run on Lucene60PointsReader points implementation, but got " + points);
         }
-        totalHits += points.getDocCount(field);
-        BKDReader reader = ((Lucene60PointsReader) points).getBKDReader(field);
+        totalHits += points.getDocCount();
+        BKDReader reader = (BKDReader) points;
         if (reader != null) {
           readers.add(reader);
           docBases.add(leaf.docBase);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
index 1b20d95..7a00cef 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
@@ -105,7 +105,7 @@ final class LatLonPointDistanceQuery extends Query {
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
         LeafReader reader = context.reader();
-        PointValues values = reader.getPointValues();
+        PointValues values = reader.getPointValues(field);
         if (values == null) {
           // No docs in this segment had any points fields
           return null;
@@ -120,7 +120,7 @@ final class LatLonPointDistanceQuery extends Query {
         // matching docids
         DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
 
-        values.intersect(field,
+        values.intersect(
                          new IntersectVisitor() {
 
                            DocIdSetBuilder.BulkAdder adder;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
index 036fe2c..ec7c682 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
@@ -98,7 +98,7 @@ final class LatLonPointInPolygonQuery extends Query {
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
         LeafReader reader = context.reader();
-        PointValues values = reader.getPointValues();
+        PointValues values = reader.getPointValues(field);
         if (values == null) {
           // No docs in this segment had any points fields
           return null;
@@ -113,7 +113,7 @@ final class LatLonPointInPolygonQuery extends Query {
         // matching docids
         DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
 
-        values.intersect(field, 
+        values.intersect( 
                          new IntersectVisitor() {
 
                            DocIdSetBuilder.BulkAdder adder;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
index 1fa894f..7ebdec4 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
@@ -110,7 +110,7 @@ abstract class RangeFieldQuery extends Query {
       final RangeFieldComparator comparator = new RangeFieldComparator();
       private DocIdSet buildMatchingDocIdSet(LeafReader reader, PointValues values) throws IOException {
         DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
-        values.intersect(field,
+        values.intersect(
             new IntersectVisitor() {
               DocIdSetBuilder.BulkAdder adder;
               @Override
@@ -157,7 +157,7 @@ abstract class RangeFieldQuery extends Query {
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
         LeafReader reader = context.reader();
-        PointValues values = reader.getPointValues();
+        PointValues values = reader.getPointValues(field);
         if (values == null) {
           // no docs in this segment indexed any ranges
           return null;
@@ -168,9 +168,9 @@ abstract class RangeFieldQuery extends Query {
         }
         checkFieldInfo(fieldInfo);
         boolean allDocsMatch = true;
-        if (values.getDocCount(field) == reader.maxDoc()) {
+        if (values.getDocCount() == reader.maxDoc()) {
           // if query crosses, docs need to be further scrutinized
-          byte[] range = getInternalRange(values.getMinPackedValue(field), values.getMaxPackedValue(field));
+          byte[] range = getInternalRange(values.getMinPackedValue(), values.getMaxPackedValue());
           // if the internal node is not equal and not contained by the query, all docs do not match
           if ((!Arrays.equals(ranges, range)
               && (comparator.contains(range) && queryType != QueryType.CONTAINS)) == false) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInGeo3DShapeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInGeo3DShapeQuery.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInGeo3DShapeQuery.java
index 105e5d8..56cb45d 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInGeo3DShapeQuery.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInGeo3DShapeQuery.java
@@ -70,7 +70,7 @@ final class PointInGeo3DShapeQuery extends Query {
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
         LeafReader reader = context.reader();
-        PointValues values = reader.getPointValues();
+        PointValues values = reader.getPointValues(field);
         if (values == null) {
           return null;
         }
@@ -99,7 +99,7 @@ final class PointInGeo3DShapeQuery extends Query {
 
         DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
 
-        values.intersect(field, new PointInShapeIntersectVisitor(result, shape, shapeBounds));
+        values.intersect(new PointInShapeIntersectVisitor(result, shape, shapeBounds));
 
         return new ConstantScoreScorer(this, score(), result.build().iterator());
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
index 97606c7..2af096a 100644
--- a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
+++ b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/TestGeo3DPoint.java
@@ -1486,11 +1486,11 @@ public class TestGeo3DPoint extends LuceneTestCase {
       docID - reader.leaves().get(subIndex).docBase, 3, Integer.BYTES, b);
 
     // Do first phase, where we just figure out the "path" that leads to the target docID:
-    leafReader.getPointValues().intersect(fieldName, visitor);
+    leafReader.getPointValues(fieldName).intersect(visitor);
 
     // Do second phase, where we we see how the wrapped visitor responded along that path:
     visitor.startSecondPhase();
-    leafReader.getPointValues().intersect(fieldName, visitor);
+    leafReader.getPointValues(fieldName).intersect(visitor);
 
     return b.toString();
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
index c6f5485..ff2e1b6 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointsFormat.java
@@ -22,16 +22,13 @@ import java.util.Collection;
 import org.apache.lucene.codecs.PointsFormat;
 import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.PointsWriter;
+import org.apache.lucene.index.AssertingLeafReader;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.MergeState;
-import org.apache.lucene.index.PointValues.IntersectVisitor;
-import org.apache.lucene.index.PointValues.Relation;
 import org.apache.lucene.index.PointValues;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.TestUtil;
 
 /**
@@ -66,88 +63,6 @@ public final class AssertingPointsFormat extends PointsFormat {
     return new AssertingPointsReader(state.segmentInfo.maxDoc(), in.fieldsReader(state));
   }
 
-  /** Validates in the 1D case that all points are visited in order, and point values are in bounds of the last cell checked */
-  static class AssertingIntersectVisitor implements IntersectVisitor {
-    final IntersectVisitor in;
-    final int numDims;
-    final int bytesPerDim;
-    final byte[] lastDocValue;
-    final byte[] lastMinPackedValue;
-    final byte[] lastMaxPackedValue;
-    private Relation lastCompareResult;
-    private int lastDocID = -1;
-    private int docBudget;
-
-    public AssertingIntersectVisitor(int numDims, int bytesPerDim, IntersectVisitor in) {
-      this.in = in;
-      this.numDims = numDims;
-      this.bytesPerDim = bytesPerDim;
-      lastMaxPackedValue = new byte[numDims*bytesPerDim];
-      lastMinPackedValue = new byte[numDims*bytesPerDim];
-      if (numDims == 1) {
-        lastDocValue = new byte[bytesPerDim];
-      } else {
-        lastDocValue = null;
-      }
-    }
-
-    @Override
-    public void visit(int docID) throws IOException {
-      assert --docBudget >= 0 : "called add() more times than the last call to grow() reserved";
-
-      // This method, not filtering each hit, should only be invoked when the cell is inside the query shape:
-      assert lastCompareResult == Relation.CELL_INSIDE_QUERY;
-      in.visit(docID);
-    }
-
-    @Override
-    public void visit(int docID, byte[] packedValue) throws IOException {
-      assert --docBudget >= 0 : "called add() more times than the last call to grow() reserved";
-
-      // This method, to filter each doc's value, should only be invoked when the cell crosses the query shape:
-      assert lastCompareResult == PointValues.Relation.CELL_CROSSES_QUERY;
-
-      // This doc's packed value should be contained in the last cell passed to compare:
-      for(int dim=0;dim<numDims;dim++) {
-        assert StringHelper.compare(bytesPerDim, lastMinPackedValue, dim*bytesPerDim, packedValue, dim*bytesPerDim) <= 0: "dim=" + dim + " of " +  numDims + " value=" + new BytesRef(packedValue);
-        assert StringHelper.compare(bytesPerDim, lastMaxPackedValue, dim*bytesPerDim, packedValue, dim*bytesPerDim) >= 0: "dim=" + dim + " of " +  numDims + " value=" + new BytesRef(packedValue);
-      }
-
-      // TODO: we should assert that this "matches" whatever relation the last call to compare had returned
-      assert packedValue.length == numDims * bytesPerDim;
-      if (numDims == 1) {
-        int cmp = StringHelper.compare(bytesPerDim, lastDocValue, 0, packedValue, 0);
-        if (cmp < 0) {
-          // ok
-        } else if (cmp == 0) {
-          assert lastDocID <= docID: "doc ids are out of order when point values are the same!";
-        } else {
-          // out of order!
-          assert false: "point values are out of order";
-        }
-        System.arraycopy(packedValue, 0, lastDocValue, 0, bytesPerDim);
-        lastDocID = docID;
-      }
-      in.visit(docID, packedValue);
-    }
-
-    @Override
-    public void grow(int count) {
-      in.grow(count);
-      docBudget = count;
-    }
-
-    @Override
-    public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-      for(int dim=0;dim<numDims;dim++) {
-        assert StringHelper.compare(bytesPerDim, minPackedValue, dim*bytesPerDim, maxPackedValue, dim*bytesPerDim) <= 0;
-      }
-      System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, numDims*bytesPerDim);
-      System.arraycopy(minPackedValue, 0, lastMinPackedValue, 0, numDims*bytesPerDim);
-      lastCompareResult = in.compare(minPackedValue, maxPackedValue);
-      return lastCompareResult;
-    }
-  }
   
   static class AssertingPointsReader extends PointsReader {
     private final PointsReader in;
@@ -169,9 +84,12 @@ public final class AssertingPointsFormat extends PointsFormat {
     }
 
     @Override
-    public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
-      in.intersect(fieldName,
-                   new AssertingIntersectVisitor(in.getNumDimensions(fieldName), in.getBytesPerDimension(fieldName), visitor));
+    public PointValues getValues(String field) throws IOException {
+      PointValues values = this.in.getValues(field);
+      if (values == null) {
+        return null;
+      }
+      return new AssertingLeafReader.AssertingPointValues(values, maxDoc);
     }
 
     @Override
@@ -202,49 +120,6 @@ public final class AssertingPointsFormat extends PointsFormat {
     public String toString() {
       return getClass().getSimpleName() + "(" + in.toString() + ")";
     }
-
-    @Override
-    public byte[] getMinPackedValue(String fieldName) throws IOException {
-      assertStats(fieldName);
-      return in.getMinPackedValue(fieldName);
-    }
-
-    @Override
-    public byte[] getMaxPackedValue(String fieldName) throws IOException {
-      assertStats(fieldName);
-      return in.getMaxPackedValue(fieldName);
-    }
-
-    @Override
-    public int getNumDimensions(String fieldName) throws IOException {
-      assertStats(fieldName);
-      return in.getNumDimensions(fieldName);
-    }
-
-    @Override
-    public int getBytesPerDimension(String fieldName) throws IOException {
-      assertStats(fieldName);
-      return in.getBytesPerDimension(fieldName);
-    }
-
-    @Override
-    public long size(String fieldName) {
-      assertStats(fieldName);
-      return in.size(fieldName);
-    }
-
-    @Override
-    public int getDocCount(String fieldName) {
-      assertStats(fieldName);
-      return in.getDocCount(fieldName);
-    }
-
-    private void assertStats(String fieldName) {
-      assert in.size(fieldName) >= 0;
-      assert in.getDocCount(fieldName) >= 0;
-      assert in.getDocCount(fieldName) <= in.size(fieldName);
-      assert in.getDocCount(fieldName) <= maxDoc;
-    }
   }
 
   static class AssertingPointsWriter extends PointsWriter {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
index fd2260b..ec7d75a 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyPointsFormat.java
@@ -24,6 +24,7 @@ import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.PointsWriter;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.PointValues;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 
@@ -114,46 +115,67 @@ class CrankyPointsFormat extends PointsFormat {
     }
 
     @Override
-    public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
-      if (random.nextInt(100) == 0) {
-        throw new IOException("Fake IOException");
-      }
-      delegate.intersect(fieldName, visitor);
-      if (random.nextInt(100) == 0) {
-        throw new IOException("Fake IOException");
-      }  
-    }
-
-    @Override
-    public byte[] getMinPackedValue(String fieldName) throws IOException {
-      if (random.nextInt(100) == 0) {
-        throw new IOException("Fake IOException");
+    public PointValues getValues(String fieldName) throws IOException {
+      final PointValues delegate = this.delegate.getValues(fieldName);
+      if (delegate == null) {
+        return null;
       }
-      return delegate.getMinPackedValue(fieldName);
-    }
-
-    @Override
-    public byte[] getMaxPackedValue(String fieldName) throws IOException {
-      if (random.nextInt(100) == 0) {
-        throw new IOException("Fake IOException");
-      }
-      return delegate.getMaxPackedValue(fieldName);
-    }
-
-    @Override
-    public int getNumDimensions(String fieldName) throws IOException {
-      if (random.nextInt(100) == 0) {
-        throw new IOException("Fake IOException");
-      }
-      return delegate.getNumDimensions(fieldName);
-    }
-
-    @Override
-    public int getBytesPerDimension(String fieldName) throws IOException {
-      if (random.nextInt(100) == 0) {
-        throw new IOException("Fake IOException");
-      }
-      return delegate.getBytesPerDimension(fieldName);
+      return new PointValues() {
+
+        @Override
+        public void intersect(IntersectVisitor visitor) throws IOException {
+          if (random.nextInt(100) == 0) {
+            throw new IOException("Fake IOException");
+          }
+          delegate.intersect(visitor);
+          if (random.nextInt(100) == 0) {
+            throw new IOException("Fake IOException");
+          }  
+        }
+
+        @Override
+        public byte[] getMinPackedValue() throws IOException {
+          if (random.nextInt(100) == 0) {
+            throw new IOException("Fake IOException");
+          }
+          return delegate.getMinPackedValue();
+        }
+
+        @Override
+        public byte[] getMaxPackedValue() throws IOException {
+          if (random.nextInt(100) == 0) {
+            throw new IOException("Fake IOException");
+          }
+          return delegate.getMaxPackedValue();
+        }
+
+        @Override
+        public int getNumDimensions() throws IOException {
+          if (random.nextInt(100) == 0) {
+            throw new IOException("Fake IOException");
+          }
+          return delegate.getNumDimensions();
+        }
+
+        @Override
+        public int getBytesPerDimension() throws IOException {
+          if (random.nextInt(100) == 0) {
+            throw new IOException("Fake IOException");
+          }
+          return delegate.getBytesPerDimension();
+        }
+
+        @Override
+        public long size() {
+          return delegate.size();
+        }
+
+        @Override
+        public int getDocCount() {
+          return delegate.getDocCount();
+        }
+
+      };
     }
 
     @Override
@@ -168,15 +190,5 @@ class CrankyPointsFormat extends PointsFormat {
     public long ramBytesUsed() {
       return delegate.ramBytesUsed();
     }
-
-    @Override
-    public long size(String fieldName) {
-      return delegate.size(fieldName);
-    }
-
-    @Override
-    public int getDocCount(String fieldName) {
-      return delegate.getDocCount(fieldName);
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java b/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
index b2958c8..6686ec4 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
@@ -18,10 +18,14 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.Iterator;
+import java.util.Objects;
 
+import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.index.PointValues.Relation;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.VirtualMethod;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
 
@@ -776,7 +780,145 @@ public class AssertingLeafReader extends FilterLeafReader {
       return result;
     }
   }
-  
+
+  /** Wraps a SortedSetDocValues but with additional asserts */
+  public static class AssertingPointValues extends PointValues {
+
+    private final PointValues in;
+
+    /** Sole constructor. */
+    public AssertingPointValues(PointValues in, int maxDoc) {
+      this.in = in;
+      assertStats(maxDoc);
+    }
+
+    private void assertStats(int maxDoc) {
+      assert in.size() > 0;
+      assert in.getDocCount() > 0;
+      assert in.getDocCount() <= in.size();
+      assert in.getDocCount() <= maxDoc;
+    }
+
+    @Override
+    public void intersect(IntersectVisitor visitor) throws IOException {
+      in.intersect(new AssertingIntersectVisitor(in.getNumDimensions(), in.getBytesPerDimension(), visitor));
+    }
+
+    @Override
+    public byte[] getMinPackedValue() throws IOException {
+      return Objects.requireNonNull(in.getMinPackedValue());
+    }
+
+    @Override
+    public byte[] getMaxPackedValue() throws IOException {
+      return Objects.requireNonNull(in.getMaxPackedValue());
+    }
+
+    @Override
+    public int getNumDimensions() throws IOException {
+      return in.getNumDimensions();
+    }
+
+    @Override
+    public int getBytesPerDimension() throws IOException {
+      return in.getBytesPerDimension();
+    }
+
+    @Override
+    public long size() {
+      return in.size();
+    }
+
+    @Override
+    public int getDocCount() {
+      return in.getDocCount();
+    }
+
+  }
+
+  /** Validates in the 1D case that all points are visited in order, and point values are in bounds of the last cell checked */
+  static class AssertingIntersectVisitor implements IntersectVisitor {
+    final IntersectVisitor in;
+    final int numDims;
+    final int bytesPerDim;
+    final byte[] lastDocValue;
+    final byte[] lastMinPackedValue;
+    final byte[] lastMaxPackedValue;
+    private Relation lastCompareResult;
+    private int lastDocID = -1;
+    private int docBudget;
+
+    AssertingIntersectVisitor(int numDims, int bytesPerDim, IntersectVisitor in) {
+      this.in = in;
+      this.numDims = numDims;
+      this.bytesPerDim = bytesPerDim;
+      lastMaxPackedValue = new byte[numDims*bytesPerDim];
+      lastMinPackedValue = new byte[numDims*bytesPerDim];
+      if (numDims == 1) {
+        lastDocValue = new byte[bytesPerDim];
+      } else {
+        lastDocValue = null;
+      }
+    }
+
+    @Override
+    public void visit(int docID) throws IOException {
+      assert --docBudget >= 0 : "called add() more times than the last call to grow() reserved";
+
+      // This method, not filtering each hit, should only be invoked when the cell is inside the query shape:
+      assert lastCompareResult == Relation.CELL_INSIDE_QUERY;
+      in.visit(docID);
+    }
+
+    @Override
+    public void visit(int docID, byte[] packedValue) throws IOException {
+      assert --docBudget >= 0 : "called add() more times than the last call to grow() reserved";
+
+      // This method, to filter each doc's value, should only be invoked when the cell crosses the query shape:
+      assert lastCompareResult == PointValues.Relation.CELL_CROSSES_QUERY;
+
+      // This doc's packed value should be contained in the last cell passed to compare:
+      for(int dim=0;dim<numDims;dim++) {
+        assert StringHelper.compare(bytesPerDim, lastMinPackedValue, dim*bytesPerDim, packedValue, dim*bytesPerDim) <= 0: "dim=" + dim + " of " +  numDims + " value=" + new BytesRef(packedValue);
+        assert StringHelper.compare(bytesPerDim, lastMaxPackedValue, dim*bytesPerDim, packedValue, dim*bytesPerDim) >= 0: "dim=" + dim + " of " +  numDims + " value=" + new BytesRef(packedValue);
+      }
+
+      // TODO: we should assert that this "matches" whatever relation the last call to compare had returned
+      assert packedValue.length == numDims * bytesPerDim;
+      if (numDims == 1) {
+        int cmp = StringHelper.compare(bytesPerDim, lastDocValue, 0, packedValue, 0);
+        if (cmp < 0) {
+          // ok
+        } else if (cmp == 0) {
+          assert lastDocID <= docID: "doc ids are out of order when point values are the same!";
+        } else {
+          // out of order!
+          assert false: "point values are out of order";
+        }
+        System.arraycopy(packedValue, 0, lastDocValue, 0, bytesPerDim);
+        lastDocID = docID;
+      }
+      in.visit(docID, packedValue);
+    }
+
+    @Override
+    public void grow(int count) {
+      in.grow(count);
+      docBudget = count;
+    }
+
+    @Override
+    public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+      for(int dim=0;dim<numDims;dim++) {
+        assert StringHelper.compare(bytesPerDim, minPackedValue, dim*bytesPerDim, maxPackedValue, dim*bytesPerDim) <= 0;
+      }
+      System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, numDims*bytesPerDim);
+      System.arraycopy(minPackedValue, 0, lastMinPackedValue, 0, numDims*bytesPerDim);
+      lastCompareResult = in.compare(minPackedValue, maxPackedValue);
+      return lastCompareResult;
+    }
+  }
+
   @Override
   public NumericDocValues getNumericDocValues(String field) throws IOException {
     NumericDocValues dv = super.getNumericDocValues(field);
@@ -860,7 +1002,16 @@ public class AssertingLeafReader extends FilterLeafReader {
       return null;
     }
   }
-  
+
+  @Override
+  public PointValues getPointValues(String field) throws IOException {
+    PointValues values = in.getPointValues(field);
+    if (values == null) {
+      return null;
+    }
+    return new AssertingPointValues(values, maxDoc());
+  }
+
   /** Wraps a Bits but with additional asserts */
   public static class AssertingBits implements Bits {
     private final Thread creationThread = Thread.currentThread();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java
index 540d3f9..6014fa5 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BasePointsFormatTestCase.java
@@ -80,11 +80,11 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
 
     DirectoryReader r = DirectoryReader.open(dir);
     LeafReader sub = getOnlyLeafReader(r);
-    PointValues values = sub.getPointValues();
+    PointValues values = sub.getPointValues("dim");
 
     // Simple test: make sure intersect can visit every doc:
     BitSet seen = new BitSet();
-    values.intersect("dim",
+    values.intersect(
                      new IntersectVisitor() {
                        @Override
                        public Relation compare(byte[] minPacked, byte[] maxPacked) {
@@ -122,11 +122,11 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
 
     DirectoryReader r = DirectoryReader.open(dir);
     LeafReader sub = getOnlyLeafReader(r);
-    PointValues values = sub.getPointValues();
+    PointValues values = sub.getPointValues("dim");
 
     // Simple test: make sure intersect can visit every doc:
     BitSet seen = new BitSet();
-    values.intersect("dim",
+    values.intersect(
                      new IntersectVisitor() {
                        @Override
                        public Relation compare(byte[] minPacked, byte[] maxPacked) {
@@ -168,7 +168,7 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
     Bits liveDocs = MultiFields.getLiveDocs(r);
 
     for(LeafReaderContext ctx : r.leaves()) {
-      PointValues values = ctx.reader().getPointValues();
+      PointValues values = ctx.reader().getPointValues("dim");
 
       NumericDocValues idValues = ctx.reader().getNumericDocValues("id");
       if (idValues == null) {
@@ -184,7 +184,7 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
       
       if (values != null) {
         BitSet seen = new BitSet();
-        values.intersect("dim",
+        values.intersect(
                          new IntersectVisitor() {
                            @Override
                            public Relation compare(byte[] minPacked, byte[] maxPacked) {
@@ -430,14 +430,14 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
 
         final BitSet hits = new BitSet();
         for(LeafReaderContext ctx : r.leaves()) {
-          PointValues dimValues = ctx.reader().getPointValues();
+          PointValues dimValues = ctx.reader().getPointValues("field");
           if (dimValues == null) {
             continue;
           }
 
           final int docBase = ctx.docBase;
           
-          dimValues.intersect("field", new IntersectVisitor() {
+          dimValues.intersect(new IntersectVisitor() {
               @Override
               public void visit(int docID) {
                 hits.set(docBase+docID);
@@ -735,13 +735,13 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
       byte[] maxValues = new byte[numDims*numBytesPerDim];
 
       for(LeafReaderContext ctx : r.leaves()) {
-        PointValues dimValues = ctx.reader().getPointValues();
+        PointValues dimValues = ctx.reader().getPointValues("field");
         if (dimValues == null) {
           continue;
         }
 
-        byte[] leafMinValues = dimValues.getMinPackedValue("field");
-        byte[] leafMaxValues = dimValues.getMaxPackedValue("field");
+        byte[] leafMinValues = dimValues.getMinPackedValue();
+        byte[] leafMaxValues = dimValues.getMaxPackedValue();
         for(int dim=0;dim<numDims;dim++) {
           if (StringHelper.compare(numBytesPerDim, leafMinValues, dim*numBytesPerDim, minValues, dim*numBytesPerDim) < 0) {
             System.arraycopy(leafMinValues, dim*numBytesPerDim, minValues, dim*numBytesPerDim, numBytesPerDim);
@@ -792,14 +792,14 @@ public abstract class BasePointsFormatTestCase extends BaseIndexFileFormatTestCa
         final BitSet hits = new BitSet();
 
         for(LeafReaderContext ctx : r.leaves()) {
-          PointValues dimValues = ctx.reader().getPointValues();
+          PointValues dimValues = ctx.reader().getPointValues("field");
           if (dimValues == null) {
             continue;
           }
 
           final int docBase = ctx.docBase;
 
-          dimValues.intersect("field", new PointValues.IntersectVisitor() {
+          dimValues.intersect(new PointValues.IntersectVisitor() {
               @Override
               public void visit(int docID) {
                 if (liveDocs == null || liveDocs.get(docBase+docID)) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
index 127549f..161d1a9 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
@@ -105,9 +105,10 @@ public class RandomCodec extends AssertingCodec {
 
         return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap) {
           @Override
-          public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
+          public void writeField(FieldInfo fieldInfo, PointsReader reader) throws IOException {
 
-            boolean singleValuePerDoc = values.size(fieldInfo.name) == values.getDocCount(fieldInfo.name);
+            PointValues values = reader.getValues(fieldInfo.name);
+            boolean singleValuePerDoc = values.size() == values.getDocCount();
 
             try (BKDWriter writer = new RandomlySplittingBKDWriter(writeState.segmentInfo.maxDoc(),
                                                                    writeState.directory,
@@ -116,10 +117,10 @@ public class RandomCodec extends AssertingCodec {
                                                                    fieldInfo.getPointNumBytes(),
                                                                    maxPointsInLeafNode,
                                                                    maxMBSortInHeap,
-                                                                   values.size(fieldInfo.name),
+                                                                   values.size(),
                                                                    singleValuePerDoc,
                                                                    bkdSplitRandomSeed ^ fieldInfo.name.hashCode())) {
-                values.intersect(fieldInfo.name, new IntersectVisitor() {
+                values.intersect(new IntersectVisitor() {
                     @Override
                     public void visit(int docID) {
                       throw new IllegalStateException();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
index 4fd8fab..a3eaa80 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
@@ -262,7 +262,7 @@ public class QueryUtils {
       }
 
       @Override
-      public PointValues getPointValues() {
+      public PointValues getPointValues(String fieldName) {
         return null;
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index e62c563..e9dc5ab 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -2601,12 +2601,12 @@ public abstract class LuceneTestCase extends Assert {
     final Map<Integer,Set<BytesRef>> docValues = new HashMap<>();
     for(LeafReaderContext ctx : reader.leaves()) {
 
-      PointValues points = ctx.reader().getPointValues();
+      PointValues points = ctx.reader().getPointValues(fieldName);
       if (points == null) {
         continue;
       }
 
-      points.intersect(fieldName,
+      points.intersect(
                        new PointValues.IntersectVisitor() {
                          @Override
                          public void visit(int docID) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/solr/core/src/java/org/apache/solr/index/SlowCompositeReaderWrapper.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/index/SlowCompositeReaderWrapper.java b/solr/core/src/java/org/apache/solr/index/SlowCompositeReaderWrapper.java
index 7c7098c..5031faf 100644
--- a/solr/core/src/java/org/apache/solr/index/SlowCompositeReaderWrapper.java
+++ b/solr/core/src/java/org/apache/solr/index/SlowCompositeReaderWrapper.java
@@ -237,7 +237,7 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
   }
 
   @Override
-  public PointValues getPointValues() {
+  public PointValues getPointValues(String field) {
     ensureOpen();
     return null;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/solr/core/src/java/org/apache/solr/uninverting/FieldCacheImpl.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/uninverting/FieldCacheImpl.java b/solr/core/src/java/org/apache/solr/uninverting/FieldCacheImpl.java
index f06f7e3..b63e5e9 100644
--- a/solr/core/src/java/org/apache/solr/uninverting/FieldCacheImpl.java
+++ b/solr/core/src/java/org/apache/solr/uninverting/FieldCacheImpl.java
@@ -277,12 +277,12 @@ class FieldCacheImpl implements FieldCache {
     
     final void uninvertPoints(LeafReader reader, String field) throws IOException {
       final int maxDoc = reader.maxDoc();
-      PointValues values = reader.getPointValues();
+      PointValues values = reader.getPointValues(field);
       assert values != null;
-      assert values.size(field) > 0;
+      assert values.size() > 0;
 
       final boolean setDocsWithField;
-      final int docCount = values.getDocCount(field);
+      final int docCount = values.getDocCount();
       assert docCount <= maxDoc;
       if (docCount == maxDoc) {
         // Fast case: all docs have this field:
@@ -293,7 +293,7 @@ class FieldCacheImpl implements FieldCache {
       }
 
       BytesRef scratch = new BytesRef();
-      values.intersect(field, new IntersectVisitor() {
+      values.intersect(new IntersectVisitor() {
         @Override
         public void visit(int docID) throws IOException { 
           throw new AssertionError(); 
@@ -512,11 +512,11 @@ class FieldCacheImpl implements FieldCache {
 
     private BitsEntry createValuePoints(LeafReader reader, String field) throws IOException {
       final int maxDoc = reader.maxDoc();
-      PointValues values = reader.getPointValues();
+      PointValues values = reader.getPointValues(field);
       assert values != null;
-      assert values.size(field) > 0;
+      assert values.size() > 0;
       
-      final int docCount = values.getDocCount(field);
+      final int docCount = values.getDocCount();
       assert docCount <= maxDoc;
       if (docCount == maxDoc) {
         // Fast case: all docs have this field:
@@ -615,14 +615,14 @@ class FieldCacheImpl implements FieldCache {
         if (info.getPointDimensionCount() != 1) {
           throw new IllegalStateException("Type mismatch: " + field + " was indexed with dimensions=" + info.getPointDimensionCount());
         }
-        PointValues values = reader.getPointValues();
+        PointValues values = reader.getPointValues(field);
         // no actual points for this field (e.g. all points deleted)
-        if (values == null || values.size(field) == 0) {
+        if (values == null || values.size() == 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));
+        if (values.size() != values.getDocCount()) {
+          throw new IllegalStateException("Type mismatch: " + field + " was indexed with multiple values, numValues=" + values.size() + ",numDocs=" + values.getDocCount());
         }
       } else {
         // postings case 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c22725f0/solr/core/src/test/org/apache/solr/search/TestDocSet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestDocSet.java b/solr/core/src/test/org/apache/solr/search/TestDocSet.java
index 5b44514..2849f09 100644
--- a/solr/core/src/test/org/apache/solr/search/TestDocSet.java
+++ b/solr/core/src/test/org/apache/solr/search/TestDocSet.java
@@ -448,7 +448,7 @@ public class TestDocSet extends LuceneTestCase {
       }
 
       @Override
-      public PointValues getPointValues() {
+      public PointValues getPointValues(String field) {
         return null;
       }