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/09/15 22:01:58 UTC

svn commit: r1703282 - in /lucene/dev/branches/lucene6780/lucene: core/src/java/org/apache/lucene/util/ sandbox/src/java/org/apache/lucene/bkdtree/ sandbox/src/java/org/apache/lucene/search/ sandbox/src/test/org/apache/lucene/bkdtree/

Author: mikemccand
Date: Tue Sep 15 20:01:57 2015
New Revision: 1703282

URL: http://svn.apache.org/r1703282
Log:
LUCENE-6698, LUCENE-6780: add BKDDistanceQuery

Added:
    lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDDistanceQuery.java   (with props)
Modified:
    lucene/dev/branches/lucene6780/lucene/core/src/java/org/apache/lucene/util/SloppyMath.java
    lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInBBoxQuery.java
    lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInPolygonQuery.java
    lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeReader.java
    lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeWriter.java
    lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoBoundingBox.java
    lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointDistanceQuery.java
    lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointDistanceQueryImpl.java
    lucene/dev/branches/lucene6780/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java

Modified: lucene/dev/branches/lucene6780/lucene/core/src/java/org/apache/lucene/util/SloppyMath.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6780/lucene/core/src/java/org/apache/lucene/util/SloppyMath.java?rev=1703282&r1=1703281&r2=1703282&view=diff
==============================================================================
--- lucene/dev/branches/lucene6780/lucene/core/src/java/org/apache/lucene/util/SloppyMath.java (original)
+++ lucene/dev/branches/lucene6780/lucene/core/src/java/org/apache/lucene/util/SloppyMath.java Tue Sep 15 20:01:57 2015
@@ -33,8 +33,10 @@ package org.apache.lucene.util;
 public class SloppyMath {
   
   /**
-   * Returns the distance in kilometers between two points
-   * specified in decimal degrees (latitude/longitude).
+   * Returns the Haversine distance in kilometers between two points
+   * specified in decimal degrees (latitude/longitude).  This works correctly
+   * even if the dateline is between the two points.
+   *
    * @param lat1 Latitude of the first point.
    * @param lon1 Longitude of the first point.
    * @param lat2 Latitude of the second point.

Added: lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDDistanceQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDDistanceQuery.java?rev=1703282&view=auto
==============================================================================
--- lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDDistanceQuery.java (added)
+++ lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDDistanceQuery.java Tue Sep 15 20:01:57 2015
@@ -0,0 +1,246 @@
+package org.apache.lucene.bkdtree;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.GeoBoundingBox;
+import org.apache.lucene.search.GeoPointDistanceQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.GeoProjectionUtils;
+import org.apache.lucene.util.GeoUtils;
+import org.apache.lucene.util.SloppyMath;
+import org.apache.lucene.util.ToStringUtils;
+
+/** Finds all previously indexed points that fall within the specified distance from a center point.
+ *
+ * <p>The field must be indexed with {@link BKDTreeDocValuesFormat}, and {@link BKDPointField} added per document.
+ *
+ * @lucene.experimental */
+
+public class BKDDistanceQuery extends Query {
+  final String field;
+  final double centerLat;
+  final double centerLon;
+  final double radiusMeters;
+  final double minLon, maxLon;
+  final double minLat, maxLat;
+
+  public BKDDistanceQuery(String field, double centerLat, double centerLon, double radiusMeters) {
+    this.field = field;
+    if (BKDTreeWriter.validLat(centerLat) == false) {
+      throw new IllegalArgumentException("centerLat=" + centerLat + " is not a valid latitude");
+    }
+    if (BKDTreeWriter.validLon(centerLon) == false) {
+      throw new IllegalArgumentException("centerLon=" + centerLon + " is not a valid longitude");
+    }
+    if (radiusMeters <= 0.0) {
+      throw new IllegalArgumentException("radiusMeters=" + radiusMeters + " is not a valid radius");
+    }
+    this.centerLat = centerLat;
+    this.centerLon = centerLon;
+    this.radiusMeters = radiusMeters;
+
+    GeoBoundingBox bbox = GeoPointDistanceQuery.computeBBox(centerLon, centerLat, radiusMeters);
+    minLon = bbox.minLon;
+    minLat = bbox.minLat;
+    maxLon = bbox.maxLon;
+    maxLat = bbox.maxLat;
+
+    //System.out.println("distance query bbox: lon=" + minLon + " TO " + maxLon + "; lat=" + minLat + " TO " + maxLat);
+    assert minLat <= maxLat: "minLat=" + minLat + " maxLat=" + maxLat;
+  }
+
+  /** Used by rewrite, when circle crosses the date line */
+  private BKDDistanceQuery(String field, double centerLat, double centerLon, double radiusMeters,
+                           double minLat, double maxLat, double minLon, double maxLon) {
+    this.field = field;
+    assert BKDTreeWriter.validLat(centerLat);
+    assert BKDTreeWriter.validLon(centerLon);
+    assert radiusMeters > 0.0;
+
+    this.centerLat = centerLat;
+    this.centerLon = centerLon;
+    this.radiusMeters = radiusMeters;
+    this.minLat = minLat;
+    this.maxLat = maxLat;
+    this.minLon = minLon;
+    this.maxLon = maxLon;
+  }    
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+
+    // 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:
+
+    return new ConstantScoreWeight(this) {
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        LeafReader reader = context.reader();
+        SortedNumericDocValues sdv = reader.getSortedNumericDocValues(field);
+        if (sdv == null) {
+          // No docs in this segment had this field
+          return null;
+        }
+
+        if (sdv instanceof BKDTreeSortedNumericDocValues == false) {
+          throw new IllegalStateException("field \"" + field + "\" was not indexed with BKDTreeDocValuesFormat: got: " + sdv);
+        }
+        BKDTreeSortedNumericDocValues treeDV = (BKDTreeSortedNumericDocValues) sdv;
+        BKDTreeReader tree = treeDV.getBKDTreeReader();
+
+        DocIdSet result = tree.intersect(minLat, maxLat, minLon, maxLon,
+                                         new BKDTreeReader.LatLonFilter() {
+
+                                           @Override
+                                           public boolean accept(double lat, double lon) {
+                                             double distanceMeters = SloppyMath.haversin(centerLat, centerLon, lat, lon) * 1000.0;
+                                             boolean result = distanceMeters <= radiusMeters;
+                                             System.out.println("accept? centerLat=" + centerLat + " centerLon=" + centerLon + " lat=" + lat + " lon=" + lon + " distanceMeters=" + distanceMeters + " vs " + radiusMeters + " result=" + result);
+                                             return result;
+                                           }
+
+                                           @Override
+                                           public BKDTreeReader.Relation compare(double cellLatMin, double cellLatMax, double cellLonMin, double cellLonMax) {
+                                             //System.out.println("compare lat=" + cellLatMin + " TO " + cellLatMax + "; lon=" + cellLonMin + " TO " + cellLonMax);
+                                             if (GeoUtils.rectWithinCircle(cellLonMin, cellLatMin, cellLonMax, cellLatMax, centerLon, centerLat, radiusMeters)) {
+                                               //System.out.println("  CELL_INSIDE_SHAPE");
+                                               return BKDTreeReader.Relation.CELL_INSIDE_SHAPE;
+                                             } else if (GeoUtils.rectCrossesCircle(cellLonMin, cellLatMin, cellLonMax, cellLatMax, centerLon, centerLat, radiusMeters)) {
+                                               //System.out.println("  SHAPE_CROSSES_CELL");
+                                               return BKDTreeReader.Relation.SHAPE_CROSSES_CELL;
+                                             } else {
+                                               //System.out.println("  SHAPE_OUTSIDE_CELL");
+                                               return BKDTreeReader.Relation.SHAPE_OUTSIDE_CELL;
+                                             }
+                                           }
+                                         }, treeDV.delegate);
+
+        return new ConstantScoreScorer(this, score(), result.iterator());
+      }
+    };
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    // nocommit re-enable, using docsWithField?
+    if (false && radiusMeters >= GeoProjectionUtils.SEMIMINOR_AXIS) {
+      return new MatchAllDocsQuery();
+    }
+
+    if (maxLon < minLon) {
+      // Crosses date line: we just rewrite into OR of two bboxes:
+
+      // Disable coord here because a multi-valued doc could match both circles and get unfairly boosted:
+      BooleanQuery.Builder q = new BooleanQuery.Builder();
+      q.setDisableCoord(true);
+
+      // E.g.: maxLon = -179, minLon = 179
+      BKDDistanceQuery left = new BKDDistanceQuery(field, centerLat, centerLon, radiusMeters, minLat, maxLat, BKDTreeWriter.MIN_LON_INCL, maxLon);
+      q.add(new BooleanClause(left, BooleanClause.Occur.SHOULD));
+      BKDDistanceQuery right = new BKDDistanceQuery(field, centerLat, centerLon, radiusMeters, minLat, maxLat, minLon, BKDTreeWriter.MAX_LON_INCL);
+      q.add(new BooleanClause(right, BooleanClause.Occur.SHOULD));
+      return q.build();
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!(o instanceof BKDDistanceQuery)) return false;
+    if (!super.equals(o)) return false;
+
+    BKDDistanceQuery that = (BKDDistanceQuery) o;
+
+    if (Double.compare(that.centerLat, centerLat) != 0) return false;
+    if (Double.compare(that.centerLon, centerLon) != 0) return false;
+    if (Double.compare(that.radiusMeters, radiusMeters) != 0) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    long temp;
+    temp = Double.doubleToLongBits(centerLon);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    temp = Double.doubleToLongBits(centerLat);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    temp = Double.doubleToLongBits(radiusMeters);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    return result;
+  }
+
+  @Override
+  public String toString(String field) {
+    // nocommit get crossesDateLine into this
+    final StringBuilder sb = new StringBuilder();
+    sb.append(getClass().getSimpleName());
+    sb.append(':');
+    if (!this.field.equals(field)) {
+      sb.append(" field=");
+      sb.append(this.field);
+      sb.append(':');
+    }
+    return sb.append( " Center: [")
+        .append(centerLon)
+        .append(',')
+        .append(centerLat)
+        .append(']')
+        .append(" Distance: ")
+        .append(radiusMeters)
+        .append(" meters")
+        .append("]")
+        .toString();
+  }
+
+  public double getCenterLon() {
+    return this.centerLon;
+  }
+
+  public double getCenterLat() {
+    return this.centerLat;
+  }
+
+  public double getRadiusMeters() {
+    return this.radiusMeters;
+  }
+}

Modified: lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInBBoxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInBBoxQuery.java?rev=1703282&r1=1703281&r2=1703282&view=diff
==============================================================================
--- lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInBBoxQuery.java (original)
+++ lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInBBoxQuery.java Tue Sep 15 20:01:57 2015
@@ -18,6 +18,7 @@ package org.apache.lucene.bkdtree;
  */
 
 import java.io.IOException;
+import java.util.Set;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReader;
@@ -78,7 +79,6 @@ public class BKDPointInBBoxQuery extends
     // used in the first pass:
 
     return new ConstantScoreWeight(this) {
-
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
         LeafReader reader = context.reader();
@@ -96,9 +96,7 @@ public class BKDPointInBBoxQuery extends
 
         DocIdSet result = tree.intersect(minLat, maxLat, minLon, maxLon, null, treeDV.delegate);
 
-        final DocIdSetIterator disi = result.iterator();
-
-        return new ConstantScoreScorer(this, score(), disi);
+        return new ConstantScoreScorer(this, score(), result.iterator());
       }
     };
   }

Modified: lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInPolygonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInPolygonQuery.java?rev=1703282&r1=1703281&r2=1703282&view=diff
==============================================================================
--- lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInPolygonQuery.java (original)
+++ lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInPolygonQuery.java Tue Sep 15 20:01:57 2015
@@ -19,12 +19,15 @@ package org.apache.lucene.bkdtree;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Set;
 
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
@@ -147,9 +150,7 @@ public class BKDPointInPolygonQuery exte
                                            }
                                          }, treeDV.delegate);
 
-        final DocIdSetIterator disi = result.iterator();
-
-        return new ConstantScoreScorer(this, score(), disi);
+        return new ConstantScoreScorer(this, score(), result.iterator());
       }
     };
   }

Modified: lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeReader.java?rev=1703282&r1=1703281&r2=1703282&view=diff
==============================================================================
--- lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeReader.java (original)
+++ lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeReader.java Tue Sep 15 20:01:57 2015
@@ -131,13 +131,39 @@ final class BKDTreeReader implements Acc
     return state.docs.build(hitCount);
   }
 
+  private boolean accept(QueryState state, int docID) throws IOException {
+    //System.out.println("    check accept docID=" + docID);
+    state.sndv.setDocument(docID);
+    // How many values this doc has:
+    int count = state.sndv.count();
+    for(int j=0;j<count;j++) {
+      long enc = state.sndv.valueAt(j);
+
+      int latEnc = (int) ((enc>>32) & 0xffffffffL);
+      int lonEnc = (int) (enc & 0xffffffffL);
+
+      if (latEnc >= state.latMinEnc &&
+          latEnc < state.latMaxEnc &&
+          lonEnc >= state.lonMinEnc &&
+          lonEnc < state.lonMaxEnc &&
+          (state.latLonFilter == null ||
+           state.latLonFilter.accept(BKDTreeWriter.decodeLat(latEnc), BKDTreeWriter.decodeLon(lonEnc)))) {
+        //System.out.println("      yes");
+        return true;
+      }
+    }
+
+    return false;
+  }
+
   /** Fast path: this is called when the query rect fully encompasses all cells under this node. */
   private int addAll(QueryState state, int nodeID) throws IOException {
-
+    //System.out.println("  addAll nodeID=" + nodeID);
     //long latRange = (long) cellLatMaxEnc - (long) cellLatMinEnc;
     //long lonRange = (long) cellLonMaxEnc - (long) cellLonMinEnc;
 
     if (nodeID >= leafNodeOffset) {
+      //System.out.println("    leaf");
 
       /*
       System.out.println("A: " + BKDTreeWriter.decodeLat(cellLatMinEnc)
@@ -161,6 +187,8 @@ final class BKDTreeReader implements Acc
       state.docs.grow(count);
       for(int i=0;i<count;i++) {
         int docID = state.in.readInt();
+        assert accept(state, docID);
+        //System.out.println("  docID=" + docID);
         state.docs.add(docID);
       }
 
@@ -188,6 +216,8 @@ final class BKDTreeReader implements Acc
                         int cellLatMinEnc, int cellLatMaxEnc, int cellLonMinEnc, int cellLonMaxEnc)
     throws IOException {
 
+    //System.out.println("BKD: intersect nodeID=" + nodeID);
+
     // 2.06 sec -> 1.52 sec for 225 OSM London queries:
     if (state.latLonFilter != null) {
 
@@ -230,6 +260,8 @@ final class BKDTreeReader implements Acc
     //System.out.println("\nintersect node=" + nodeID + " vs " + leafNodeOffset);
 
     if (nodeID >= leafNodeOffset) {
+      //System.out.println("  intersect leaf");
+
       // Leaf node; scan and filter all points in this block:
       //System.out.println("    intersect leaf nodeID=" + nodeID + " vs leafNodeOffset=" + leafNodeOffset + " fp=" + leafBlockFPs[nodeID-leafNodeOffset]);
       int hitCount = 0;
@@ -256,27 +288,9 @@ final class BKDTreeReader implements Acc
       state.docs.grow(count);
       for(int i=0;i<count;i++) {
         int docID = state.in.readInt();
-        state.sndv.setDocument(docID);
-        // How many values this doc has:
-        int docValueCount = state.sndv.count();
-        for(int j=0;j<docValueCount;j++) {
-          long enc = state.sndv.valueAt(j);
-
-          int latEnc = (int) ((enc>>32) & 0xffffffffL);
-          int lonEnc = (int) (enc & 0xffffffffL);
-
-          if (latEnc >= state.latMinEnc &&
-              latEnc < state.latMaxEnc &&
-              lonEnc >= state.lonMinEnc &&
-              lonEnc < state.lonMaxEnc &&
-              (state.latLonFilter == null ||
-               state.latLonFilter.accept(BKDTreeWriter.decodeLat(latEnc), BKDTreeWriter.decodeLon(lonEnc)))) {
-            state.docs.add(docID);
-            hitCount++;
-
-            // Stop processing values for this doc:
-            break;
-          }
+        if (accept(state, docID)) {
+          state.docs.add(docID);
+          hitCount++;
         }
       }
 

Modified: lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeWriter.java?rev=1703282&r1=1703281&r2=1703282&view=diff
==============================================================================
--- lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeWriter.java (original)
+++ lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDTreeWriter.java Tue Sep 15 20:01:57 2015
@@ -128,7 +128,6 @@ class BKDTreeWriter {
   }
 
   public void add(double lat, double lon, int docID) throws IOException {
-
     if (validLat(lat) == false) {
       throw new IllegalArgumentException("invalid lat: " + lat);
     }
@@ -691,6 +690,7 @@ class BKDTreeWriter {
         // on those lists:
         int docID = docIDs[i];
         if (docID != lastDocID) {
+          //System.out.println("  docID=" + docID);
           out.writeInt(docID);
           lastDocID = docID;
         }

Modified: lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoBoundingBox.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoBoundingBox.java?rev=1703282&r1=1703281&r2=1703282&view=diff
==============================================================================
--- lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoBoundingBox.java (original)
+++ lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoBoundingBox.java Tue Sep 15 20:01:57 2015
@@ -21,8 +21,8 @@ import org.apache.lucene.util.GeoUtils;
 
 // nocommit rename to GeoRect?  It's not just a bounding box ... it's any rect ...
 
-/** NOTE: package private; just used so {@link GeoPointInPolygonQuery} can communicate its bounding box to {@link GeoPointInBBoxQuery}. */
-class GeoBoundingBox {
+/** Represents a lat/lon rectangle. */
+public class GeoBoundingBox {
   public final double minLon;
   public final double maxLon;
   public final double minLat;

Modified: lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointDistanceQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointDistanceQuery.java?rev=1703282&r1=1703281&r2=1703282&view=diff
==============================================================================
--- lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointDistanceQuery.java (original)
+++ lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointDistanceQuery.java Tue Sep 15 20:01:57 2015
@@ -43,15 +43,15 @@ import org.apache.lucene.util.GeoUtils;
 public final class GeoPointDistanceQuery extends GeoPointInBBoxQuery {
   protected final double centerLon;
   protected final double centerLat;
-  protected final double radius;
+  protected final double radiusMeters;
 
   /** NOTE: radius is in meters. */
-  public GeoPointDistanceQuery(final String field, final double centerLon, final double centerLat, final double radius) {
-    this(field, computeBBox(centerLon, centerLat, radius), centerLon, centerLat, radius);
+  public GeoPointDistanceQuery(final String field, final double centerLon, final double centerLat, final double radiusMeters) {
+    this(field, computeBBox(centerLon, centerLat, radiusMeters), centerLon, centerLat, radiusMeters);
   }
 
   private GeoPointDistanceQuery(final String field, GeoBoundingBox bbox, final double centerLon,
-                                final double centerLat, final double radius) {
+                                final double centerLat, final double radiusMeters) {
     super(field, bbox.minLon, bbox.minLat, bbox.maxLon, bbox.maxLat);
 
     if (GeoUtils.isValidLon(centerLon) == false) {
@@ -62,9 +62,13 @@ public final class GeoPointDistanceQuery
       throw new IllegalArgumentException("invalid centerLat " + centerLat);
     }
 
+    if (radiusMeters <= 0.0) {
+      throw new IllegalArgumentException("invalid radiusMeters " + radiusMeters);
+    }
+
     this.centerLon = centerLon;
     this.centerLat = centerLat;
-    this.radius = radius;
+    this.radiusMeters = radiusMeters;
   }
 
   @Override
@@ -83,10 +87,11 @@ public final class GeoPointDistanceQuery
     return new GeoPointDistanceQueryImpl(field, this, new GeoBoundingBox(this.minLon, this.maxLon, this.minLat, this.maxLat));
   }
 
-  static GeoBoundingBox computeBBox(final double centerLon, final double centerLat, final double radius) {
+  // nocommit move this to util
+  public static GeoBoundingBox computeBBox(final double centerLon, final double centerLat, final double radiusMeters) {
     final double radLat = StrictMath.toRadians(centerLat);
     final double radLon = StrictMath.toRadians(centerLon);
-    double radDistance = (radius + 12000) / GeoProjectionUtils.SEMIMAJOR_AXIS;
+    double radDistance = (radiusMeters + 12000) / GeoProjectionUtils.SEMIMAJOR_AXIS;
     double minLat = radLat - radDistance;
     double maxLat = radLat + radDistance;
     double minLon;
@@ -124,7 +129,7 @@ public final class GeoPointDistanceQuery
 
     if (Double.compare(that.centerLat, centerLat) != 0) return false;
     if (Double.compare(that.centerLon, centerLon) != 0) return false;
-    if (Double.compare(that.radius, radius) != 0) return false;
+    if (Double.compare(that.radiusMeters, radiusMeters) != 0) return false;
 
     return true;
   }
@@ -137,7 +142,7 @@ public final class GeoPointDistanceQuery
     result = 31 * result + (int) (temp ^ (temp >>> 32));
     temp = Double.doubleToLongBits(centerLat);
     result = 31 * result + (int) (temp ^ (temp >>> 32));
-    temp = Double.doubleToLongBits(radius);
+    temp = Double.doubleToLongBits(radiusMeters);
     result = 31 * result + (int) (temp ^ (temp >>> 32));
     return result;
   }
@@ -158,17 +163,8 @@ public final class GeoPointDistanceQuery
         .append(centerLat)
         .append(']')
         .append(" Distance: ")
-        .append(radius)
-        .append(" m")
-        .append(" Lower Left: [")
-        .append(minLon)
-        .append(',')
-        .append(minLat)
-        .append(']')
-        .append(" Upper Right: [")
-        .append(maxLon)
-        .append(',')
-        .append(maxLat)
+        .append(radiusMeters)
+        .append(" meters")
         .append("]")
         .toString();
   }
@@ -181,7 +177,7 @@ public final class GeoPointDistanceQuery
     return this.centerLat;
   }
 
-  public double getRadius() {
-    return this.radius;
+  public double getRadiusMeters() {
+    return this.radiusMeters;
   }
 }

Modified: lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointDistanceQueryImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointDistanceQueryImpl.java?rev=1703282&r1=1703281&r2=1703282&view=diff
==============================================================================
--- lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointDistanceQueryImpl.java (original)
+++ lucene/dev/branches/lucene6780/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointDistanceQueryImpl.java Tue Sep 15 20:01:57 2015
@@ -62,7 +62,7 @@ final class GeoPointDistanceQueryImpl ex
     protected short computeMaxShift() {
       final short shiftFactor;
 
-      if (query.radius > 1000000) {
+      if (query.radiusMeters > 1000000) {
         shiftFactor = 5;
       } else {
         shiftFactor = 4;
@@ -73,12 +73,12 @@ final class GeoPointDistanceQueryImpl ex
 
     @Override
     protected boolean cellCrosses(final double minLon, final double minLat, final double maxLon, final double maxLat) {
-      return GeoUtils.rectCrossesCircle(minLon, minLat, maxLon, maxLat, query.centerLon, query.centerLat, query.radius);
+      return GeoUtils.rectCrossesCircle(minLon, minLat, maxLon, maxLat, query.centerLon, query.centerLat, query.radiusMeters);
     }
 
     @Override
     protected boolean cellWithin(final double minLon, final double minLat, final double maxLon, final double maxLat) {
-      return GeoUtils.rectWithinCircle(minLon, minLat, maxLon, maxLat, query.centerLon, query.centerLat, query.radius);
+      return GeoUtils.rectWithinCircle(minLon, minLat, maxLon, maxLat, query.centerLon, query.centerLat, query.radiusMeters);
     }
 
     @Override
@@ -95,7 +95,7 @@ final class GeoPointDistanceQueryImpl ex
      */
     @Override
     protected boolean postFilter(final double lon, final double lat) {
-      return (SloppyMath.haversin(query.centerLat, query.centerLon, lat, lon) * 1000.0 <= query.radius);
+      return (SloppyMath.haversin(query.centerLat, query.centerLon, lat, lon) * 1000.0 <= query.radiusMeters);
     }
   }
 
@@ -119,7 +119,7 @@ final class GeoPointDistanceQueryImpl ex
     return result;
   }
 
-  public double getRadius() {
-    return query.getRadius();
+  public double getRadiusMeters() {
+    return query.getRadiusMeters();
   }
 }

Modified: lucene/dev/branches/lucene6780/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6780/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java?rev=1703282&r1=1703281&r2=1703282&view=diff
==============================================================================
--- lucene/dev/branches/lucene6780/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java (original)
+++ lucene/dev/branches/lucene6780/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java Tue Sep 15 20:01:57 2015
@@ -17,6 +17,16 @@ package org.apache.lucene.bkdtree;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.lucene53.Lucene53Codec;
@@ -40,30 +50,32 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.GeoDistanceUtils;
+import org.apache.lucene.util.GeoProjectionUtils;
+import org.apache.lucene.util.GeoUtils;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase.Nightly;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.SloppyMath;
 import org.apache.lucene.util.TestUtil;
 import org.junit.BeforeClass;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-
 // TODO: can test framework assert we don't leak temp files?
 
 public class TestBKDTree extends LuceneTestCase {
 
   private static boolean smallBBox;
 
+  // error threshold for point-distance queries (in meters)
+  // @todo haversine is sloppy, would be good to have a better heuristic for
+  // determining the possible haversine error
+
   @BeforeClass
   public static void beforeClass() {
     smallBBox = random().nextBoolean();
+    if (VERBOSE && smallBBox) {
+      System.out.println("TEST: using small bbox");
+    }
   }
 
   public void testAllLatEqual() throws Exception {
@@ -299,55 +311,55 @@ public class TestBKDTree extends LuceneT
 
     boolean haveRealDoc = false;
 
-    for (int docID=0;docID<numPoints;docID++) {
+    for (int id=0;id<numPoints;id++) {
       int x = random().nextInt(20);
       if (x == 17) {
         // Some docs don't have a point:
-        lats[docID] = Double.NaN;
+        lats[id] = Double.NaN;
         if (VERBOSE) {
-          System.out.println("  doc=" + docID + " is missing");
+          System.out.println("  id=" + id + " is missing");
         }
         continue;
       }
 
-      if (docID > 0 && x < 3 && haveRealDoc) {
-        int oldDocID;
+      if (id > 0 && x < 3 && haveRealDoc) {
+        int oldID;
         while (true) {
-          oldDocID = random().nextInt(docID);
-          if (Double.isNaN(lats[oldDocID]) == false) {
+          oldID = random().nextInt(id);
+          if (Double.isNaN(lats[oldID]) == false) {
             break;
           }
         }
             
         if (x == 0) {
           // Identical lat to old point
-          lats[docID] = lats[oldDocID];
-          lons[docID] = randomLon();
+          lats[id] = lats[oldID];
+          lons[id] = randomLon();
           if (VERBOSE) {
-            System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + lons[docID] + " (same lat as doc=" + oldDocID + ")");
+            System.out.println("  id=" + id + " lat=" + lats[id] + " lon=" + lons[id] + " (same lat as doc=" + oldID + ")");
           }
         } else if (x == 1) {
           // Identical lon to old point
-          lats[docID] = randomLat();
-          lons[docID] = lons[oldDocID];
+          lats[id] = randomLat();
+          lons[id] = lons[oldID];
           if (VERBOSE) {
-            System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + lons[docID] + " (same lon as doc=" + oldDocID + ")");
+            System.out.println("  id=" + id + " lat=" + lats[id] + " lon=" + lons[id] + " (same lon as doc=" + oldID + ")");
           }
         } else {
           assert x == 2;
           // Fully identical point:
-          lats[docID] = lats[oldDocID];
-          lons[docID] = lons[oldDocID];
+          lats[id] = lats[oldID];
+          lons[id] = lons[oldID];
           if (VERBOSE) {
-            System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + lons[docID] + " (same lat/lon as doc=" + oldDocID + ")");
+            System.out.println("  id=" + id + " lat=" + lats[id] + " lon=" + lons[id] + " (same lat/lon as doc=" + oldID + ")");
           }
         }
       } else {
-        lats[docID] = randomLat();
-        lons[docID] = randomLon();
+        lats[id] = randomLat();
+        lons[id] = randomLon();
         haveRealDoc = true;
         if (VERBOSE) {
-          System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + lons[docID]);
+          System.out.println("  id=" + id + " lat=" + lats[id] + " lon=" + lons[id]);
         }
       }
     }
@@ -402,7 +414,8 @@ public class TestBKDTree extends LuceneT
         }
       }
     }
-    if (random().nextBoolean()) {
+    // nocommit
+    if (true || random().nextBoolean()) {
       w.forceMerge(1);
     }
     final IndexReader r = DirectoryReader.open(w, true);
@@ -412,6 +425,8 @@ public class TestBKDTree extends LuceneT
     IndexSearcher s = newSearcher(r, false);
 
     int numThreads = TestUtil.nextInt(random(), 2, 5);
+    // nocommit
+    numThreads = 1;
 
     List<Thread> threads = new ArrayList<>();
     final int iters = atLeast(100);
@@ -437,100 +452,105 @@ public class TestBKDTree extends LuceneT
             NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
 
             for (int iter=0;iter<iters && failed.get() == false;iter++) {
-              double lat0 = randomLat();
-              double lat1 = randomLat();
-              double lon0 = randomLon();
-              double lon1 = randomLon();
-
-              if (lat1 < lat0) {
-                double x = lat0;
-                lat0 = lat1;
-                lat1 = x;
+
+              if (VERBOSE) {
+                System.out.println("\nTEST: iter=" + iter + " s=" + s);
               }
+              Query query;
+              VerifyHits verifyHits;
+
+              if (random().nextBoolean()) {
+                // BBox 
+                final GeoBoundingBox bbox = randomBBox(true);
+
+                query = new BKDPointInBBoxQuery("point", bbox.minLat, bbox.maxLat, bbox.minLon, bbox.maxLon);
+
+                verifyHits = new VerifyHits() {
+                    @Override
+                    protected Boolean shouldMatch(double pointLat, double pointLon) {
+                      return rectContainsPointEnc(bbox.minLat, bbox.maxLat, bbox.minLon, bbox.maxLon, pointLat, pointLon);
+                    }
+                    @Override
+                    protected void describe(int docID, double lat, double lon) {
+                    }
+                  };
+
+              // nocommit change this to always temporarily for more efficient beasting:
+              } else if (random().nextBoolean()) {
+                // Distance
+
+                final double centerLat = randomLat();
+                final double centerLon = randomLon();
 
-              boolean crossesDateLine;
-              if (lon1 < lon0) {
-                if (random().nextBoolean()) {
-                  double x = lon0;
-                  lon0 = lon1;
-                  lon1 = x;
-                  crossesDateLine = false;
-                } else {
-                  crossesDateLine = true;
+                // nocommit is this max value right (i want to at most span the entire earth)?:
+                final double radiusMeters = random().nextDouble() * GeoProjectionUtils.SEMIMAJOR_AXIS * 2.0 * Math.PI;
+
+                if (VERBOSE) {
+                  System.out.println("  radiusMeters = " + new DecimalFormat("#,###.00").format(radiusMeters));
                 }
-              } else {
-                crossesDateLine = false;
-              }
 
-              if (VERBOSE) {
-                System.out.println("\nTEST: iter=" + iter + " lat=" + lat0 + " TO " + lat1 + " lon=" + lon0 + " TO " + lon1 + " crossesDateLine=" + crossesDateLine);
-              }
+                query = new BKDDistanceQuery("point", centerLat, centerLon, radiusMeters);
+
+                verifyHits = new VerifyHits() {
+                    @Override
+                    protected Boolean shouldMatch(double pointLat, double pointLon) {
+                      double distanceKM = SloppyMath.haversin(centerLat, centerLon, pointLat, pointLon);
+                      boolean result = distanceKM*1000.0 <= radiusMeters;
+                      //System.out.println("  shouldMatch?  centerLon=" + centerLon + " centerLat=" + centerLat + " pointLon=" + pointLon + " pointLat=" + pointLat + " result=" + result + " distanceMeters=" + (distanceKM * 1000));
+                      return result;
+                    }
+
+                    @Override
+                    protected void describe(int docID, double pointLat, double pointLon) {
+                      double distanceKM = SloppyMath.haversin(centerLat, centerLon, pointLat, pointLon);
+                      System.out.println("  docID=" + docID + " centerLon=" + centerLon + " centerLat=" + centerLat + " pointLon=" + pointLon + " pointLat=" + pointLat + " distanceMeters=" + (distanceKM * 1000) + " vs radiusMeters=" + radiusMeters);
+                    }
+                   };
 
-              Query query;
               // TODO: get poly query working with dateline crossing too (how?)!
-              if (crossesDateLine || random().nextBoolean()) {
-                query = new BKDPointInBBoxQuery("point", lat0, lat1, lon0, lon1);
               } else {
+                final GeoBoundingBox bbox = randomBBox(false);
+
+                // Polygon
                 double[] lats = new double[5];
                 double[] lons = new double[5];
-                lats[0] = lat0;
-                lons[0] = lon0;
-                lats[1] = lat1;
-                lons[1] = lon0;
-                lats[2] = lat1;
-                lons[2] = lon1;
-                lats[3] = lat0;
-                lons[3] = lon1;
-                lats[4] = lat0;
-                lons[4] = lon0;
+                lats[0] = bbox.minLat;
+                lons[0] = bbox.minLon;
+                lats[1] = bbox.maxLat;
+                lons[1] = bbox.minLon;
+                lats[2] = bbox.maxLat;
+                lons[2] = bbox.maxLon;
+                lats[3] = bbox.minLat;
+                lons[3] = bbox.maxLon;
+                lats[4] = bbox.minLat;
+                lons[4] = bbox.minLon;
                 query = new BKDPointInPolygonQuery("point", lats, lons);
-              }
 
-              if (VERBOSE) {
-                System.out.println("  using query: " + query);
+                verifyHits = new VerifyHits() {
+                    @Override
+                    protected Boolean shouldMatch(double pointLat, double pointLon) {
+                      if (Math.abs(bbox.minLat-pointLat) < TOLERANCE ||
+                          Math.abs(bbox.maxLat-pointLat) < TOLERANCE ||
+                          Math.abs(bbox.minLon-pointLon) < TOLERANCE ||
+                          Math.abs(bbox.maxLon-pointLon) < TOLERANCE) {
+                        // The poly check quantizes slightly differently, so we allow for boundary cases to disagree
+                        return null;
+                      } else {
+                        return rectContainsPointEnc(bbox.minLat, bbox.maxLat, bbox.minLon, bbox.maxLon, pointLat, pointLon);
+                      }
+                    }
+
+                    @Override
+                    protected void describe(int docID, double lat, double lon) {
+                    }
+                  };
               }
 
-              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);
-                boolean expected = deleted.contains(id) == false && rectContainsPointEnc(lat0, lat1, lon0, lon1, lats[id], lons[id]);
-                if (hits.get(docID) != expected) {
-                  if (query instanceof BKDPointInPolygonQuery &&
-                      (Math.abs(lat0-lats[id]) < TOLERANCE ||
-                       Math.abs(lat1-lats[id]) < TOLERANCE ||
-                       Math.abs(lon0-lons[id]) < TOLERANCE ||
-                       Math.abs(lon1-lons[id]) < TOLERANCE)) {
-                    // The poly check quantizes slightly differently, so we allow for boundary cases to disagree
-                  } else {
-                    // We do exact quantized comparison so the bbox query should never disagree:
-                    fail(Thread.currentThread().getName() + ": iter=" + iter + " id=" + id + " docID=" + docID + " lat=" + lats[id] + " lon=" + lons[id] + " (bbox: lat=" + lat0 + " TO " + lat1 + " lon=" + lon0 + " TO " + lon1 + ") expected " + expected + " but got: " + hits.get(docID) + " deleted?=" + deleted.contains(id) + " query=" + query + " crossesDateLine=" + crossesDateLine);
-                  }
-                }
+                System.out.println("  query=" + query);
               }
+
+              verifyHits.test(s, docIDToID, deleted, query, lats, lons);
             }
           }
         };
@@ -550,8 +570,8 @@ public class TestBKDTree extends LuceneT
                                               double pointLat, double pointLon) {
     if (Double.isNaN(pointLat)) {
       return false;
-    }
-    int rectLatMinEnc = BKDTreeWriter.encodeLat(rectLatMin);
+    } 
+   int rectLatMinEnc = BKDTreeWriter.encodeLat(rectLatMin);
     int rectLatMaxEnc = BKDTreeWriter.encodeLat(rectLatMax);
     int rectLonMinEnc = BKDTreeWriter.encodeLon(rectLonMin);
     int rectLonMaxEnc = BKDTreeWriter.encodeLon(rectLonMax);
@@ -601,6 +621,14 @@ public class TestBKDTree extends LuceneT
     }
   }
 
+  static double quantizeLat(double lat) {
+    return BKDTreeWriter.decodeLat(BKDTreeWriter.encodeLat(lat));
+  }
+
+  static double quantizeLon(double lon) {
+    return BKDTreeWriter.decodeLon(BKDTreeWriter.encodeLon(lon));
+  }
+
   public void testEncodeDecodeMax() throws Exception {
     int x = BKDTreeWriter.encodeLat(Math.nextAfter(90.0, Double.POSITIVE_INFINITY));
     assertTrue(x < Integer.MAX_VALUE);
@@ -632,6 +660,126 @@ public class TestBKDTree extends LuceneT
   private static DocValuesFormat getDocValuesFormat() {
     int maxPointsInLeaf = TestUtil.nextInt(random(), 16, 2048);
     int maxPointsSortInHeap = TestUtil.nextInt(random(), maxPointsInLeaf, 1024*1024);
+    if (VERBOSE) {
+      System.out.println("  BKD params: maxPointsInLeaf=" + maxPointsInLeaf + " maxPointsSortInHeap=" + maxPointsSortInHeap);
+    }
     return new BKDTreeDocValuesFormat(maxPointsInLeaf, maxPointsSortInHeap);
   }
+
+  private static abstract class VerifyHits {
+
+    public void test(IndexSearcher s, NumericDocValues docIDToID, Set<Integer> deleted, Query query, double[] lats, double[] lons) throws Exception {
+      int maxDoc = s.getIndexReader().maxDoc();
+      final FixedBitSet hits = new FixedBitSet(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);
+          }
+        });
+
+      for(int docID=0;docID<maxDoc;docID++) {
+        int id = (int) docIDToID.get(docID);
+        Boolean expected;
+        if (deleted.contains(id)) {
+          expected = false;
+        } else if (Double.isNaN(lats[id])) {
+          expected = false;
+        } else {
+          expected = shouldMatch(lats[id], lons[id]);
+        }
+
+        // null means it's a borderline case which is allowed to be wrong:
+        if (expected != null) {
+          if (hits.get(docID) != expected) {
+            if (expected) {
+              System.out.println(Thread.currentThread().getName() + ": id=" + id + " should match but did not");
+            } else {
+              System.out.println(Thread.currentThread().getName() + ": id=" + id + " should not match but did");
+            }
+            System.out.println("  query=" + query +
+                               " docID=" + docID + "\n  lat=" + lats[id] + " lon=" + lons[id] +
+                               "\n  deleted?=" + deleted.contains(id));
+            if (Double.isNaN(lats[id]) == false) {
+              describe(docID, lats[id], lons[id]);
+            }
+            fail("wrong hit");
+          }
+        }
+      }
+    }
+
+    /** Return true if we definitely should match, false if we definitely
+     *  should not match, and null if it's a borderline case which might
+     *  go either way. */
+    protected abstract Boolean shouldMatch(double lat, double lon);
+
+    protected abstract void describe(int docID, double lat, double lon);
+  }
+
+  private static GeoBoundingBox randomBBox(boolean canCrossDateLine) {
+    double lat0 = randomLat();
+    double lat1 = randomLat();
+    double lon0 = randomLon();
+    double lon1 = randomLon();
+
+    if (lat1 < lat0) {
+      double x = lat0;
+      lat0 = lat1;
+      lat1 = x;
+    }
+
+    if (canCrossDateLine == false && lon1 < lon0) {
+      double x = lon0;
+      lon0 = lon1;
+      lon1 = x;
+    }
+
+    // Don't fixup lon0/lon1, so we can sometimes cross dateline:
+    return new GeoBoundingBox(lon0, lon1, lat0, lat1);
+  }
+
+  static class GeoBoundingBox {
+    public final double minLon;
+    public final double maxLon;
+    public final double minLat;
+    public final double maxLat;
+
+    public GeoBoundingBox(double minLon, double maxLon, double minLat, double maxLat) {
+      if (GeoUtils.isValidLon(minLon) == false) {
+        throw new IllegalArgumentException("invalid minLon " + minLon);
+      }
+      if (GeoUtils.isValidLon(maxLon) == false) {
+        throw new IllegalArgumentException("invalid maxLon " + minLon);
+      }
+      if (GeoUtils.isValidLat(minLat) == false) {
+        throw new IllegalArgumentException("invalid minLat " + minLat);
+      }
+      if (GeoUtils.isValidLat(maxLat) == false) {
+        throw new IllegalArgumentException("invalid maxLat " + minLat);
+      }
+      this.minLon = minLon;
+      this.maxLon = maxLon;
+      this.minLat = minLat;
+      this.maxLat = maxLat;
+    }
+
+    @Override
+    public String toString() {
+      return "GeoBoundingBox(lat=" + minLat + " TO " + maxLat + " lon=" + minLon + " TO " + maxLon + " crossesDateLine=" + (maxLon < minLon) + ")";
+    }
+  }
 }