You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2016/03/02 17:25:21 UTC

lucene-solr git commit: LUCENE-7059: always visit 1D points in sorted order; fix tie-break but in BKDWriter; fix BKDWriter to pass on maxMBSortInHeap to the OfflineSorter too

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 8bbb1dcac -> 3cbc48ed3


LUCENE-7059: always visit 1D points in sorted order; fix tie-break but in BKDWriter; fix BKDWriter to pass on maxMBSortInHeap to the OfflineSorter too


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

Branch: refs/heads/branch_6x
Commit: 3cbc48ed3085ed2bac5096d6828a7f129003619d
Parents: 8bbb1dc
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Mar 2 11:25:27 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Mar 2 11:25:41 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/index/CheckIndex.java     |  23 +-
 .../org/apache/lucene/index/PointValues.java    |   4 +-
 .../apache/lucene/index/PointValuesWriter.java  |   1 +
 .../index/SlowCompositeReaderWrapper.java       |   2 +-
 .../apache/lucene/search/PointInSetQuery.java   |  13 -
 .../org/apache/lucene/util/OfflineSorter.java   |   2 +-
 .../org/apache/lucene/util/bkd/BKDWriter.java   |  41 ++-
 .../apache/lucene/index/TestDuelingCodecs.java  |   4 +-
 .../apache/lucene/index/TestPointValues.java    |  62 +++++
 .../apache/lucene/search/TestPointQueries.java  |  37 +--
 .../facet/range/TestRangeFacetCounts.java       |   8 +-
 .../org/apache/lucene/index/SorterTestBase.java |   6 +-
 .../lucene/document/TestBigIntegerPoint.java    |   4 +-
 .../lucene/document/TestInetAddressPoint.java   |   4 +-
 .../apache/lucene/document/TestLatLonPoint.java |   2 +-
 .../document/TestLatLonPointDistanceQuery.java  |   2 +-
 .../lucene/search/TestDocValuesRangeQuery.java  |   4 +-
 .../codecs/asserting/AssertingPointFormat.java  |  86 +++++-
 .../lucene/index/BasePointFormatTestCase.java   | 262 +++++++++++--------
 .../org/apache/lucene/util/LuceneTestCase.java  | 111 ++++----
 20 files changed, 440 insertions(+), 238 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index 0c24461..9f711df 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -1706,8 +1706,12 @@ public final class CheckIndex implements Closeable {
             lastMaxPacked.length = bytesPerDim;
             lastMinPacked.length = bytesPerDim;
             scratch.length = bytesPerDim;
+            byte[] lastPackedValue = new byte[dimCount*bytesPerDim];
             values.intersect(fieldInfo.name,
                              new PointValues.IntersectVisitor() {
+
+                               private int lastDocID = -1;
+
                                @Override
                                public void visit(int docID) {
                                  throw new RuntimeException("codec called IntersectVisitor.visit without a packed value for docID=" + docID);
@@ -1725,12 +1729,27 @@ public final class CheckIndex implements Closeable {
 
                                    if (scratch.compareTo(lastMinPacked) < 0) {
                                      // This doc's point, in this dimension, is lower than the minimum value of the last cell checked:
-                                     throw new RuntimeException("packed value " + Arrays.toString(packedValue) + " for docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
+                                     throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
                                    }
                                    if (scratch.compareTo(lastMaxPacked) > 0) {
                                      // This doc's point, in this dimension, is greater than the maximum value of the last cell checked:
-                                     throw new RuntimeException("packed value " + Arrays.toString(packedValue) + " for docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
+                                     throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
+                                   }
+                                 }
+
+                                 // In the 1D case, PointValues must make a single in-order sweep through all values, and tie-break by
+                                 // increasing docID:
+                                 if (dimCount == 1) {
+                                   int cmp = StringHelper.compare(bytesPerDim, lastPackedValue, 0, packedValue, 0);
+                                   if (cmp > 0) {
+                                     throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for docID=" + docID + " is out-of-order vs the previous document's value " + Arrays.toString(lastPackedValue));
+                                   } else if (cmp == 0) {
+                                     if (docID < lastDocID) {
+                                       throw new RuntimeException("packed points value is the same, but docID=" + docID + " is out of order vs previous docID=" + lastDocID);
+                                     }
                                    }
+                                   System.arraycopy(packedValue, 0, lastPackedValue, 0, bytesPerDim);
+                                   lastDocID = docID;
                                  }
 
                                  status.totalValuePoints++;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/core/src/java/org/apache/lucene/index/PointValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValues.java b/lucene/core/src/java/org/apache/lucene/index/PointValues.java
index 108dde3..34d315f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValues.java
@@ -61,7 +61,9 @@ public abstract class PointValues {
     void visit(int docID) throws IOException;
 
     /** Called for all documents in a leaf cell that crosses the query.  The consumer
-     *  should scrutinize the packedValue to decide whether to accept it. */
+     *  should scrutinize the packedValue to decide whether to accept it.  In the 1D case,
+     *  values are visited in increasing order, and in the case of ties, in increasing
+     *  docID order. */
     void visit(int docID, byte[] packedValue) throws IOException;
 
     /** Called for non-leaf cells to test how the cell relates to the query, to

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
index 283f7bd..2fa8b4f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
@@ -43,6 +43,7 @@ class PointValuesWriter {
     packedValue = new byte[fieldInfo.getPointDimensionCount() * fieldInfo.getPointNumBytes()];
   }
 
+  // TODO: if exactly the same value is added to exactly the same doc, should we dedup?
   public void addPackedValue(int docID, BytesRef value) {
     if (value == null) {
       throw new IllegalArgumentException("field=" + fieldInfo.name + ": point value cannot be null");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java b/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
index db9579c..e44c53c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
@@ -236,7 +236,7 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
   @Override
   public PointValues getPointValues() {
     ensureOpen();
-    return MultiPointValues.get(in);
+    return null;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index 42f5bb1..3d6086c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -172,10 +172,6 @@ public abstract class PointInSetQuery extends Query {
       this.sortedPackedPoints = sortedPackedPoints;
       lastMaxPackedValue = new byte[bytesPerDim];
       scratch.length = bytesPerDim;
-      resetIterator();
-    }
-
-    private void resetIterator() {
       this.iterator = sortedPackedPoints.iterator();
       nextQueryPoint = iterator.next();
     }
@@ -211,15 +207,6 @@ public abstract class PointInSetQuery extends Query {
 
     @Override
     public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-      
-      // NOTE: this is messy ... we need it in cases where a single vistor (us) is shared across multiple leaf readers
-      // (e.g. SlowCompositeReaderWrapper), in which case we need to reset our iterator to re-start the merge sort.  Maybe we should instead
-      // add an explicit .start() to IntersectVisitor, and clarify the semantics that in the 1D case all cells will be visited in order?
-      if (StringHelper.compare(bytesPerDim, lastMaxPackedValue, 0, minPackedValue, 0) > 0) {    
-        resetIterator();
-      }
-      System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, bytesPerDim);
-
       while (nextQueryPoint != null) {
         scratch.bytes = minPackedValue;
         int cmpMin = nextQueryPoint.compareTo(scratch);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
index b146102..283dc1f 100644
--- a/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
@@ -120,7 +120,7 @@ public class OfflineSorter {
       final long minBufferSizeBytes = MIN_BUFFER_SIZE_MB*MB;
       if (sortBufferByteSize <  minBufferSizeBytes
           || totalAvailableBytes > 10 * minBufferSizeBytes) { // lets see if we need/should to grow the heap 
-        if (totalAvailableBytes/2 > minBufferSizeBytes){ // there is enough mem for a reasonable buffer
+        if (totalAvailableBytes/2 > minBufferSizeBytes) { // there is enough mem for a reasonable buffer
           sortBufferByteSize = totalAvailableBytes/2; // grow the heap
         } else {
           //heap seems smallish lets be conservative fall back to the free/2 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
index 4397f2e..9d6ad97 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
@@ -102,6 +102,7 @@ public class BKDWriter implements Closeable {
 
   final TrackingDirectoryWrapper tempDir;
   final String tempFileNamePrefix;
+  final double maxMBSortInHeap;
 
   final byte[] scratchDiff;
   final byte[] scratchPackedValue;
@@ -169,6 +170,8 @@ public class BKDWriter implements Closeable {
 
     // We write first maxPointsSortInHeap in heap, then cutover to offline for additional points:
     heapPointWriter = new HeapPointWriter(16, maxPointsSortInHeap, packedBytesLength);
+
+    this.maxMBSortInHeap = maxMBSortInHeap;
   }
 
   public static void verifyParams(int numDims, int maxPointsInLeafNode, double maxMBSortInHeap) {
@@ -550,7 +553,7 @@ public class BKDWriter implements Closeable {
     //int[] swapCount = new int[1];
     //int[] cmpCount = new int[1];
 
-    //System.out.println("SORT length=" + length);
+    // System.out.println("SORT length=" + length);
 
     // All buffered points are still in heap; just do in-place sort:
     new IntroSorter() {
@@ -623,13 +626,11 @@ public class BKDWriter implements Closeable {
           return cmp;
         }
 
-        // Tie-break
-        cmp = Integer.compare(writer.docIDs[i], writer.docIDs[j]);
-        if (cmp != 0) {
-          return cmp;
-        }
+        // Tie-break by docID:
 
-        return Long.compare(writer.ords[i], writer.ords[j]);
+        // No need to tie break on ord, for the case where the same doc has the same value in a given dimension indexed more than once: it
+        // can't matter at search time since we don't write ords into the index:
+        return Integer.compare(writer.docIDs[i], writer.docIDs[j]);
       }
     }.sort(start, start+length);
     //System.out.println("LEN=" + length + " SWAP=" + swapCount[0] + " CMP=" + cmpCount[0]);
@@ -679,29 +680,23 @@ public class BKDWriter implements Closeable {
             return cmp;
           }
 
-          // Tie-break by docID and then ord:
-          reader.reset(a.bytes, a.offset + packedBytesLength, a.length);
-          final int docIDA = reader.readVInt();
-          final long ordA = reader.readVLong();
+          // Tie-break by docID:
+          reader.reset(a.bytes, a.offset + packedBytesLength + Long.BYTES, a.length);
+          final int docIDA = reader.readInt();
 
-          reader.reset(b.bytes, b.offset + packedBytesLength, b.length);
-          final int docIDB = reader.readVInt();
-          final long ordB = reader.readVLong();
-
-          cmp = Integer.compare(docIDA, docIDB);
-          if (cmp != 0) {
-            return cmp;
-          }
+          reader.reset(b.bytes, b.offset + packedBytesLength + Long.BYTES, b.length);
+          final int docIDB = reader.readInt();
 
-          // TODO: is this really necessary?  If OfflineSorter is stable, we can safely return 0 here, and avoid writing ords?
-          return Long.compare(ordA, ordB);
+          // No need to tie break on ord, for the case where the same doc has the same value in a given dimension indexed more than once: it
+          // can't matter at search time since we don't write ords into the index:
+          return Integer.compare(docIDA, docIDB);
         }
       };
 
       // TODO: this is sort of sneaky way to get the final OfflinePointWriter from OfflineSorter:
       IndexOutput[] lastWriter = new IndexOutput[1];
 
-      OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix, cmp) {
+      OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix, cmp, OfflineSorter.BufferSize.megabytes(Math.max(1, (long) maxMBSortInHeap)), OfflineSorter.MAX_TEMPFILES) {
 
           /** We write/read fixed-byte-width file that {@link OfflinePointReader} can read. */
           @Override
@@ -753,7 +748,7 @@ public class BKDWriter implements Closeable {
 
   /** Writes the BKD tree to the provided {@link IndexOutput} and returns the file offset where index was written. */
   public long finish(IndexOutput out) throws IOException {
-    //System.out.println("\nBKDTreeWriter.finish pointCount=" + pointCount + " out=" + out + " heapWriter=" + heapWriter);
+    // System.out.println("\nBKDTreeWriter.finish pointCount=" + pointCount + " out=" + out + " heapWriter=" + heapPointWriter);
 
     // TODO: specialize the 1D case?  it's much faster at indexing time (no partitioning on recruse...)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java b/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
index 62fe28a..b79e638 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
@@ -162,8 +162,8 @@ public class TestDuelingCodecs extends LuceneTestCase {
     createRandomIndex(numdocs, leftWriter, seed);
     createRandomIndex(numdocs, rightWriter, seed);
 
-    leftReader = wrapReader(leftWriter.getReader());
-    rightReader = wrapReader(rightWriter.getReader());
+    leftReader = wrapReader(leftWriter.getReader(), false);
+    rightReader = wrapReader(rightWriter.getReader(), false);
     
     // check that our readers are valid
     TestUtil.checkReader(leftReader);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/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 3e1fe2d..7551d3c 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
@@ -17,8 +17,16 @@
 package org.apache.lucene.index;
 
 
+import java.io.IOException;
+
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.PointFormat;
+import org.apache.lucene.codecs.PointReader;
+import org.apache.lucene.codecs.PointWriter;
+import org.apache.lucene.codecs.lucene60.Lucene60PointReader;
+import org.apache.lucene.codecs.lucene60.Lucene60PointWriter;
 import org.apache.lucene.document.BinaryPoint;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoublePoint;
@@ -26,6 +34,9 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FloatPoint;
 import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.index.PointValues;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
@@ -478,4 +489,55 @@ public class TestPointValues extends LuceneTestCase {
       field.numericValue();
     });
   }
+
+  public void testTieBreakByDocID() throws Exception {
+    Directory dir = newFSDirectory(createTempDir());
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    IndexWriter w = new IndexWriter(dir, iwc);
+    Document doc = new Document();
+    doc.add(new IntPoint("int", 17));
+    for(int i=0;i<300000;i++) {
+      w.addDocument(doc);
+      if (false && random().nextInt(1000) == 17) {
+        w.commit();
+      }
+    }
+
+    IndexReader r = DirectoryReader.open(w);
+
+    for(LeafReaderContext ctx : r.leaves()) {
+      PointValues points = ctx.reader().getPointValues();
+      points.intersect("int",
+                       new IntersectVisitor() {
+
+                         int lastDocID = -1;
+
+                         @Override
+                         public void visit(int docID) {
+                           if (docID < lastDocID) {
+                             fail("docs out of order: docID=" + docID + " but lastDocID=" + lastDocID);
+                           }
+                           lastDocID = docID;
+                         }
+
+                         @Override
+                         public void visit(int docID, byte[] packedValue) {
+                           visit(docID);
+                         }
+
+                         @Override
+                         public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                           if (random().nextBoolean()) {
+                             return Relation.CELL_CROSSES_QUERY;
+                           } else {
+                             return Relation.CELL_INSIDE_QUERY;
+                           }
+                         }
+                       });
+    }
+    
+    r.close();
+    w.close();
+    dir.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
index 8a578fa..fb31792 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -53,11 +53,13 @@ import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
+import org.apache.lucene.index.MultiReader;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.PointValues;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.SlowCompositeReaderWrapper;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
@@ -360,7 +362,7 @@ public class TestPointQueries extends LuceneTestCase {
     final IndexReader r = w.getReader();
     w.close();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     int numThreads = TestUtil.nextInt(random(), 2, 5);
 
@@ -627,7 +629,7 @@ public class TestPointQueries extends LuceneTestCase {
     final IndexReader r = w.getReader();
     w.close();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     int numThreads = TestUtil.nextInt(random(), 2, 5);
 
@@ -847,7 +849,7 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     assertEquals(1, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, true, 0L, true)));
     assertEquals(1, s.count(LongPoint.newRangeQuery("value", 0L, true, Long.MAX_VALUE, true)));
@@ -885,7 +887,7 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     assertEquals(1, s.count(BinaryPoint.newRangeQuery("value",
         toUTF8("aaa"),
@@ -950,7 +952,7 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     assertEquals(2, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
     assertEquals(1, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, true, Long.MAX_VALUE, false)));
@@ -976,7 +978,6 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = w.getReader();
 
-    // We can't wrap with "exotic" readers because the query must see the RangeTreeDVFormat:
     IndexSearcher s = newSearcher(r, false);
 
     assertEquals(2, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
@@ -1003,7 +1004,7 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(0, s.count(BinaryPoint.newRangeQuery("value", toUTF8("m"), true, toUTF8("n"), false)));
 
     assertEquals(2, s.count(BinaryPoint.newRangeQuery("value", (byte[]) null, true, null, true)));
@@ -1040,7 +1041,7 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(0, s.count(LongPoint.newRangeQuery("value", 17L, true, 13L, false)));
 
     IOUtils.close(r, w, dir);
@@ -1207,7 +1208,7 @@ public class TestPointQueries extends LuceneTestCase {
     w.addDocument(doc);
 
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(1, s.count(IntPoint.newExactQuery("int", 42)));
     assertEquals(0, s.count(IntPoint.newExactQuery("int", 41)));
 
@@ -1328,7 +1329,7 @@ public class TestPointQueries extends LuceneTestCase {
     final IndexReader r = w.getReader();
     w.close();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     int numThreads = TestUtil.nextInt(random(), 2, 5);
 
@@ -1467,7 +1468,7 @@ public class TestPointQueries extends LuceneTestCase {
     doc.add(new IntPoint("int", 17, 42));
     w.addDocument(doc);
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     assertEquals(0, s.count(newMultiDimIntSetQuery("int", 2, 17, 41)));
     assertEquals(1, s.count(newMultiDimIntSetQuery("int", 2, 17, 42)));
@@ -1490,7 +1491,7 @@ public class TestPointQueries extends LuceneTestCase {
     doc.add(new IntPoint("int", 34, 79));
     w.addDocument(doc);
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     assertEquals(0, s.count(newMultiDimIntSetQuery("int", 2, 17, 41)));
     assertEquals(1, s.count(newMultiDimIntSetQuery("int", 2, 17, 42)));
@@ -1523,7 +1524,7 @@ public class TestPointQueries extends LuceneTestCase {
       w.addDocument(doc);
     }
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     assertEquals(zeroCount, s.count(newMultiDimIntSetQuery("int", 2, 0, 0)));
     assertEquals(10000-zeroCount, s.count(newMultiDimIntSetQuery("int", 2, 1, 1)));
@@ -1573,7 +1574,7 @@ public class TestPointQueries extends LuceneTestCase {
     w.addDocument(doc);
 
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(0, s.count(IntPoint.newSetQuery("int", 16)));
     assertEquals(1, s.count(IntPoint.newSetQuery("int", 17)));
     assertEquals(3, s.count(IntPoint.newSetQuery("int", 17, 97, 42)));
@@ -1634,7 +1635,7 @@ public class TestPointQueries extends LuceneTestCase {
     w.addDocument(doc);
 
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(0, s.count(IntPoint.newSetQuery("int", 16)));
     assertEquals(1, s.count(IntPoint.newSetQuery("int", 17)));
     assertEquals(1, s.count(IntPoint.newSetQuery("int", 17, 97, 42)));
@@ -1685,7 +1686,7 @@ public class TestPointQueries extends LuceneTestCase {
     w.addDocument(doc);
 
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(0, s.count(IntPoint.newSetQuery("int")));
     assertEquals(0, s.count(LongPoint.newSetQuery("long")));
     assertEquals(0, s.count(FloatPoint.newSetQuery("float")));
@@ -1719,7 +1720,7 @@ public class TestPointQueries extends LuceneTestCase {
     }
 
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 0)));
     assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 0, -7)));
     assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 7, 0)));
@@ -1777,7 +1778,7 @@ public class TestPointQueries extends LuceneTestCase {
     }
 
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 0)));
     assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 0, -7)));
     assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 7, 0)));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
index f7a1970..c4233c0 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
@@ -218,7 +218,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
 
     final TaxonomyReader tr = new DirectoryTaxonomyReader(tw);
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     if (VERBOSE) {
       System.out.println("TEST: searcher=" + s);
@@ -375,7 +375,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
     }
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     FacetsConfig config = new FacetsConfig();
     
     int numIters = atLeast(10);
@@ -516,7 +516,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
     }
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     FacetsConfig config = new FacetsConfig();
     
     int numIters = atLeast(10);
@@ -671,7 +671,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
     }
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     FacetsConfig config = new FacetsConfig();
     
     int numIters = atLeast(10);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java b/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
index 93caa701..0015c6c 100644
--- a/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
+++ b/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
@@ -174,7 +174,8 @@ public abstract class SorterTestBase extends LuceneTestCase {
     doc.add(new Field(TERM_VECTORS_FIELD, Integer.toString(id), TERM_VECTORS_TYPE));
     byte[] bytes = new byte[4];
     NumericUtils.intToBytes(id, bytes, 0);
-    doc.add(new BinaryPoint(DIMENSIONAL_FIELD, bytes));
+    // TODO: index time sorting doesn't yet support points
+    //doc.add(new BinaryPoint(DIMENSIONAL_FIELD, bytes));
     return doc;
   }
 
@@ -378,6 +379,8 @@ public abstract class SorterTestBase extends LuceneTestCase {
     }
   }
 
+  // TODO: index sorting doesn't yet support points
+  /*
   public void testPoints() throws Exception {
     PointValues values = sortedReader.getPointValues();
     values.intersect(DIMENSIONAL_FIELD,
@@ -398,4 +401,5 @@ public abstract class SorterTestBase extends LuceneTestCase {
                        }
                      });
   }
+  */
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
index 3e8cf3d..f6d407d 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
@@ -40,7 +40,7 @@ public class TestBigIntegerPoint extends LuceneTestCase {
     
     // search and verify we found our doc
     IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(BigIntegerPoint.newExactQuery("field", large)));
     assertEquals(1, searcher.count(BigIntegerPoint.newRangeQuery("field", large.subtract(BigInteger.ONE), false, large.add(BigInteger.ONE), false)));
     assertEquals(1, searcher.count(BigIntegerPoint.newSetQuery("field", large)));
@@ -65,7 +65,7 @@ public class TestBigIntegerPoint extends LuceneTestCase {
     
     // search and verify we found our doc
     IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(BigIntegerPoint.newExactQuery("field", negative)));
     assertEquals(1, searcher.count(BigIntegerPoint.newRangeQuery("field", negative.subtract(BigInteger.ONE), false, negative.add(BigInteger.ONE), false)));
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
index c9be31f..9854001 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
@@ -40,7 +40,7 @@ public class TestInetAddressPoint extends LuceneTestCase {
     
     // search and verify we found our doc
     IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(InetAddressPoint.newExactQuery("field", address)));
     assertEquals(1, searcher.count(InetAddressPoint.newPrefixQuery("field", address, 24)));
     assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("1.2.3.3"), false, InetAddress.getByName("1.2.3.5"), false)));
@@ -66,7 +66,7 @@ public class TestInetAddressPoint extends LuceneTestCase {
     
     // search and verify we found our doc
     IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(InetAddressPoint.newExactQuery("field", address)));
     assertEquals(1, searcher.count(InetAddressPoint.newPrefixQuery("field", address, 64)));
     assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("fec0::f66c"), false, InetAddress.getByName("fec0::f66e"), false)));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
index b67dec8..1d3bfac 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
@@ -38,7 +38,7 @@ public class TestLatLonPoint extends LuceneTestCase {
     
     // search and verify we found our doc
     IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(LatLonPoint.newBoxQuery("field", 18, 19, -66, -65)));
 
     reader.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceQuery.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceQuery.java
index 3d47b44..e37d75e 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceQuery.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceQuery.java
@@ -55,7 +55,7 @@ public class TestLatLonPointDistanceQuery extends LuceneTestCase {
     
     // search within 50km and verify we found our doc
     IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(LatLonPoint.newDistanceQuery("field", 18, -65, 50_000)));
 
     reader.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
index 1bcadd6..47cd740 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
@@ -60,7 +60,7 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
       }
       iw.commit();
       final IndexReader reader = iw.getReader();
-      final IndexSearcher searcher = newSearcher(reader);
+      final IndexSearcher searcher = newSearcher(reader, false);
       iw.close();
 
       for (int i = 0; i < 100; ++i) {
@@ -188,7 +188,7 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
     }
     iw.commit();
     final IndexReader reader = iw.getReader();
-    final IndexSearcher searcher = newSearcher(reader);
+    final IndexSearcher searcher = newSearcher(reader, false);
     iw.close();
 
     for (int i = 0; i < 100; ++i) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointFormat.java
index 3411c7b..15836de 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointFormat.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.codecs.asserting;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collection;
 
 import org.apache.lucene.codecs.PointFormat;
@@ -24,9 +25,13 @@ import org.apache.lucene.codecs.PointReader;
 import org.apache.lucene.codecs.PointWriter;
 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.StringHelper;
 import org.apache.lucene.util.TestUtil;
 
 /**
@@ -60,6 +65,83 @@ public final class AssertingPointFormat extends PointFormat {
   public PointReader fieldsReader(SegmentReadState state) throws IOException {
     return new AssertingPointReader(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;
+
+    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 {
+      // 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 {
+
+      // 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;
+        assert StringHelper.compare(bytesPerDim, lastMaxPackedValue, dim*bytesPerDim, packedValue, dim*bytesPerDim) >= 0: "dim=" + dim + " of " +  numDims;
+      }
+
+      // 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);
+      }
+      in.visit(docID, packedValue);
+    }
+
+    @Override
+    public void grow(int count) {
+      in.grow(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 AssertingPointReader extends PointReader {
     private final PointReader in;
@@ -80,8 +162,8 @@ public final class AssertingPointFormat extends PointFormat {
 
     @Override
     public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
-      // TODO: wrap the visitor and make sure things are being reasonable
-      in.intersect(fieldName, visitor);
+      in.intersect(fieldName,
+                   new AssertingIntersectVisitor(in.getNumDimensions(fieldName), in.getBytesPerDimension(fieldName), visitor));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java
index 1a81138..5113bcd 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java
@@ -163,29 +163,31 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
     w.close();
     DirectoryReader r = DirectoryReader.open(dir);
     assertEquals(1, r.numDocs());
-    PointValues values = MultiPointValues.get(r);
     Bits liveDocs = MultiFields.getLiveDocs(r);
-    NumericDocValues idValues = MultiDocValues.getNumericValues(r, "id");
-
-    if (values != null) {
-      BitSet seen = new BitSet();
-      values.intersect("dim",
-                       new IntersectVisitor() {
-                         @Override
-                         public Relation compare(byte[] minPacked, byte[] maxPacked) {
-                           return Relation.CELL_CROSSES_QUERY;
-                         }
-                         public void visit(int docID) {
-                           throw new IllegalStateException();
-                         }
-                         public void visit(int docID, byte[] packedValue) {
-                           if (liveDocs.get(docID)) {
-                             seen.set(docID);
+
+    for(LeafReaderContext ctx : r.leaves()) {
+      PointValues values = ctx.reader().getPointValues();
+      NumericDocValues idValues = ctx.reader().getNumericDocValues("id");
+      if (values != null) {
+        BitSet seen = new BitSet();
+        values.intersect("dim",
+                         new IntersectVisitor() {
+                           @Override
+                           public Relation compare(byte[] minPacked, byte[] maxPacked) {
+                             return Relation.CELL_CROSSES_QUERY;
+                           }
+                           public void visit(int docID) {
+                             throw new IllegalStateException();
+                           }
+                           public void visit(int docID, byte[] packedValue) {
+                             if (liveDocs.get(docID)) {
+                               seen.set(docID);
+                             }
+                             assertEquals(idValues.get(docID), NumericUtils.bytesToInt(packedValue, 0));
                            }
-                           assertEquals(idValues.get(docID), NumericUtils.bytesToInt(packedValue, 0));
-                         }
-                       });
-      assertEquals(0, seen.cardinality());
+                         });
+        assertEquals(0, seen.cardinality());
+      }
     }
     IOUtils.close(r, dir);
   }
@@ -361,8 +363,6 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
       DirectoryReader r = w.getReader();
       w.close();
 
-      PointValues dimValues = MultiPointValues.get(r);
-
       int iters = atLeast(100);
       for(int iter=0;iter<iters;iter++) {
         if (VERBOSE) {
@@ -386,50 +386,59 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
         }
 
         final BitSet hits = new BitSet();
-        dimValues.intersect("field", new IntersectVisitor() {
-            @Override
-            public void visit(int docID) {
-              hits.set(docID);
-              //System.out.println("visit docID=" + docID);
-            }
+        for(LeafReaderContext ctx : r.leaves()) {
+          PointValues dimValues = ctx.reader().getPointValues();
+          if (dimValues == null) {
+            continue;
+          }
 
-            @Override
-            public void visit(int docID, byte[] packedValue) {
-              //System.out.println("visit check docID=" + docID);
-              for(int dim=0;dim<numDims;dim++) {
-                BigInteger x = NumericUtils.bytesToBigInt(packedValue, dim * numBytesPerDim, numBytesPerDim);
-                if (x.compareTo(queryMin[dim]) < 0 || x.compareTo(queryMax[dim]) > 0) {
-                  //System.out.println("  no");
-                  return;
-                }
+          final int docBase = ctx.docBase;
+          
+          dimValues.intersect("field", new IntersectVisitor() {
+              @Override
+              public void visit(int docID) {
+                hits.set(docBase+docID);
+                //System.out.println("visit docID=" + docID);
               }
 
-              //System.out.println("  yes");
-              hits.set(docID);
-            }
-
-            @Override
-            public Relation compare(byte[] minPacked, byte[] maxPacked) {
-              boolean crosses = false;
-              for(int dim=0;dim<numDims;dim++) {
-                BigInteger min = NumericUtils.bytesToBigInt(minPacked, dim * numBytesPerDim, numBytesPerDim);
-                BigInteger max = NumericUtils.bytesToBigInt(maxPacked, dim * numBytesPerDim, numBytesPerDim);
-                assert max.compareTo(min) >= 0;
-
-                if (max.compareTo(queryMin[dim]) < 0 || min.compareTo(queryMax[dim]) > 0) {
-                  return Relation.CELL_OUTSIDE_QUERY;
-                } else if (min.compareTo(queryMin[dim]) < 0 || max.compareTo(queryMax[dim]) > 0) {
-                  crosses = true;
+              @Override
+              public void visit(int docID, byte[] packedValue) {
+                //System.out.println("visit check docID=" + docID);
+                for(int dim=0;dim<numDims;dim++) {
+                  BigInteger x = NumericUtils.bytesToBigInt(packedValue, dim * numBytesPerDim, numBytesPerDim);
+                  if (x.compareTo(queryMin[dim]) < 0 || x.compareTo(queryMax[dim]) > 0) {
+                    //System.out.println("  no");
+                    return;
+                  }
                 }
+
+                //System.out.println("  yes");
+                hits.set(docBase+docID);
               }
 
-              if (crosses) {
-                return Relation.CELL_CROSSES_QUERY;
-              } else {
-                return Relation.CELL_INSIDE_QUERY;
+              @Override
+              public Relation compare(byte[] minPacked, byte[] maxPacked) {
+                boolean crosses = false;
+                for(int dim=0;dim<numDims;dim++) {
+                  BigInteger min = NumericUtils.bytesToBigInt(minPacked, dim * numBytesPerDim, numBytesPerDim);
+                  BigInteger max = NumericUtils.bytesToBigInt(maxPacked, dim * numBytesPerDim, numBytesPerDim);
+                  assert max.compareTo(min) >= 0;
+
+                  if (max.compareTo(queryMin[dim]) < 0 || min.compareTo(queryMax[dim]) > 0) {
+                    return Relation.CELL_OUTSIDE_QUERY;
+                  } else if (min.compareTo(queryMin[dim]) < 0 || max.compareTo(queryMax[dim]) > 0) {
+                    crosses = true;
+                  }
+                }
+
+                if (crosses) {
+                  return Relation.CELL_CROSSES_QUERY;
+                } else {
+                  return Relation.CELL_INSIDE_QUERY;
+                }
               }
-            }
-          });
+            });
+        }
 
         for(int docID=0;docID<numDocs;docID++) {
           BigInteger[] docValues = docs[docID];
@@ -665,24 +674,39 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
         System.out.println("TEST: reader=" + r);
       }
 
-      PointValues dimValues = MultiPointValues.get(r);
-      if (VERBOSE) {
-        System.out.println("  dimValues=" + dimValues);
-      }
-      assertNotNull(dimValues);
-
       NumericDocValues idValues = MultiDocValues.getNumericValues(r, "id");
       Bits liveDocs = MultiFields.getLiveDocs(r);
 
       // Verify min/max values are correct:
-      byte[] minValues = dimValues.getMinPackedValue("field");
-      byte[] maxValues = dimValues.getMaxPackedValue("field");
+      byte[] minValues = new byte[numDims*numBytesPerDim];
+      Arrays.fill(minValues, (byte) 0xff);
+
+      byte[] maxValues = new byte[numDims*numBytesPerDim];
+
+      for(LeafReaderContext ctx : r.leaves()) {
+        PointValues dimValues = ctx.reader().getPointValues();
+        if (dimValues == null) {
+          continue;
+        }
+
+        byte[] leafMinValues = dimValues.getMinPackedValue("field");
+        byte[] leafMaxValues = dimValues.getMaxPackedValue("field");
+        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);
+          }
+          if (StringHelper.compare(numBytesPerDim, leafMaxValues, dim*numBytesPerDim, maxValues, dim*numBytesPerDim) > 0) {
+            System.arraycopy(leafMaxValues, dim*numBytesPerDim, maxValues, dim*numBytesPerDim, numBytesPerDim);
+          }
+        }
+      }
+
       byte[] scratch = new byte[numBytesPerDim];
       for(int dim=0;dim<numDims;dim++) {
-        System.arraycopy(minValues, dim*numBytesPerDim, scratch, 0, scratch.length);
+        System.arraycopy(minValues, dim*numBytesPerDim, scratch, 0, numBytesPerDim);
         //System.out.println("dim=" + dim + " expectedMin=" + new BytesRef(expectedMinValues[dim]) + " min=" + new BytesRef(scratch));
         assertTrue(Arrays.equals(expectedMinValues[dim], scratch));
-        System.arraycopy(maxValues, dim*numBytesPerDim, scratch, 0, scratch.length);
+        System.arraycopy(maxValues, dim*numBytesPerDim, scratch, 0, numBytesPerDim);
         //System.out.println("dim=" + dim + " expectedMax=" + new BytesRef(expectedMaxValues[dim]) + " max=" + new BytesRef(scratch));
         assertTrue(Arrays.equals(expectedMaxValues[dim], scratch));
       }
@@ -716,58 +740,68 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
 
         final BitSet hits = new BitSet();
 
-        dimValues.intersect("field", new PointValues.IntersectVisitor() {
-            @Override
-            public void visit(int docID) {
-              if (liveDocs == null || liveDocs.get(docID)) {
-                hits.set((int) idValues.get(docID));
-              }
-              //System.out.println("visit docID=" + docID);
-            }
+        for(LeafReaderContext ctx : r.leaves()) {
+          PointValues dimValues = ctx.reader().getPointValues();
+          if (dimValues == null) {
+            continue;
+          }
 
-            @Override
-            public void visit(int docID, byte[] packedValue) {
-              if (liveDocs != null && liveDocs.get(docID) == false) {
-                return;
-              }
-              //System.out.println("visit check docID=" + docID + " id=" + idValues.get(docID));
-              for(int dim=0;dim<numDims;dim++) {
-                //System.out.println("  dim=" + dim + " value=" + new BytesRef(packedValue, dim*numBytesPerDim, numBytesPerDim));
-                if (StringHelper.compare(numBytesPerDim, packedValue, dim*numBytesPerDim, queryMin[dim], 0) < 0 ||
-                    StringHelper.compare(numBytesPerDim, packedValue, dim*numBytesPerDim, queryMax[dim], 0) > 0) {
-                  //System.out.println("  no");
-                  return;
+          final int docBase = ctx.docBase;
+
+          dimValues.intersect("field", new PointValues.IntersectVisitor() {
+              @Override
+              public void visit(int docID) {
+                if (liveDocs == null || liveDocs.get(docBase+docID)) {
+                  hits.set((int) idValues.get(docBase+docID));
                 }
+                //System.out.println("visit docID=" + docID);
               }
 
-              //System.out.println("  yes");
-              hits.set((int) idValues.get(docID));
-            }
+              @Override
+              public void visit(int docID, byte[] packedValue) {
+                if (liveDocs != null && liveDocs.get(docBase+docID) == false) {
+                  return;
+                }
 
-            @Override
-            public Relation compare(byte[] minPacked, byte[] maxPacked) {
-              boolean crosses = false;
-              //System.out.println("compare");
-              for(int dim=0;dim<numDims;dim++) {
-                if (StringHelper.compare(numBytesPerDim, maxPacked, dim*numBytesPerDim, queryMin[dim], 0) < 0 ||
-                    StringHelper.compare(numBytesPerDim, minPacked, dim*numBytesPerDim, queryMax[dim], 0) > 0) {
-                  //System.out.println("  query_outside_cell");
-                  return Relation.CELL_OUTSIDE_QUERY;
-                } else if (StringHelper.compare(numBytesPerDim, minPacked, dim*numBytesPerDim, queryMin[dim], 0) < 0 ||
-                           StringHelper.compare(numBytesPerDim, maxPacked, dim*numBytesPerDim, queryMax[dim], 0) > 0) {
-                  crosses = true;
+                //System.out.println("visit check docID=" + docID + " id=" + idValues.get(docID));
+                for(int dim=0;dim<numDims;dim++) {
+                  //System.out.println("  dim=" + dim + " value=" + new BytesRef(packedValue, dim*numBytesPerDim, numBytesPerDim));
+                  if (StringHelper.compare(numBytesPerDim, packedValue, dim*numBytesPerDim, queryMin[dim], 0) < 0 ||
+                      StringHelper.compare(numBytesPerDim, packedValue, dim*numBytesPerDim, queryMax[dim], 0) > 0) {
+                    //System.out.println("  no");
+                    return;
+                  }
                 }
+
+                //System.out.println("  yes");
+                hits.set((int) idValues.get(docBase+docID));
               }
 
-              if (crosses) {
-                //System.out.println("  query_crosses_cell");
-                return Relation.CELL_CROSSES_QUERY;
-              } else {
-                //System.out.println("  cell_inside_query");
-                return Relation.CELL_INSIDE_QUERY;
+              @Override
+              public Relation compare(byte[] minPacked, byte[] maxPacked) {
+                boolean crosses = false;
+                //System.out.println("compare");
+                for(int dim=0;dim<numDims;dim++) {
+                  if (StringHelper.compare(numBytesPerDim, maxPacked, dim*numBytesPerDim, queryMin[dim], 0) < 0 ||
+                      StringHelper.compare(numBytesPerDim, minPacked, dim*numBytesPerDim, queryMax[dim], 0) > 0) {
+                    //System.out.println("  query_outside_cell");
+                    return Relation.CELL_OUTSIDE_QUERY;
+                  } else if (StringHelper.compare(numBytesPerDim, minPacked, dim*numBytesPerDim, queryMin[dim], 0) < 0 ||
+                             StringHelper.compare(numBytesPerDim, maxPacked, dim*numBytesPerDim, queryMax[dim], 0) > 0) {
+                    crosses = true;
+                  }
+                }
+
+                if (crosses) {
+                  //System.out.println("  query_crosses_cell");
+                  return Relation.CELL_CROSSES_QUERY;
+                } else {
+                  //System.out.println("  cell_inside_query");
+                  return Relation.CELL_INSIDE_QUERY;
+                }
               }
-            }
-          });
+            });
+        }
 
         BitSet expected = new BitSet();
         for(int ord=0;ord<numValues;ord++) {
@@ -845,7 +879,7 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
     w.forceMerge(1);
 
     DirectoryReader r = w.getReader();
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(2, s.count(IntPoint.newExactQuery("int1", 17)));
     assertEquals(2, s.count(IntPoint.newExactQuery("int2", 42)));
     r.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3cbc48ed/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 462e73e..79eb6280 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
@@ -1625,6 +1625,10 @@ public abstract class LuceneTestCase extends Assert {
   }
 
   public static IndexReader wrapReader(IndexReader r) throws IOException {
+    return wrapReader(r, true);
+  }
+
+  public static IndexReader wrapReader(IndexReader r, boolean allowSlowCompositeReader) throws IOException {
     Random random = random();
       
     // TODO: remove this, and fix those tests to wrap before putting slow around:
@@ -1632,10 +1636,12 @@ public abstract class LuceneTestCase extends Assert {
     for (int i = 0, c = random.nextInt(6)+1; i < c; i++) {
       switch(random.nextInt(6)) {
       case 0:
-        if (VERBOSE) {
-          System.out.println("NOTE: LuceneTestCase.wrapReader: wrapping previous reader=" + r + " with SlowCompositeReaderWrapper.wrap");
+        if (allowSlowCompositeReader) {
+          if (VERBOSE) {
+            System.out.println("NOTE: LuceneTestCase.wrapReader: wrapping previous reader=" + r + " with SlowCompositeReaderWrapper.wrap");
+          }
+          r = SlowCompositeReaderWrapper.wrap(r);
         }
-        r = SlowCompositeReaderWrapper.wrap(r);
         break;
       case 1:
         // will create no FC insanity in atomic case, as ParallelLeafReader has own cache key:
@@ -1656,22 +1662,24 @@ public abstract class LuceneTestCase extends Assert {
         r = new FCInvisibleMultiReader(r);
         break;
       case 3:
-        final LeafReader ar = SlowCompositeReaderWrapper.wrap(r);
-        final List<String> allFields = new ArrayList<>();
-        for (FieldInfo fi : ar.getFieldInfos()) {
-          allFields.add(fi.name);
-        }
-        Collections.shuffle(allFields, random);
-        final int end = allFields.isEmpty() ? 0 : random.nextInt(allFields.size());
-        final Set<String> fields = new HashSet<>(allFields.subList(0, end));
-        // will create no FC insanity as ParallelLeafReader has own cache key:
-        if (VERBOSE) {
-          System.out.println("NOTE: LuceneTestCase.wrapReader: wrapping previous reader=" + r + " with ParallelLeafReader(SlowCompositeReaderWapper)");
+        if (allowSlowCompositeReader) {
+          final LeafReader ar = SlowCompositeReaderWrapper.wrap(r);
+          final List<String> allFields = new ArrayList<>();
+          for (FieldInfo fi : ar.getFieldInfos()) {
+            allFields.add(fi.name);
+          }
+          Collections.shuffle(allFields, random);
+          final int end = allFields.isEmpty() ? 0 : random.nextInt(allFields.size());
+          final Set<String> fields = new HashSet<>(allFields.subList(0, end));
+          // will create no FC insanity as ParallelLeafReader has own cache key:
+          if (VERBOSE) {
+            System.out.println("NOTE: LuceneTestCase.wrapReader: wrapping previous reader=" + r + " with ParallelLeafReader(SlowCompositeReaderWapper)");
+          }
+          r = new ParallelLeafReader(
+                                     new FieldFilterLeafReader(ar, fields, false),
+                                     new FieldFilterLeafReader(ar, fields, true)
+                                     );
         }
-        r = new ParallelLeafReader(
-                                   new FieldFilterLeafReader(ar, fields, false),
-                                   new FieldFilterLeafReader(ar, fields, true)
-                                   );
         break;
       case 4:
         // Häckidy-Hick-Hack: a standard Reader will cause FC insanity, so we use
@@ -1701,7 +1709,9 @@ public abstract class LuceneTestCase extends Assert {
       }
     }
     if (wasOriginallyAtomic) {
-      r = SlowCompositeReaderWrapper.wrap(r);
+      if (allowSlowCompositeReader) {
+        r = SlowCompositeReaderWrapper.wrap(r);
+      }
     } else if ((r instanceof CompositeReader) && !(r instanceof FCInvisibleMultiReader)) {
       // prevent cache insanity caused by e.g. ParallelCompositeReader, to fix we wrap one more time:
       r = new FCInvisibleMultiReader(r);
@@ -2588,40 +2598,45 @@ public abstract class LuceneTestCase extends Assert {
   }
 
   // naive silly memory heavy uninversion!!  maps docID -> packed values (a Set because a given doc can be multi-valued)
-  private Map<Integer,Set<BytesRef>> uninvert(String fieldName, PointValues points) throws IOException {
+  private Map<Integer,Set<BytesRef>> uninvert(String fieldName, IndexReader reader) throws IOException {
     final Map<Integer,Set<BytesRef>> docValues = new HashMap<>();
-    points.intersect(fieldName, new PointValues.IntersectVisitor() {
-        @Override
-        public void visit(int docID) {
-          throw new UnsupportedOperationException();
-        }
+    for(LeafReaderContext ctx : reader.leaves()) {
 
-        @Override
-        public void visit(int docID, byte[] packedValue) throws IOException {
-          if (docValues.containsKey(docID) == false) {
-            docValues.put(docID, new HashSet<BytesRef>());
-          }
-          docValues.get(docID).add(new BytesRef(packedValue.clone()));
-        }
+      PointValues points = ctx.reader().getPointValues();
+      if (points == null) {
+        continue;
+      }
+
+      points.intersect(fieldName,
+                       new PointValues.IntersectVisitor() {
+                         @Override
+                         public void visit(int docID) {
+                           throw new UnsupportedOperationException();
+                         }
+
+                         @Override
+                         public void visit(int docID, byte[] packedValue) throws IOException {
+                           int topDocID = ctx.docBase + docID;
+                           if (docValues.containsKey(topDocID) == false) {
+                             docValues.put(topDocID, new HashSet<BytesRef>());
+                           }
+                           docValues.get(topDocID).add(new BytesRef(packedValue.clone()));
+                         }
+
+                         @Override
+                         public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                           // We pretend our query shape is so hairy that it crosses every single cell:
+                           return PointValues.Relation.CELL_CROSSES_QUERY;
+                         }
+                       });
+    }
 
-        @Override
-        public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-          // We pretend our query shape is so hairy that it crosses every single cell:
-          return PointValues.Relation.CELL_CROSSES_QUERY;
-        }
-      });
     return docValues;
   }
 
   public void assertPointsEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
-    assertPointsEquals(info,
-                       MultiFields.getMergedFieldInfos(leftReader),
-                       MultiPointValues.get(leftReader),
-                       MultiFields.getMergedFieldInfos(rightReader),
-                       MultiPointValues.get(rightReader));
-  }
-
-  public void assertPointsEquals(String info, FieldInfos fieldInfos1, PointValues points1, FieldInfos fieldInfos2, PointValues points2) throws IOException {
+    FieldInfos fieldInfos1 = MultiFields.getMergedFieldInfos(leftReader);
+    FieldInfos fieldInfos2 = MultiFields.getMergedFieldInfos(rightReader);
     for(FieldInfo fieldInfo1 : fieldInfos1) {
       if (fieldInfo1.getPointDimensionCount() != 0) {
         FieldInfo fieldInfo2 = fieldInfos2.fieldInfo(fieldInfo1.name);
@@ -2631,8 +2646,8 @@ public abstract class LuceneTestCase extends Assert {
         assertEquals(info, fieldInfo2.getPointNumBytes(), fieldInfo2.getPointNumBytes());
 
         assertEquals(info + " field=" + fieldInfo1.name,
-                     uninvert(fieldInfo1.name, points1),
-                     uninvert(fieldInfo1.name, points2));
+                     uninvert(fieldInfo1.name, leftReader),
+                     uninvert(fieldInfo1.name, rightReader));
       }
     }