You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by nk...@apache.org on 2015/10/21 23:59:09 UTC

svn commit: r1709926 [2/2] - in /lucene/dev/trunk/lucene: core/src/java/org/apache/lucene/util/ sandbox/src/java/org/apache/lucene/bkdtree/ sandbox/src/java/org/apache/lucene/document/ sandbox/src/java/org/apache/lucene/search/ sandbox/src/java/org/apa...

Modified: lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/search/TestGeoPointQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/search/TestGeoPointQuery.java?rev=1709926&r1=1709925&r2=1709926&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/search/TestGeoPointQuery.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/search/TestGeoPointQuery.java Wed Oct 21 21:59:08 2015
@@ -17,58 +17,61 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-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 org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.GeoPointField;
-import org.apache.lucene.document.NumericDocValuesField;
-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.RandomIndexWriter;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.GeoDistanceUtils;
-import org.apache.lucene.util.GeoProjectionUtils;
+import org.apache.lucene.util.BaseGeoPointTestCase;
+import org.apache.lucene.util.GeoRect;
 import org.apache.lucene.util.GeoUtils;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.SloppyMath;
-import org.apache.lucene.util.TestGeoUtils;
 import org.apache.lucene.util.TestUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Test;
 
 /**
  * Unit testing for basic GeoPoint query logic
  *
  * @lucene.experimental
  */
-public class TestGeoPointQuery extends LuceneTestCase {
+
+public class TestGeoPointQuery extends BaseGeoPointTestCase {
+
   private static Directory directory = null;
   private static IndexReader reader = null;
   private static IndexSearcher searcher = null;
 
-  private static final String FIELD_NAME = "geoField";
+  // error threshold for point-distance queries (in percent) NOTE: Guideline from USGS
+  private static final double DISTANCE_PCT_ERR = 0.005;
+
+  @Override
+  protected void addPointToDoc(String field, Document doc, double lat, double lon) {
+    doc.add(new GeoPointField(field, lon, lat, Field.Store.NO));
+  }
+
+  @Override
+  protected Query newBBoxQuery(String field, GeoRect rect) {
+    return new GeoPointInBBoxQuery(field, rect.minLon, rect.minLat, rect.maxLon, rect.maxLat);
+  }
+
+  @Override
+  protected Query newDistanceQuery(String field, double centerLat, double centerLon, double radiusMeters) {
+    return new GeoPointDistanceQuery(field, centerLon, centerLat, radiusMeters);
+  }
 
-  // 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
-  private static final int DISTANCE_ERR = 1000;
+  @Override
+  protected Query newDistanceRangeQuery(String field, double centerLat, double centerLon, double minRadiusMeters, double radiusMeters) {
+    return new GeoPointDistanceRangeQuery(field, centerLon, centerLat, minRadiusMeters, radiusMeters);
+  }
+
+  @Override
+  protected Query newPolygonQuery(String field, double[] lats, double[] lons) {
+    return new GeoPointInPolygonQuery(field, lons, lats);
+  }
 
   @BeforeClass
   public static void beforeClass() throws Exception {
@@ -143,19 +146,81 @@ public class TestGeoPointQuery extends L
     return searcher.search(q, limit);
   }
 
+  @Override
+  protected Boolean rectContainsPoint(GeoRect rect, double pointLat, double pointLon) {
+    if (GeoUtils.compare(pointLon, rect.minLon) == 0.0 ||
+        GeoUtils.compare(pointLon, rect.maxLon) == 0.0 ||
+        GeoUtils.compare(pointLat, rect.minLat) == 0.0 ||
+        GeoUtils.compare(pointLat, rect.maxLat) == 0.0) {
+      // Point is very close to rect boundary
+      return null;
+    }
+
+    if (rect.minLon < rect.maxLon) {
+      return GeoUtils.bboxContains(pointLon, pointLat, rect.minLon, rect.minLat, rect.maxLon, rect.maxLat);
+    } else {
+      // Rect crosses dateline:
+      return GeoUtils.bboxContains(pointLon, pointLat, -180.0, rect.minLat, rect.maxLon, rect.maxLat)
+          || GeoUtils.bboxContains(pointLon, pointLat, rect.minLon, rect.minLat, 180.0, rect.maxLat);
+    }
+  }
+
+  @Override
+  protected Boolean polyRectContainsPoint(GeoRect rect, double pointLat, double pointLon) {
+    return rectContainsPoint(rect, pointLat, pointLon);
+  }
+
+  @Override
+  protected Boolean circleContainsPoint(double centerLat, double centerLon, double radiusMeters, double pointLat, double pointLon) {
+    if (radiusQueryCanBeWrong(centerLat, centerLon, pointLon, pointLat, radiusMeters)) {
+      return null;
+    } else {
+      return SloppyMath.haversin(centerLat, centerLon, pointLat, pointLon)*1000.0 <= radiusMeters;
+    }
+  }
+
+  @Override
+  protected Boolean distanceRangeContainsPoint(double centerLat, double centerLon, double minRadiusMeters, double radiusMeters, double pointLat, double pointLon) {
+    if (radiusQueryCanBeWrong(centerLat, centerLon, pointLon, pointLat, minRadiusMeters)
+        || radiusQueryCanBeWrong(centerLat, centerLon, pointLon, pointLat, radiusMeters)) {
+      return null;
+    } else {
+      final double d = SloppyMath.haversin(centerLat, centerLon, pointLat, pointLon)*1000.0;
+      return d >= minRadiusMeters && d <= radiusMeters;
+    }
+  }
+
+  private static boolean radiusQueryCanBeWrong(double centerLat, double centerLon, double ptLon, double ptLat,
+                                               final double radius) {
+    final long hashedCntr = GeoUtils.mortonHash(centerLon, centerLat);
+    centerLon = GeoUtils.mortonUnhashLon(hashedCntr);
+    centerLat = GeoUtils.mortonUnhashLat(hashedCntr);
+    final long hashedPt = GeoUtils.mortonHash(ptLon, ptLat);
+    ptLon = GeoUtils.mortonUnhashLon(hashedPt);
+    ptLat = GeoUtils.mortonUnhashLat(hashedPt);
+
+    double ptDistance = SloppyMath.haversin(centerLat, centerLon, ptLat, ptLon)*1000.0;
+    double delta = StrictMath.abs(ptDistance - radius);
+
+    // if its within the distance error then it can be wrong
+    return delta < (ptDistance*DISTANCE_PCT_ERR);
+  }
+
+  public void testRectCrossesCircle() throws Exception {
+    assertTrue(GeoUtils.rectCrossesCircle(-180, -90, 180, 0.0, 0.667, 0.0, 88000.0));
+  }
+
   private TopDocs geoDistanceRangeQuery(double lon, double lat, double minRadius, double maxRadius, int limit)
       throws Exception {
     GeoPointDistanceRangeQuery q = new GeoPointDistanceRangeQuery(FIELD_NAME, lon, lat, minRadius, maxRadius);
     return searcher.search(q, limit);
   }
 
-  @Test
   public void testBBoxQuery() throws Exception {
     TopDocs td = bboxQuery(-96.7772, 32.778650, -96.77690000, 32.778950, 5);
     assertEquals("GeoBoundingBoxQuery failed", 4, td.totalHits);
   }
 
-  @Test
   public void testPolyQuery() throws Exception {
     TopDocs td = polygonQuery(new double[]{-96.7682647, -96.8280029, -96.6288757, -96.4929199,
             -96.6041564, -96.7449188, -96.76826477, -96.7682647},
@@ -164,7 +229,6 @@ public class TestGeoPointQuery extends L
     assertEquals("GeoPolygonQuery failed", 2, td.totalHits);
   }
 
-  @Test
   public void testPacManPolyQuery() throws Exception {
     // pacman
     double[] px = {0, 10, 10, 0, -8, -10, -8, 0, 10, 10, 0};
@@ -188,25 +252,21 @@ public class TestGeoPointQuery extends L
     assertTrue(GeoUtils.rectWithinPoly(-5, 0, -2, 5, px, py, xMin, yMin, xMax, yMax));
   }
 
-  @Test
   public void testBBoxCrossDateline() throws Exception {
     TopDocs td = bboxQuery(179.0, -45.0, -179.0, -44.0, 20);
     assertEquals("BBoxCrossDateline query failed", 2, td.totalHits);
   }
 
-  @Test
   public void testWholeMap() throws Exception {
     TopDocs td = bboxQuery(-179.9, -89.9, 179.9, 89.9, 20);
     assertEquals("testWholeMap failed", 24, td.totalHits);
   }
 
-  @Test
   public void smallTest() throws Exception {
     TopDocs td = geoDistanceQuery(-73.998776, 40.720611, 1, 20);
     assertEquals("smallTest failed", 2, td.totalHits);
   }
 
-  @Test
   public void testInvalidBBox() throws Exception {
     try {
       bboxQuery(179.0, -92.0, 181.0, -91.0, 20);
@@ -216,35 +276,38 @@ public class TestGeoPointQuery extends L
     throw new Exception("GeoBoundingBox should not accept invalid lat/lon");
   }
 
-  @Test
   public void testGeoDistanceQuery() throws Exception {
     TopDocs td = geoDistanceQuery(-96.4538113027811, 32.94823588839368, 6000, 20);
     assertEquals("GeoDistanceQuery failed", 2, td.totalHits);
   }
 
-  @Test
   public void testMultiValuedQuery() throws Exception {
     TopDocs td = bboxQuery(-96.4538113027811, 32.7559529921407, -96.7706036567688, 32.7756745755423, 20);
     // 3 single valued docs + 2 multi-valued docs
     assertEquals("testMultiValuedQuery failed", 5, td.totalHits);
   }
 
+  public void testTooBigRadius() throws Exception {
+    try {
+      geoDistanceQuery(0.0, 85.0, 4000000, 20);
+    } catch (IllegalArgumentException e) {
+      e.getMessage().contains("exceeds maxRadius");
+    }
+  }
+
   /**
    * Explicitly large
    */
-  @Nightly
   public void testGeoDistanceQueryHuge() throws Exception {
-    TopDocs td = geoDistanceQuery(-96.4538113027811, 32.94823588839368, 2000000, 20);
-    assertEquals("GeoDistanceQuery failed", 13, td.totalHits);
+    TopDocs td = geoDistanceQuery(-96.4538113027811, 32.94823588839368, 6000000, 20);
+    assertEquals("GeoDistanceQuery failed",18, td.totalHits);
   }
 
-  @Test
   public void testGeoDistanceQueryCrossDateline() throws Exception {
     TopDocs td = geoDistanceQuery(-179.9538113027811, 32.94823588839368, 120000, 20);
     assertEquals("GeoDistanceQuery failed", 3, td.totalHits);
   }
 
-  @Test
   public void testInvalidGeoDistanceQuery() throws Exception {
     try {
       geoDistanceQuery(181.0, 92.0, 120000, 20);
@@ -254,413 +317,14 @@ public class TestGeoPointQuery extends L
     throw new Exception("GeoDistanceQuery should not accept invalid lat/lon as origin");
   }
 
-  @Test
   public void testMaxDistanceRangeQuery() throws Exception {
-    TopDocs td = geoDistanceRangeQuery(0.0, 0.0, 10000, GeoProjectionUtils.SEMIMINOR_AXIS, 20);
+    TopDocs td = geoDistanceRangeQuery(0.0, 0.0, 10, 20000000, 20);
     assertEquals("GeoDistanceRangeQuery failed", 24, td.totalHits);
   }
 
-  public void testRandomTiny() throws Exception {
-    // Make sure single-leaf-node case is OK:
-    doTestRandom(10);
-  }
-
-  public void testRandom() throws Exception {
-    doTestRandom(10000);
-  }
-
-  @Test
   public void testMortonEncoding() throws Exception {
     long hash = GeoUtils.mortonHash(180, 90);
     assertEquals(180.0, GeoUtils.mortonUnhashLon(hash), 0);
     assertEquals(90.0, GeoUtils.mortonUnhashLat(hash), 0);
   }
-
-  @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] = TestGeoUtils.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] = TestGeoUtils.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] = TestGeoUtils.randomLat();
-        lons[docID] = TestGeoUtils.randomLon();
-        haveRealDoc = true;
-        if (VERBOSE) {
-          //System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + lons[docID]);
-        }
-      }
-    }
-
-    verify(lats, lons);
-  }
-
-  private static void verify(double[] lats, double[] lons) throws Exception {
-    IndexWriterConfig iwc = newIndexWriterConfig();
-    Directory dir;
-    if (lats.length > 100000) {
-      dir = newFSDirectory(createTempDir("TestGeoPointQuery"));
-      iwc.setCodec(TestUtil.getDefaultCodec());
-    } 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) {
-        if (VERBOSE) {
-          System.out.println("  id=" + id + " lat=" + lats[id] + " lon=" + lons[id]);
-        }
-        doc.add(new GeoPointField(FIELD_NAME, lons[id], lats[id], Field.Store.NO));
-      } else if (VERBOSE) {
-        System.out.println("  id=" + id + " skipped");
-      }
-      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);
-    }
-    IndexReader r = DirectoryReader.open(w, true);
-    w.close();
-
-    IndexSearcher s = newSearcher(r);
-
-    // Make sure queries are thread safe:
-    int numThreads = TestUtil.nextInt(random(), 2, 5);
-
-    List<Thread> threads = new ArrayList<>();
-    final int iters = atLeast(10);
-
-    final CountDownLatch startingGun = new CountDownLatch(1);
-
-    for(int i=0;i<numThreads;i++) {
-      Thread thread = new Thread() {
-          @Override
-          public void run() {
-            try {
-              _run();
-            } catch (Exception e) {
-              throw new RuntimeException(e);
-            }
-          }
-
-          private void _run() throws Exception {
-            startingGun.await();
-
-            NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
-
-            for (int iter=0;iter<iters;iter++) {
-              if (VERBOSE) {
-                System.out.println("\nTEST: iter=" + iter);
-              }
-
-              Query query;
-
-              VerifyHits verifyHits;
-
-              if (random().nextBoolean()) {
-                GeoBoundingBox bbox = randomBBox();
-
-                query = new GeoPointInBBoxQuery(FIELD_NAME, bbox.minLon, bbox.minLat, bbox.maxLon, bbox.maxLat);
-                verifyHits = new VerifyHits() {
-                    @Override
-                    protected Boolean shouldMatch(double pointLat, double pointLon) {
-
-                      // morton encode & decode to compare apples to apples (that is, compare with same hash precision error
-                      // present in the index)
-                      long pointHash = GeoUtils.mortonHash(pointLon, pointLat);
-                      pointLon = GeoUtils.mortonUnhashLon(pointHash);
-                      pointLat = GeoUtils.mortonUnhashLat(pointHash);
-
-                      if (bboxQueryCanBeWrong(bbox, pointLat, pointLon)) {
-                        return null;
-                      } else {
-                        return rectContainsPointEnc(bbox, pointLat, pointLon);
-                      }
-                    }
-                   };
-              } else if (random().nextBoolean()) {
-                // generate a random bounding box
-                GeoBoundingBox bbox = randomBBox();
-
-                double centerLat = bbox.minLat + ((bbox.maxLat - bbox.minLat)/2.0);
-                double centerLon = bbox.minLon + ((bbox.maxLon - bbox.minLon)/2.0);
-
-                // radius (in meters) as a function of the random generated bbox
-                final double radius =  random().nextDouble() * (0.05 * GeoProjectionUtils.SEMIMINOR_AXIS);
-
-                // randomly test range queries
-                final boolean rangeQuery = random().nextBoolean();
-                final double radiusMax = (rangeQuery) ? radius + random().nextDouble() * (0.05 * GeoProjectionUtils.SEMIMINOR_AXIS) : 0;
-
-                if (VERBOSE) {
-                  System.out.println("\t radius = " + radius + ((rangeQuery) ? " : " + radiusMax : ""));
-                }
-
-                // query using the centroid of the bounding box
-                if (rangeQuery) {
-                  query = new GeoPointDistanceRangeQuery(FIELD_NAME, centerLon, centerLat, radius, radiusMax);
-                } else {
-                  query = new GeoPointDistanceQuery(FIELD_NAME, centerLon, centerLat, radius);
-                }
-
-                verifyHits = new VerifyHits() {
-                    @Override
-                    protected Boolean shouldMatch(double pointLat, double pointLon) {
-                      if (Double.isNaN(pointLat) || Double.isNaN(pointLon)) {
-                        return null;
-                      }
-                      if (radiusQueryCanBeWrong(centerLat, centerLon, pointLon, pointLat, radius)
-                          || (rangeQuery && radiusQueryCanBeWrong(centerLat, centerLon, pointLon, pointLat, radiusMax))) {
-                        return null;
-                      } else {
-                        return distanceContainsPt(centerLon, centerLat, pointLon, pointLat, radius, (rangeQuery) ? radiusMax : 0);
-                      }
-                    }
-                   };
-              } else {
-                GeoBoundingBox bbox = randomBBox();
-
-                double[] pLats = new double[5];
-                double[] pLons = new double[5];
-                pLats[0] = bbox.minLat;
-                pLons[0] = bbox.minLon;
-                pLats[1] = bbox.maxLat;
-                pLons[1] = bbox.minLon;
-                pLats[2] = bbox.maxLat;
-                pLons[2] = bbox.maxLon;
-                pLats[3] = bbox.minLat;
-                pLons[3] = bbox.maxLon;
-                pLats[4] = bbox.minLat;
-                pLons[4] = bbox.minLon;
-                query = new GeoPointInPolygonQuery(FIELD_NAME, pLons, pLats);
-
-                verifyHits = new VerifyHits() {
-                    @Override
-                    protected Boolean shouldMatch(double pointLat, double pointLon) {
-                      // morton encode & decode to compare apples to apples (that is, compare with same hash precision error
-                      // present in the index)
-                      long pointHash = GeoUtils.mortonHash(pointLon, pointLat);
-                      pointLon = GeoUtils.mortonUnhashLon(pointHash);
-                      pointLat = GeoUtils.mortonUnhashLat(pointHash);
-
-                      if (bboxQueryCanBeWrong(bbox, pointLat, pointLon)) {
-                        return null;
-                      } else {
-                        return rectContainsPointEnc(bbox, pointLat, pointLon);
-                      }
-                    }
-                  };
-              }
-
-              verifyHits.test(s, docIDToID, deleted, query, lats, lons);
-            }
-          }
-        };
-      thread.setName("T" + i);
-      thread.start();
-      threads.add(thread);
-    }
-
-    startingGun.countDown();
-    for(Thread thread : threads) {
-      thread.join();
-    }
-
-    IOUtils.close(r, dir);
-  }
-
-  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 {
-          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) {
-            System.out.println(Thread.currentThread().getName() + ": id=" + id +
-                               " docID=" + docID + " lat=" + lats[id] + " lon=" + lons[id] +
-                               " deleted?=" + deleted.contains(id) + " expected=" + expected + " but got " + hits.get(docID) +
-                               " query=" + query);
-            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);
-  }
-
-  private static boolean distanceContainsPt(double lonA, double latA, double lonB, double latB, final double radius,
-                                            final double maxRadius) {
-    final long hashedPtA = GeoUtils.mortonHash(lonA, latA);
-    lonA = GeoUtils.mortonUnhashLon(hashedPtA);
-    latA = GeoUtils.mortonUnhashLat(hashedPtA);
-    final long hashedPtB = GeoUtils.mortonHash(lonB, latB);
-    lonB = GeoUtils.mortonUnhashLon(hashedPtB);
-    latB = GeoUtils.mortonUnhashLat(hashedPtB);
-
-    if (maxRadius == 0) {
-      return (SloppyMath.haversin(latA, lonA, latB, lonB)*1000.0 <= radius);
-    }
-
-    return SloppyMath.haversin(latA, lonA, latB, lonB)*1000.0 >= radius
-        && SloppyMath.haversin(latA, lonA, latB, lonB)*1000.0 <= maxRadius;
-  }
-
-  private static boolean rectContainsPointEnc(GeoBoundingBox bbox, double pointLat, double pointLon) {
-    // We should never see a deleted doc here?
-    assert Double.isNaN(pointLat) == false;
-    return GeoUtils.bboxContains(pointLon, pointLat, bbox.minLon, bbox.minLat, bbox.maxLon, bbox.maxLat);
-  }
-
-  private static boolean radiusQueryCanBeWrong(double centerLat, double centerLon, double ptLon, double ptLat,
-                                               final double radius) {
-    final long hashedCntr = GeoUtils.mortonHash(centerLon, centerLat);
-    centerLon = GeoUtils.mortonUnhashLon(hashedCntr);
-    centerLat = GeoUtils.mortonUnhashLat(hashedCntr);
-    final long hashedPt = GeoUtils.mortonHash(ptLon, ptLat);
-    ptLon = GeoUtils.mortonUnhashLon(hashedPt);
-    ptLat = GeoUtils.mortonUnhashLat(hashedPt);
-
-    double ptDistance = SloppyMath.haversin(centerLat, centerLon, ptLat, ptLon)*1000.0;
-    double delta = StrictMath.abs(ptDistance - radius);
-
-    // if its within the distance error then it can be wrong
-    return delta < DISTANCE_ERR;
-  }
-
-  private static boolean bboxQueryCanBeWrong(GeoBoundingBox bbox, double lat, double lon) {
-    // we can tolerate variance at the GeoUtils.TOLERANCE decimal place
-    final int tLon = (int)(lon/(GeoUtils.TOLERANCE-1));
-    final int tLat = (int)(lat/(GeoUtils.TOLERANCE-1));
-    final int tMinLon = (int)(bbox.minLon/(GeoUtils.TOLERANCE-1));
-    final int tMinLat = (int)(bbox.minLat/(GeoUtils.TOLERANCE-1));
-    final int tMaxLon = (int)(bbox.maxLon/(GeoUtils.TOLERANCE-1));
-    final int tMaxLat = (int)(bbox.maxLat/(GeoUtils.TOLERANCE-1));
-
-    return ((tMinLon - tLon) == 0 || (tMinLat - tLat) == 0
-         || (tMaxLon - tLon) == 0 || (tMaxLat - tLat) == 0);
-  }
-
-  private static GeoBoundingBox randomBBox() {
-    double lat0 = TestGeoUtils.randomLat();
-    double lat1 = TestGeoUtils.randomLat();
-    double lon0 = TestGeoUtils.randomLon();
-    double lon1 = TestGeoUtils.randomLon();
-
-    if (lat1 < lat0) {
-      double x = lat0;
-      lat0 = lat1;
-      lat1 = x;
-    }
-
-    if (lon1 < lon0) {
-      double x = lon0;
-      lon0 = lon1;
-      lon1 = x;
-    }
-
-    return new GeoBoundingBox(lon0, lon1, lat0, lat1);
-  }
 }

Modified: lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/util/TestGeoUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/util/TestGeoUtils.java?rev=1709926&r1=1709925&r2=1709926&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/util/TestGeoUtils.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/util/TestGeoUtils.java Wed Oct 21 21:59:08 2015
@@ -17,12 +17,17 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
+import java.io.PrintWriter;
+import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.junit.BeforeClass;
-import org.junit.Test;
+
+import com.carrotsearch.randomizedtesting.generators.RandomInts;
 
 /**
  * Tests class for methods in GeoUtils
@@ -31,6 +36,9 @@ import org.junit.Test;
  */
 public class TestGeoUtils extends LuceneTestCase {
 
+  private static final double LON_SCALE = (0x1L<<GeoUtils.BITS)/360.0D;
+  private static final double LAT_SCALE = (0x1L<<GeoUtils.BITS)/180.0D;
+
   // Global bounding box we will "cover" in the random test; we have to make this "smallish" else the queries take very long:
   private static double originLat;
   private static double originLon;
@@ -54,7 +62,6 @@ public class TestGeoUtils extends Lucene
     }
   }
 
-  @Test
   public void testGeoHash() {
     int numPoints = atLeast(100);
     String randomGeoHashString;
@@ -63,8 +70,8 @@ public class TestGeoUtils extends Lucene
     int randomLevel;
     for (int i = 0; i < numPoints; ++i) {
       // random point
-      double lat = randomLatFullRange();
-      double lon = randomLonFullRange();
+      double lat = randomLat(false);
+      double lon = randomLon(false);
 
       // compute geohash straight from lat/lon and from morton encoded value to ensure they're the same
       randomGeoHashString = GeoHashUtils.stringEncode(lon, lat, randomLevel = random().nextInt(12 - 1) + 1);
@@ -90,7 +97,6 @@ public class TestGeoUtils extends Lucene
    * Pass condition: lat=42.6, lng=-5.6 should be encoded as "ezs42e44yx96",
    * lat=57.64911 lng=10.40744 should be encoded as "u4pruydqqvj8"
    */
-  @Test
   public void testEncode() {
     String hash = GeoHashUtils.stringEncode(-5.6, 42.6, 12);
     assertEquals("ezs42e44yx96", hash);
@@ -103,7 +109,6 @@ public class TestGeoUtils extends Lucene
    * Pass condition: lat=52.3738007, lng=4.8909347 should be encoded and then
    * decoded within 0.00001 of the original value
    */
-  @Test
   public void testDecodePreciseLongitudeLatitude() {
     final String geohash = GeoHashUtils.stringEncode(4.8909347, 52.3738007);
     final long hash = GeoHashUtils.mortonEncode(geohash);
@@ -116,7 +121,6 @@ public class TestGeoUtils extends Lucene
    * Pass condition: lat=84.6, lng=10.5 should be encoded and then decoded
    * within 0.00001 of the original value
    */
-  @Test
   public void testDecodeImpreciseLongitudeLatitude() {
     final String geohash = GeoHashUtils.stringEncode(10.5, 84.6);
 
@@ -126,7 +130,6 @@ public class TestGeoUtils extends Lucene
     assertEquals(10.5, GeoUtils.mortonUnhashLon(hash), 0.00001D);
   }
 
-  @Test
   public void testDecodeEncode() {
     final String geoHash = "u173zq37x014";
     assertEquals(geoHash, GeoHashUtils.stringEncode(4.8909347, 52.3738007));
@@ -139,7 +142,6 @@ public class TestGeoUtils extends Lucene
     assertEquals(geoHash, GeoHashUtils.stringEncode(lon, lat));
   }
 
-  @Test
   public void testNeighbors() {
     String geohash = "gcpv";
     List<String> expectedNeighbors = new ArrayList<>();
@@ -201,19 +203,332 @@ public class TestGeoUtils extends Lucene
     assertEquals(expectedNeighbors, neighbors);
   }
 
-  public static double randomLatFullRange() {
-    return (180d * random().nextDouble()) - 90d;
+  public void testClosestPointOnBBox() {
+    double[] result = new double[2];
+    GeoDistanceUtils.closestPointOnBBox(20, 30, 40, 50, 70, 70, result);
+    assertEquals(40.0, result[0], 0.0);
+    assertEquals(50.0, result[1], 0.0);
+
+    GeoDistanceUtils.closestPointOnBBox(-20, -20, 0, 0, 70, 70, result);
+    assertEquals(0.0, result[0], 0.0);
+    assertEquals(0.0, result[1], 0.0);
+  }
+
+  private static class Cell {
+    static int nextCellID;
+
+    final Cell parent;
+    final int cellID;
+    final double minLon, maxLon;
+    final double minLat, maxLat;
+    final int splitCount;
+
+    public Cell(Cell parent,
+                double minLon, double minLat,
+                double maxLon, double maxLat,
+                int splitCount) {
+      assert maxLon >= minLon;
+      assert maxLat >= minLat;
+      this.parent = parent;
+      this.minLon = minLon;
+      this.minLat = minLat;
+      this.maxLon = maxLon;
+      this.maxLat = maxLat;
+      this.cellID = nextCellID++;
+      this.splitCount = splitCount;
+    }
+
+    /** Returns true if the quantized point lies within this cell, inclusive on all bounds. */
+    public boolean contains(double lon, double lat) {
+      return lon >= minLon && lon <= maxLon && lat >= minLat && lat <= maxLat;
+    }
+
+    @Override
+    public String toString() {
+      return "cell=" + cellID + (parent == null ? "" : " parentCellID=" + parent.cellID) + " lon: " + minLon + " TO " + maxLon + ", lat: " + minLat + " TO " + maxLat + ", splits: " + splitCount;
+    }
+  }
+
+  public long scaleLon(final double val) {
+    return (long) ((val-GeoUtils.MIN_LON_INCL) * LON_SCALE);
   }
 
-  public static double randomLonFullRange() {
-    return (360d * random().nextDouble()) - 180d;
+  public long scaleLat(final double val) {
+    return (long) ((val-GeoUtils.MIN_LAT_INCL) * LAT_SCALE);
   }
 
-  public static double randomLat() {
-    return GeoUtils.normalizeLat(originLat + latRange * (random().nextDouble() - 0.5));
+  public double unscaleLon(final long val) {
+    return (val / LON_SCALE) + GeoUtils.MIN_LON_INCL;
   }
 
-  public static double randomLon() {
-    return GeoUtils.normalizeLon(originLon + lonRange * (random().nextDouble() - 0.5));
+  public double unscaleLat(final long val) {
+    return (val / LAT_SCALE) + GeoUtils.MIN_LAT_INCL;
+  }
+
+  public double randomLat(boolean small) {
+    double result;
+    if (small) {
+      result = GeoUtils.normalizeLat(originLat + latRange * (random().nextDouble() - 0.5));
+    } else {
+      result = -90 + 180.0 * random().nextDouble();
+    }
+    return unscaleLat(scaleLat(result));
+  }
+
+  public double randomLon(boolean small) {
+    double result;
+    if (small) {
+      result = GeoUtils.normalizeLon(originLon + lonRange * (random().nextDouble() - 0.5));
+    } else {
+      result = -180 + 360.0 * random().nextDouble();
+    }
+    return unscaleLon(scaleLon(result));
+  }
+
+  private void findMatches(Set<Integer> hits, PrintWriter log, Cell root,
+                           double centerLon, double centerLat, double radiusMeters,
+                           double[] docLons, double[] docLats) {
+
+    if (VERBOSE) {
+      log.println("  root cell: " + root);
+    }
+
+    List<Cell> queue = new ArrayList<>();
+    queue.add(root);
+
+    int recurseDepth = RandomInts.randomIntBetween(random(), 5, 15);
+
+    while (queue.size() > 0) {
+      Cell cell = queue.get(queue.size()-1);
+      queue.remove(queue.size()-1);
+      if (VERBOSE) {
+        log.println("  cycle: " + cell + " queue.size()=" + queue.size());
+      }
+
+      if (random().nextInt(10) == 7 || cell.splitCount > recurseDepth) {
+        if (VERBOSE) {
+          log.println("    leaf");
+        }
+        // Leaf cell: brute force check all docs that fall within this cell:
+        for(int docID=0;docID<docLons.length;docID++) {
+          if (cell.contains(docLons[docID], docLats[docID])) {
+            double distanceMeters = SloppyMath.haversin(centerLat, centerLon, docLats[docID], docLons[docID]) * 1000.0;
+            if (distanceMeters <= radiusMeters) {
+              if (VERBOSE) {
+                log.println("    check doc=" + docID + ": match!");
+              }
+              hits.add(docID);
+            } else {
+              if (VERBOSE) {
+                log.println("    check doc=" + docID + ": no match");
+              }
+            }
+          }
+        }
+      } else {
+
+        if (GeoUtils.rectWithinCircle(cell.minLon, cell.minLat, cell.maxLon, cell.maxLat, centerLon, centerLat, radiusMeters)) {
+          // Query circle fully contains this cell, just addAll:
+          if (VERBOSE) {
+            log.println("    circle fully contains cell: now addAll");
+          }
+          for(int docID=0;docID<docLons.length;docID++) {
+            if (cell.contains(docLons[docID], docLats[docID])) {
+              if (VERBOSE) {
+                log.println("    addAll doc=" + docID);
+              }
+              hits.add(docID);
+            }
+          }
+          continue;
+        } else if (GeoUtils.rectWithin(root.minLon, root.minLat, root.maxLon, root.maxLat,
+                                       cell.minLon, cell.minLat, cell.maxLon, cell.maxLat)) {
+          // Fall through below to "recurse"
+          if (VERBOSE) {
+            log.println("    cell fully contains circle: keep splitting");
+          }
+        } else if (GeoUtils.rectCrossesCircle(cell.minLon, cell.minLat, cell.maxLon, cell.maxLat,
+                                              centerLon, centerLat, radiusMeters)) {
+          // Fall through below to "recurse"
+          if (VERBOSE) {
+            log.println("    cell overlaps circle: keep splitting");
+          }
+        } else {
+          if (VERBOSE) {
+            log.println("    no overlap: drop this cell");
+            for(int docID=0;docID<docLons.length;docID++) {
+              if (cell.contains(docLons[docID], docLats[docID])) {
+                if (VERBOSE) {
+                  log.println("    skip doc=" + docID);
+                }
+              }
+            }
+          }
+          continue;
+        }
+          
+        // Randomly split:
+        if (random().nextBoolean()) {
+
+          // Split on lon:
+          double splitValue = cell.minLon + (cell.maxLon - cell.minLon) * random().nextDouble();
+          if (VERBOSE) {
+            log.println("    now split on lon=" + splitValue);
+          }
+          Cell cell1 = new Cell(cell,
+                                cell.minLon, cell.minLat,
+                                splitValue, cell.maxLat,
+                                cell.splitCount+1);
+          Cell cell2 = new Cell(cell,
+                                splitValue, cell.minLat,
+                                cell.maxLon, cell.maxLat,
+                                cell.splitCount+1);
+          if (VERBOSE) {
+            log.println("    split cell1: " + cell1);
+            log.println("    split cell2: " + cell2);
+          }
+          queue.add(cell1);
+          queue.add(cell2);
+        } else {
+
+          // Split on lat:
+          double splitValue = cell.minLat + (cell.maxLat - cell.minLat) * random().nextDouble();
+          if (VERBOSE) {
+            log.println("    now split on lat=" + splitValue);
+          }
+          Cell cell1 = new Cell(cell,
+                                cell.minLon, cell.minLat,
+                                cell.maxLon, splitValue,
+                                cell.splitCount+1);
+          Cell cell2 = new Cell(cell,
+                                cell.minLon, splitValue,
+                                cell.maxLon, cell.maxLat,
+                                cell.splitCount+1);
+          if (VERBOSE) {
+            log.println("    split cells:\n      " + cell1 + "\n      " + cell2);
+          }
+          queue.add(cell1);
+          queue.add(cell2);
+        }
+      }
+    }
+  }
+
+  /** Tests consistency of GeoUtils.rectWithinCircle, .rectCrossesCircle, .rectWithin and SloppyMath.haversine distance check */
+  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-6846")
+  public void testGeoRelations() throws Exception {
+
+    int numDocs = atLeast(1000);
+    
+    // boolean useSmallRanges = random().nextBoolean();
+
+    // TODO: the GeoUtils APIs have bugs if you use large distances:
+    boolean useSmallRanges = true;
+
+    if (VERBOSE) {
+      System.out.println("TEST: " + numDocs + " docs useSmallRanges=" + useSmallRanges);
+    }
+
+    double[] docLons = new double[numDocs];
+    double[] docLats = new double[numDocs];
+    for(int docID=0;docID<numDocs;docID++) {
+      docLons[docID] = randomLon(useSmallRanges);
+      docLats[docID] = randomLat(useSmallRanges);
+      if (VERBOSE) {
+        System.out.println("  doc=" + docID + ": lon=" + docLons[docID] + " lat=" + docLats[docID]);
+      }
+    }
+
+    int iters = atLeast(10);
+
+    iters = atLeast(50);
+    
+    for(int iter=0;iter<iters;iter++) {
+
+      Cell.nextCellID = 0;
+
+      double centerLon = randomLon(useSmallRanges);
+      double centerLat = randomLat(useSmallRanges);
+
+      // So the circle covers at most 50% of the earth's surface:
+
+      double radiusMeters;
+
+      // TODO: GeoUtils APIs are still buggy for large distances:
+      if (true || useSmallRanges) {
+        // Approx 3 degrees lon at the equator:
+        radiusMeters = random().nextDouble() * 333000;
+      } else {
+        radiusMeters = random().nextDouble() * GeoProjectionUtils.SEMIMAJOR_AXIS * Math.PI / 2.0;
+      }
+
+      StringWriter sw = new StringWriter();
+      PrintWriter log = new PrintWriter(sw, true);
+
+      if (VERBOSE) {
+        log.println("\nTEST: iter=" + iter + " radiusMeters=" + radiusMeters + " centerLon=" + centerLon + " centerLat=" + centerLat);
+      }
+
+      GeoRect bbox = GeoUtils.circleToBBox(centerLon, centerLat, radiusMeters);
+      
+      Set<Integer> hits = new HashSet<>();
+
+      if (bbox.maxLon < bbox.minLon) {
+        // Crosses dateline
+        log.println("  circle crosses dateline; first right query");
+        findMatches(hits, log,
+                    new Cell(null,
+                             -180, bbox.minLat,
+                             bbox.maxLon, bbox.maxLat,
+                             0),
+                    centerLon, centerLat, radiusMeters,
+                    docLons, docLats);
+        log.println("  circle crosses dateline; now left query");
+        findMatches(hits, log,
+                    new Cell(null,
+                             bbox.minLon, bbox.minLat,
+                             180, bbox.maxLat,
+                             0),
+                    centerLon, centerLat, radiusMeters,
+                    docLons, docLats);
+      } else {
+        // Start with the root cell that fully contains the shape:
+        findMatches(hits, log,
+                    new Cell(null,
+                             bbox.minLon, bbox.minLat,
+                             bbox.maxLon, bbox.maxLat,
+                             0),
+                    centerLon, centerLat, radiusMeters,
+                    docLons, docLats);
+      }
+
+      if (VERBOSE) {
+        log.println("  " + hits.size() + " hits");
+      }
+
+      int failCount = 0;
+
+      // Done matching, now verify:
+      for(int docID=0;docID<numDocs;docID++) {
+        double distanceMeters = SloppyMath.haversin(centerLat, centerLon, docLats[docID], docLons[docID]) * 1000.0;
+        boolean expected = distanceMeters <= radiusMeters;
+
+        boolean actual = hits.contains(docID);
+        if (actual != expected) {
+          if (actual) {
+            log.println("doc=" + docID + " matched but should not");
+          } else {
+            log.println("doc=" + docID + " did not match but should");
+          }
+          log.println("  lon=" + docLons[docID] + " lat=" + docLats[docID] + " distanceMeters=" + distanceMeters + " vs radiusMeters=" + radiusMeters);
+          failCount++;
+        }
+      }
+
+      if (failCount != 0) {
+        System.out.print(sw.toString());
+        fail(failCount + " incorrect hits (see above)");
+      }
+    }
   }
 }