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 2015/08/15 21:07:52 UTC

svn commit: r1696089 - in /lucene/dev/branches/lucene6699/lucene: spatial/src/test/org/apache/lucene/spatial/spatial4j/ spatial3d/src/java/org/apache/lucene/bkdtree3d/ spatial3d/src/java/org/apache/lucene/geo3d/ spatial3d/src/test/org/apache/lucene/bkd...

Author: mikemccand
Date: Sat Aug 15 19:07:52 2015
New Revision: 1696089

URL: http://svn.apache.org/r1696089
Log:
LUCENE-6699: add tests, fix bugs

Modified:
    lucene/dev/branches/lucene6699/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java
    lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/BKD3DTreeWriter.java
    lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesConsumer.java
    lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/HeapWriter.java
    lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java
    lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/PointInGeo3DShapeQuery.java
    lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoArea.java
    lucene/dev/branches/lucene6699/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java

Modified: lucene/dev/branches/lucene6699/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java?rev=1696089&r1=1696088&r2=1696089&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java (original)
+++ lucene/dev/branches/lucene6699/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java Sat Aug 15 19:07:52 2015
@@ -58,30 +58,30 @@ public abstract class Geo3dShapeRectRela
   }
 
   protected GeoBBox getBoundingBox(final GeoShape path) {
-      Bounds bounds = path.getBounds(null);
+    Bounds bounds = path.getBounds(null);
 
-      double leftLon;
-      double rightLon;
-      if (bounds.checkNoLongitudeBound()) {
-        leftLon = -Math.PI;
-        rightLon = Math.PI;
-      } else {
-        leftLon = bounds.getLeftLongitude().doubleValue();
-        rightLon = bounds.getRightLongitude().doubleValue();
-      }
-      double minLat;
-      if (bounds.checkNoBottomLatitudeBound()) {
-        minLat = -Math.PI * 0.5;
-      } else {
-        minLat = bounds.getMinLatitude().doubleValue();
-      }
-      double maxLat;
-      if (bounds.checkNoTopLatitudeBound()) {
-        maxLat = Math.PI * 0.5;
-      } else {
-        maxLat = bounds.getMaxLatitude().doubleValue();
-      }
-      return GeoBBoxFactory.makeGeoBBox(planetModel, maxLat, minLat, leftLon, rightLon);
+    double leftLon;
+    double rightLon;
+    if (bounds.checkNoLongitudeBound()) {
+      leftLon = -Math.PI;
+      rightLon = Math.PI;
+    } else {
+      leftLon = bounds.getLeftLongitude().doubleValue();
+      rightLon = bounds.getRightLongitude().doubleValue();
+    }
+    double minLat;
+    if (bounds.checkNoBottomLatitudeBound()) {
+      minLat = -Math.PI * 0.5;
+    } else {
+      minLat = bounds.getMinLatitude().doubleValue();
+    }
+    double maxLat;
+    if (bounds.checkNoTopLatitudeBound()) {
+      maxLat = Math.PI * 0.5;
+    } else {
+      maxLat = bounds.getMaxLatitude().doubleValue();
+    }
+    return GeoBBoxFactory.makeGeoBBox(planetModel, maxLat, minLat, leftLon, rightLon);
   }
 
   abstract class Geo3dRectIntersectionTestHelper extends RectIntersectionTestHelper<Geo3dShape> {

Modified: lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/BKD3DTreeWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/BKD3DTreeWriter.java?rev=1696089&r1=1696088&r2=1696089&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/BKD3DTreeWriter.java (original)
+++ lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/BKD3DTreeWriter.java Sat Aug 15 19:07:52 2015
@@ -201,7 +201,7 @@ class BKD3DTreeWriter {
       if (success) {
         IOUtils.close(sortedWriter, reader);
       } else {
-        IOUtils.closeWhileHandlingException(reader);
+        IOUtils.closeWhileHandlingException(sortedWriter, reader);
         try {
           sortedWriter.destroy();
         } catch (Throwable t) {
@@ -283,6 +283,7 @@ class BKD3DTreeWriter {
                       heapWriter.ords[i],
                       heapWriter.docIDs[i]);
       }
+      sorted.close();
 
       return sorted;
     } else {

Modified: lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesConsumer.java?rev=1696089&r1=1696088&r2=1696089&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesConsumer.java Sat Aug 15 19:07:52 2015
@@ -108,11 +108,13 @@ class Geo3DDocValuesConsumer extends Doc
       BytesRef value = valuesIt.next();
       // TODO: we should allow multi-valued here, just appended into the BDV
       // 3 ints packed into byte[]
-      assert value.length == 12;
-      int x = Geo3DDocValuesFormat.readInt(value.bytes, value.offset);
-      int y = Geo3DDocValuesFormat.readInt(value.bytes, value.offset+4);
-      int z = Geo3DDocValuesFormat.readInt(value.bytes, value.offset+8);
-      writer.add(x, y, z, docID);
+      if (value != null) {
+        assert value.length == 12;
+        int x = Geo3DDocValuesFormat.readInt(value.bytes, value.offset);
+        int y = Geo3DDocValuesFormat.readInt(value.bytes, value.offset+4);
+        int z = Geo3DDocValuesFormat.readInt(value.bytes, value.offset+8);
+        writer.add(x, y, z, docID);
+      }
     }
 
     long indexStartFP = writer.finish(out);

Modified: lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/HeapWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/HeapWriter.java?rev=1696089&r1=1696088&r2=1696089&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/HeapWriter.java (original)
+++ lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/HeapWriter.java Sat Aug 15 19:07:52 2015
@@ -24,6 +24,7 @@ final class HeapWriter implements Writer
   final int[] docIDs;
   final long[] ords;
   private int nextWrite;
+  private boolean closed;
 
   public HeapWriter(int count) {
     xs = new int[count];
@@ -45,11 +46,13 @@ final class HeapWriter implements Writer
 
   @Override
   public Reader getReader(long start) {
+    assert closed;
     return new HeapReader(xs, ys, zs, ords, docIDs, (int) start, xs.length);
   }
 
   @Override
   public void close() {
+    closed = true;
     if (nextWrite != xs.length) {
       throw new IllegalStateException("only wrote " + nextWrite + " values, but expected " + xs.length);
     }

Modified: lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java?rev=1696089&r1=1696088&r2=1696089&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java (original)
+++ lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java Sat Aug 15 19:07:52 2015
@@ -34,6 +34,7 @@ final class OfflineWriter implements Wri
   final OutputStreamDataOutput out;
   final long count;
   private long countWritten;
+  private boolean closed;
 
   public OfflineWriter(Path tempDir, long count) throws IOException {
     tempFile = Files.createTempFile(tempDir, "size" + count + ".", "");
@@ -53,11 +54,13 @@ final class OfflineWriter implements Wri
 
   @Override
   public Reader getReader(long start) throws IOException {
+    assert closed;
     return new OfflineReader(tempFile, start, count-start);
   }
 
   @Override
   public void close() throws IOException {
+    closed = true;
     out.close();
     if (count != countWritten) {
       throw new IllegalStateException("wrote " + countWritten + " values, but expected " + count);

Modified: lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/PointInGeo3DShapeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/PointInGeo3DShapeQuery.java?rev=1696089&r1=1696088&r2=1696089&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/PointInGeo3DShapeQuery.java (original)
+++ lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/PointInGeo3DShapeQuery.java Sat Aug 15 19:07:52 2015
@@ -72,9 +72,6 @@ public class PointInGeo3DShapeQuery exte
     // I don't use RandomAccessWeight here: it's no good to approximate with "match all docs"; this is an inverted structure and should be
     // used in the first pass:
 
-    // TODO: except that the polygon verify is costly!  The approximation should be all docs in all overlapping cells, and matches() should
-    // then check the polygon
-
     return new ConstantScoreWeight(this) {
 
       @Override
@@ -97,8 +94,10 @@ public class PointInGeo3DShapeQuery exte
         DocIdSet result = tree.intersect(new BKD3DTreeReader.ValueFilter() {
                                            @Override
                                            public boolean accept(int docID) {
+                                             //System.out.println("  accept? docID=" + docID);
                                              BytesRef bytes = treeDV.get(docID);
                                              if (bytes == null) {
+                                               //System.out.println("    false (null)");
                                                return false;
                                              }
 
@@ -107,28 +106,44 @@ public class PointInGeo3DShapeQuery exte
                                              double y = Geo3DDocValuesFormat.decodeValue(Geo3DDocValuesFormat.readInt(bytes.bytes, bytes.offset+4));
                                              double z = Geo3DDocValuesFormat.decodeValue(Geo3DDocValuesFormat.readInt(bytes.bytes, bytes.offset+8));
                                              // True if x,y,z is within shape
-                                             return shape.isWithin(x,y,z);
+                                             //System.out.println("    x=" + x + " y=" + y + " z=" + z);
+                                             //System.out.println("    ret: " + shape.isWithin(x, y, z));
+
+                                             return shape.isWithin(x, y, z);
                                            }
 
                                            @Override
-                                           public BKD3DTreeReader.Relation compare(int xMinEnc, int xMaxEnc, int yMinEnc, int yMaxEnc, int zMinEnc, int zMaxEnc) {
-                                             double xMin = Geo3DDocValuesFormat.decodeValue(xMinEnc);
-                                             double xMax = Geo3DDocValuesFormat.decodeValue(xMaxEnc);
-                                             double yMin = Geo3DDocValuesFormat.decodeValue(yMinEnc);
-                                             double yMax = Geo3DDocValuesFormat.decodeValue(yMaxEnc);
-                                             double zMin = Geo3DDocValuesFormat.decodeValue(zMinEnc);
-                                             double zMax = Geo3DDocValuesFormat.decodeValue(zMaxEnc);
+                                           public BKD3DTreeReader.Relation compare(int cellXMinEnc, int cellXMaxEnc, int cellYMinEnc, int cellYMaxEnc, int cellZMinEnc, int cellZMaxEnc) {
+                                             assert cellXMinEnc <= cellXMaxEnc;
+                                             assert cellYMinEnc <= cellYMaxEnc;
+                                             assert cellZMinEnc <= cellZMaxEnc;
+
+                                             double cellXMin = Geo3DDocValuesFormat.decodeValue(cellXMinEnc);
+                                             double cellXMax = Geo3DDocValuesFormat.decodeValue(cellXMaxEnc);
+                                             double cellYMin = Geo3DDocValuesFormat.decodeValue(cellYMinEnc);
+                                             double cellYMax = Geo3DDocValuesFormat.decodeValue(cellYMaxEnc);
+                                             double cellZMin = Geo3DDocValuesFormat.decodeValue(cellZMinEnc);
+                                             double cellZMax = Geo3DDocValuesFormat.decodeValue(cellZMaxEnc);
+                                             //System.out.println("  compare: x=" + cellXMin + "-" + cellXMax + " y=" + cellYMin + "-" + cellYMax + " z=" + cellZMin + "-" + cellZMax);
 
-                                             GeoArea xyzSolid = GeoAreaFactory.makeGeoArea(planetModel, xMin, xMax, yMin, yMax, zMin, zMax);
+                                             GeoArea xyzSolid = GeoAreaFactory.makeGeoArea(planetModel, cellXMin, cellXMax, cellYMin, cellYMax, cellZMin, cellZMax);
 
-                                             // nocommit untested!
                                              switch(xyzSolid.getRelationship(shape)) {
                                              case GeoArea.CONTAINS:
+                                               // Shape fully contains the cell
+                                               //System.out.println("    inside");
+                                               return BKD3DTreeReader.Relation.INSIDE;
                                              case GeoArea.OVERLAPS:
+                                               // They do overlap but neither contains the other:
+                                               //System.out.println("    crosses1");
                                                return BKD3DTreeReader.Relation.CROSSES;
                                              case GeoArea.WITHIN:
-                                               return BKD3DTreeReader.Relation.INSIDE;
+                                               // Cell fully contains the shape:
+                                               //System.out.println("    crosses2");
+                                               return BKD3DTreeReader.Relation.CROSSES;
                                              case GeoArea.DISJOINT:
+                                               // They do not overlap at all
+                                               //System.out.println("    outside");
                                                return BKD3DTreeReader.Relation.OUTSIDE;
                                              default:
                                                assert false;

Modified: lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoArea.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoArea.java?rev=1696089&r1=1696088&r2=1696089&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoArea.java (original)
+++ lucene/dev/branches/lucene6699/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoArea.java Sat Aug 15 19:07:52 2015
@@ -33,13 +33,13 @@ public interface GeoArea extends Members
 
   // Relationship values for "getRelationship()"
   
-  /** The referenced shape CONTAINS this shape */
+  /** The referenced shape CONTAINS this area */
   public static final int CONTAINS = 0;
-  /** The referenced shape IS WITHIN this shape */
+  /** The referenced shape IS WITHIN this area */
   public static final int WITHIN = 1;
-  /** The referenced shape OVERLAPS this shape */
+  /** The referenced shape OVERLAPS this area */
   public static final int OVERLAPS = 2;
-  /** The referenced shape has no relation to this shape */
+  /** The referenced shape has no relation to this area */
   public static final int DISJOINT = 3;
 
   /**

Modified: lucene/dev/branches/lucene6699/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6699/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java?rev=1696089&r1=1696088&r2=1696089&view=diff
==============================================================================
--- lucene/dev/branches/lucene6699/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java (original)
+++ lucene/dev/branches/lucene6699/lucene/spatial3d/src/test/org/apache/lucene/bkdtree3d/TestGeo3DPointField.java Sat Aug 15 19:07:52 2015
@@ -17,31 +17,63 @@ package org.apache.lucene.bkdtree3d;
  * limitations under the License.
  */
 
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.lucene53.Lucene53Codec;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.geo3d.GeoBBoxFactory;
 import org.apache.lucene.geo3d.GeoCircle;
+import org.apache.lucene.geo3d.GeoPoint;
+import org.apache.lucene.geo3d.GeoShape;
 import org.apache.lucene.geo3d.PlanetModel;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 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.NumericDocValues;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SimpleCollector;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
+import org.junit.BeforeClass;
 
+import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
 
+import static org.apache.lucene.bkdtree3d.Geo3DDocValuesFormat.decodeValue;
 import static org.apache.lucene.bkdtree3d.Geo3DDocValuesFormat.encodeValue;
 
 public class TestGeo3DPointField extends LuceneTestCase {
 
+  private static boolean smallBBox;
+
+  @BeforeClass
+  public static void beforeClass() {
+    smallBBox = random().nextBoolean();
+    if (VERBOSE) {
+      System.out.println("TEST: smallBBox=" + smallBBox);
+    }
+  }
+
   public void testBasic() throws Exception {
     Directory dir = newDirectory();
     int maxPointsInLeaf = TestUtil.nextInt(random(), 16, 2048);
@@ -303,5 +335,304 @@ public class TestGeo3DPointField extends
     in.close();
     dir.close();
   }
+
+  public void testRandomTiny() throws Exception {
+    // Make sure single-leaf-node case is OK:
+    doTestRandom(10);
+  }
+
+  public void testRandomMedium() throws Exception {
+    doTestRandom(10000);
+  }
+
+  @Nightly
+  public void testRandomBig() throws Exception {
+    doTestRandom(200000);
+  }
+
+  private void doTestRandom(int count) throws Exception {
+    int numPoints = atLeast(count);
+
+    if (VERBOSE) {
+      System.out.println("TEST: numPoints=" + numPoints);
+    }
+
+    double[] lats = new double[numPoints];
+    double[] lons = new double[numPoints];
+
+    boolean haveRealDoc = false;
+
+    for (int docID=0;docID<numPoints;docID++) {
+      int x = random().nextInt(20);
+      if (x == 17) {
+        // Some docs don't have a point:
+        lats[docID] = Double.NaN;
+        if (VERBOSE) {
+          System.out.println("  doc=" + docID + " is missing");
+        }
+        continue;
+      }
+
+      if (docID > 0 && x < 3 && haveRealDoc) {
+        int oldDocID;
+        while (true) {
+          oldDocID = random().nextInt(docID);
+          if (Double.isNaN(lats[oldDocID]) == false) {
+            break;
+          }
+        }
+            
+        if (x == 0) {
+          // Identical lat to old point
+          lats[docID] = lats[oldDocID];
+          lons[docID] = toRadians(randomLon());
+          if (VERBOSE) {
+            System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + lons[docID] + " (same lat as doc=" + oldDocID + ")");
+          }
+        } else if (x == 1) {
+          // Identical lon to old point
+          lats[docID] = toRadians(randomLat());
+          lons[docID] = lons[oldDocID];
+          if (VERBOSE) {
+            System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + lons[docID] + " (same lon as doc=" + oldDocID + ")");
+          }
+        } else {
+          assert x == 2;
+          // Fully identical point:
+          lats[docID] = lats[oldDocID];
+          lons[docID] = lons[oldDocID];
+          if (VERBOSE) {
+            System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + lons[docID] + " (same lat/lon as doc=" + oldDocID + ")");
+          }
+        }
+      } else {
+        lats[docID] = toRadians(randomLat());
+        lons[docID] = toRadians(randomLon());
+        haveRealDoc = true;
+        if (VERBOSE) {
+          System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + lons[docID]);
+        }
+      }
+    }
+
+    verify(lats, lons);
+  }
+
+  private static double randomLat() {
+    if (smallBBox) {
+      return 2.0 * (random().nextDouble()-0.5);
+    } else {
+      return -90 + 180.0 * random().nextDouble();
+    }
+  }
+
+  private static double randomLon() {
+    if (smallBBox) {
+      return 2.0 * (random().nextDouble()-0.5);
+    } else {
+      return -180 + 360.0 * random().nextDouble();
+    }
+  }
+
+  private static void verify(double[] lats, double[] lons) throws Exception {
+    int maxPointsInLeaf = TestUtil.nextInt(random(), 16, 2048);
+    int maxPointsSortInHeap = TestUtil.nextInt(random(), maxPointsInLeaf, 1024*1024);
+    IndexWriterConfig iwc = newIndexWriterConfig();
+
+    PlanetModel planetModel;
+    if (random().nextBoolean()) {
+      planetModel = PlanetModel.WGS84;
+    } else {
+      planetModel = PlanetModel.SPHERE;
+    }
+
+    // Else we can get O(N^2) merging:
+    int mbd = iwc.getMaxBufferedDocs();
+    if (mbd != -1 && mbd < lats.length/100) {
+      iwc.setMaxBufferedDocs(lats.length/100);
+    }
+    final DocValuesFormat dvFormat = new Geo3DDocValuesFormat(maxPointsInLeaf, maxPointsSortInHeap);
+    Codec codec = new Lucene53Codec() {
+        @Override
+        public DocValuesFormat getDocValuesFormatForField(String field) {
+          if (field.equals("point")) {
+            return dvFormat;
+          } else {
+            return super.getDocValuesFormatForField(field);
+          }
+        }
+      };
+    iwc.setCodec(codec);
+    Directory dir;
+    if (lats.length > 100000) {
+      dir = newFSDirectory(createTempDir("TestBKDTree"));
+    } else {
+      dir = newDirectory();
+    }
+    Set<Integer> deleted = new HashSet<>();
+    // RandomIndexWriter is too slow here:
+    IndexWriter w = new IndexWriter(dir, iwc);
+    for(int id=0;id<lats.length;id++) {
+      Document doc = new Document();
+      doc.add(newStringField("id", ""+id, Field.Store.NO));
+      doc.add(new NumericDocValuesField("id", id));
+      if (Double.isNaN(lats[id]) == false) {
+        doc.add(new Geo3DPointField("point", planetModel, lats[id], lons[id]));
+      }
+      w.addDocument(doc);
+      if (id > 0 && random().nextInt(100) == 42) {
+        int idToDelete = random().nextInt(id);
+        w.deleteDocuments(new Term("id", ""+idToDelete));
+        deleted.add(idToDelete);
+        if (VERBOSE) {
+          System.out.println("  delete id=" + idToDelete);
+        }
+      }
+    }
+    if (random().nextBoolean()) {
+      w.forceMerge(1);
+    }
+    final IndexReader r = DirectoryReader.open(w, true);
+    w.close();
+
+    // We can't wrap with "exotic" readers because the geo3d query must see the Geo3DDVFormat:
+    IndexSearcher s = newSearcher(r, false);
+
+    int numThreads = TestUtil.nextInt(random(), 2, 5);
+
+    List<Thread> threads = new ArrayList<>();
+    final int iters = atLeast(100);
+
+    final CountDownLatch startingGun = new CountDownLatch(1);
+    final AtomicBoolean failed = new AtomicBoolean();
+
+    for(int i=0;i<numThreads;i++) {
+      Thread thread = new Thread() {
+          @Override
+          public void run() {
+            try {
+              _run();
+            } catch (Exception e) {
+              failed.set(true);
+              throw new RuntimeException(e);
+            }
+          }
+
+          private void _run() throws Exception {
+            startingGun.await();
+
+            NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
+
+            for (int iter=0;iter<iters && failed.get() == false;iter++) {
+
+              // nocommit randomize other shapes?
+
+              GeoShape shape;
+              if (random().nextBoolean()) {
+                double lat = toRadians(randomLat());
+                double lon = toRadians(randomLon());
+
+                double angle;
+                if (smallBBox) {
+                  angle = random().nextDouble() * Math.PI/360.0;
+                } else {
+                  angle = random().nextDouble() * Math.PI/2.0;
+                }
+
+                if (VERBOSE) {
+                  System.out.println("\nTEST: iter=" + iter + " shape=GeoCircle lat=" + lat + " lon=" + lon + " angle=" + angle);
+                }
+
+                shape = new GeoCircle(planetModel, lat, lon, angle);
+
+              } else {
+                double lat0 = toRadians(randomLat());
+                double lat1 = toRadians(randomLat());
+                if (lat1 < lat0) {
+                  double x = lat0;
+                  lat0 = lat1;
+                  lat1 = x;
+                }
+                double lon0 = toRadians(randomLon());
+                double lon1 = toRadians(randomLon());
+                if (lon1 < lon0) {
+                  double x = lon0;
+                  lon0 = lon1;
+                  lon1 = x;
+                }
+
+                if (VERBOSE) {
+                  System.out.println("\nTEST: iter=" + iter + " shape=GeoBBox lat0=" + lat0 + " lat1=" + lat1 + " lon0=" + lon0 + " lon1=" + lon1);
+                }
+
+                shape = GeoBBoxFactory.makeGeoBBox(planetModel, lat1, lat0, lon0, lon1);
+              }
+
+              Query query = new PointInGeo3DShapeQuery(planetModel, "point", shape);
+
+              if (VERBOSE) {
+                System.out.println("  using query: " + query);
+              }
+
+              final FixedBitSet hits = new FixedBitSet(r.maxDoc());
+
+              s.search(query, new SimpleCollector() {
+
+                  private int docBase;
+
+                  @Override
+                  public boolean needsScores() {
+                    return false;
+                  }
+
+                  @Override
+                  protected void doSetNextReader(LeafReaderContext context) throws IOException {
+                    docBase = context.docBase;
+                  }
+
+                  @Override
+                  public void collect(int doc) {
+                    hits.set(docBase+doc);
+                  }
+                });
+
+              if (VERBOSE) {
+                System.out.println("  hitCount: " + hits.cardinality());
+              }
+      
+              for(int docID=0;docID<r.maxDoc();docID++) {
+                int id = (int) docIDToID.get(docID);
+                if (Double.isNaN(lats[id]) == false) {
+
+                  // Accurate point:
+                  GeoPoint point1 = new GeoPoint(planetModel, lats[id], lons[id]);
+
+                  // Quantized point (32 bits per dim):
+                  GeoPoint point2 = new GeoPoint(decodeValue(encodeValue(point1.x)),
+                                                 decodeValue(encodeValue(point1.y)),
+                                                 decodeValue(encodeValue(point1.z)));
+
+                  boolean expected = deleted.contains(id) == false && shape.isWithin(point2);
+                  if (hits.get(docID) != expected) {
+                    fail(Thread.currentThread().getName() + ": iter=" + iter + " id=" + id + " docID=" + docID + " lat=" + lats[id] + " lon=" + lons[id] + " expected " + expected + " but got: " + hits.get(docID) + " deleted?=" + deleted.contains(id) + "\n  point1=" + point1 + "\n  point2=" + point2 + "\n  query=" + query);
+                  }
+                } else {
+                  assertFalse(hits.get(docID));
+                }
+
+              }
+            }
+          }
+        };
+      thread.setName("T" + i);
+      thread.start();
+      threads.add(thread);
+    }
+    startingGun.countDown();
+    for(Thread thread : threads) {
+      thread.join();
+    }
+    IOUtils.close(r, dir);
+  }
 }