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 2016/02/05 18:02:59 UTC

[55/87] [abbrv] lucene-solr git commit: LUCENE-6997: refactors lucene-spatial module to a new lucene-spatial-extras module, and refactors sandbox GeoPointField and queries to lucene-spatial module

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial/src/test/org/apache/lucene/spatial/search/TestGeoPointQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/search/TestGeoPointQuery.java b/lucene/spatial/src/test/org/apache/lucene/spatial/search/TestGeoPointQuery.java
new file mode 100644
index 0000000..68075a6
--- /dev/null
+++ b/lucene/spatial/src/test/org/apache/lucene/spatial/search/TestGeoPointQuery.java
@@ -0,0 +1,399 @@
+<<<<<<< HEAD:lucene/sandbox/src/test/org/apache/lucene/search/TestGeoPointQuery.java
+=======
+package org.apache.lucene.spatial.search;
+
+>>>>>>> LUCENE-6997: refactors lucene-spatial module to a new lucene-spatial-extras module, and refactors sandbox GeoPointField and queries to lucene-spatial module:lucene/spatial/src/test/org/apache/lucene/spatial/search/TestGeoPointQuery.java
+/*
+ * 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.
+ */
+package org.apache.lucene.search;
+
+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.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.spatial.document.GeoPointField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.spatial.util.GeoBoundingBox;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.spatial.util.BaseGeoPointTestCase;
+import org.apache.lucene.spatial.util.GeoRelationUtils;
+import org.apache.lucene.spatial.util.GeoUtils;
+import org.apache.lucene.util.SloppyMath;
+import org.apache.lucene.util.TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import static org.apache.lucene.spatial.util.GeoDistanceUtils.DISTANCE_PCT_ERR;
+import static org.apache.lucene.spatial.util.GeoUtils.MAX_LAT_INCL;
+import static org.apache.lucene.spatial.util.GeoUtils.MAX_LON_INCL;
+import static org.apache.lucene.spatial.util.GeoUtils.MIN_LAT_INCL;
+import static org.apache.lucene.spatial.util.GeoUtils.MIN_LON_INCL;
+
+
+/**
+ * Unit testing for basic GeoPoint query logic
+ *
+ * @lucene.experimental
+ */
+
+public class TestGeoPointQuery extends BaseGeoPointTestCase {
+
+  private static Directory directory = null;
+  private static IndexReader reader = null;
+  private static IndexSearcher searcher = null;
+
+  @Override
+  protected boolean forceSmall() {
+    return false;
+  }
+
+  @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 newRectQuery(String field, GeoBoundingBox 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);
+  }
+
+  @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 {
+    directory = newDirectory();
+
+    RandomIndexWriter writer = new RandomIndexWriter(random(), directory,
+            newIndexWriterConfig(new MockAnalyzer(random()))
+                    .setMaxBufferedDocs(TestUtil.nextInt(random(), 100, 1000))
+                    .setMergePolicy(newLogMergePolicy()));
+
+    // create some simple geo points
+    final FieldType storedPoint = new FieldType(GeoPointField.TYPE_STORED);
+    // this is a simple systematic test
+    GeoPointField[] pts = new GeoPointField[] {
+         new GeoPointField(FIELD_NAME, -96.774, 32.763420, storedPoint),
+         new GeoPointField(FIELD_NAME, -96.7759895324707, 32.7559529921407, storedPoint),
+         new GeoPointField(FIELD_NAME, -96.77701950073242, 32.77866942010977, storedPoint),
+         new GeoPointField(FIELD_NAME, -96.7706036567688, 32.7756745755423, storedPoint),
+         new GeoPointField(FIELD_NAME, -139.73458170890808, 27.703618681345585, storedPoint),
+         new GeoPointField(FIELD_NAME, -96.4538113027811, 32.94823588839368, storedPoint),
+         new GeoPointField(FIELD_NAME, -96.65084838867188, 33.06047141970814, storedPoint),
+         new GeoPointField(FIELD_NAME, -96.7772, 32.778650, storedPoint),
+         new GeoPointField(FIELD_NAME, -177.23537676036358, -88.56029371730983, storedPoint),
+         new GeoPointField(FIELD_NAME, -26.779373834241003, 33.541429799076354, storedPoint),
+         new GeoPointField(FIELD_NAME, -77.35379276106497, 26.774024500421728, storedPoint),
+         new GeoPointField(FIELD_NAME, -14.796283808944777, -90.0, storedPoint),
+         new GeoPointField(FIELD_NAME, -178.8538113027811, 32.94823588839368, storedPoint),
+         new GeoPointField(FIELD_NAME, 178.8538113027811, 32.94823588839368, storedPoint),
+         new GeoPointField(FIELD_NAME, -73.998776, 40.720611, storedPoint),
+         new GeoPointField(FIELD_NAME, -179.5, -44.5, storedPoint)};
+
+    for (GeoPointField p : pts) {
+        Document doc = new Document();
+        doc.add(p);
+        writer.addDocument(doc);
+    }
+
+    // add explicit multi-valued docs
+    for (int i=0; i<pts.length; i+=2) {
+      Document doc = new Document();
+      doc.add(pts[i]);
+      doc.add(pts[i+1]);
+      writer.addDocument(doc);
+    }
+
+    // index random string documents
+    for (int i=0; i<random().nextInt(10); ++i) {
+      Document doc = new Document();
+      doc.add(new StringField("string", Integer.toString(i), Field.Store.NO));
+      writer.addDocument(doc);
+    }
+
+    reader = writer.getReader();
+    searcher = newSearcher(reader);
+    writer.close();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    searcher = null;
+    reader.close();
+    reader = null;
+    directory.close();
+    directory = null;
+  }
+
+  private TopDocs bboxQuery(double minLon, double minLat, double maxLon, double maxLat, int limit) throws Exception {
+    GeoPointInBBoxQuery q = new GeoPointInBBoxQuery(FIELD_NAME, minLon, minLat, maxLon, maxLat);
+    return searcher.search(q, limit);
+  }
+
+  private TopDocs polygonQuery(double[] lon, double[] lat, int limit) throws Exception {
+    GeoPointInPolygonQuery q = new GeoPointInPolygonQuery(FIELD_NAME, lon, lat);
+    return searcher.search(q, limit);
+  }
+
+  private TopDocs geoDistanceQuery(double lon, double lat, double radius, int limit) throws Exception {
+    GeoPointDistanceQuery q = new GeoPointDistanceQuery(FIELD_NAME, lon, lat, radius);
+    return searcher.search(q, limit);
+  }
+
+  @Override
+  protected Boolean rectContainsPoint(GeoBoundingBox 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 GeoRelationUtils.pointInRectPrecise(pointLon, pointLat, rect.minLon, rect.minLat, rect.maxLon, rect.maxLat);
+    } else {
+      // Rect crosses dateline:
+      return GeoRelationUtils.pointInRectPrecise(pointLon, pointLat, -180.0, rect.minLat, rect.maxLon, rect.maxLat)
+          || GeoRelationUtils.pointInRectPrecise(pointLon, pointLat, rect.minLon, rect.minLat, 180.0, rect.maxLat);
+    }
+  }
+
+  @Override
+  protected Boolean polyRectContainsPoint(GeoBoundingBox 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(GeoRelationUtils.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);
+  }
+
+  public void testBBoxQuery() throws Exception {
+    TopDocs td = bboxQuery(-96.7772, 32.778650, -96.77690000, 32.778950, 5);
+    assertEquals("GeoBoundingBoxQuery failed", 4, td.totalHits);
+  }
+
+  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},
+        new double[]{33.073130, 32.9942669, 32.938386, 33.0374494,
+            33.1369762, 33.1162747, 33.073130, 33.073130}, 5);
+    assertEquals("GeoPolygonQuery failed", 2, td.totalHits);
+  }
+
+  public void testPacManPolyQuery() throws Exception {
+    // pacman
+    double[] px = {0, 10, 10, 0, -8, -10, -8, 0, 10, 10, 0};
+    double[] py = {0, 5, 9, 10, 9, 0, -9, -10, -9, -5, 0};
+
+    // shape bbox
+    double xMinA = -10;
+    double xMaxA = 10;
+    double yMinA = -10;
+    double yMaxA = 10;
+
+    // candidate crosses cell
+    double xMin = 2;//-5;
+    double xMax = 11;//0.000001;
+    double yMin = -1;//0;
+    double yMax = 1;//5;
+
+    // test cell crossing poly
+    assertTrue(GeoRelationUtils.rectCrossesPolyApprox(xMin, yMin, xMax, yMax, px, py, xMinA, yMinA, xMaxA, yMaxA));
+    assertFalse(GeoRelationUtils.rectCrossesPolyApprox(-5, 0,  0.000001, 5, px, py, xMin, yMin, xMax, yMax));
+    assertTrue(GeoRelationUtils.rectWithinPolyApprox(-5, 0, -2, 5, px, py, xMin, yMin, xMax, yMax));
+  }
+
+  public void testBBoxCrossDateline() throws Exception {
+    TopDocs td = bboxQuery(179.0, -45.0, -179.0, -44.0, 20);
+    assertEquals("BBoxCrossDateline query failed", 2, td.totalHits);
+  }
+
+  public void testWholeMap() throws Exception {
+    TopDocs td = bboxQuery(MIN_LON_INCL, MIN_LAT_INCL, MAX_LON_INCL, MAX_LAT_INCL, 25);
+    assertEquals("testWholeMap failed", 24, td.totalHits);
+    td = polygonQuery(new double[] {GeoUtils.MIN_LON_INCL, GeoUtils.MIN_LON_INCL, GeoUtils.MAX_LON_INCL, GeoUtils.MAX_LON_INCL, GeoUtils.MIN_LON_INCL},
+        new double[] {GeoUtils.MIN_LAT_INCL, GeoUtils.MAX_LAT_INCL, GeoUtils.MAX_LAT_INCL, GeoUtils.MIN_LAT_INCL, GeoUtils.MIN_LAT_INCL}, 20);
+    assertEquals("testWholeMap failed", 24, td.totalHits);
+  }
+
+  public void smallTest() throws Exception {
+    TopDocs td = geoDistanceQuery(-73.998776, 40.720611, 1, 20);
+    assertEquals("smallTest failed", 2, td.totalHits);
+  }
+
+  public void testInvalidBBox() throws Exception {
+    try {
+      bboxQuery(179.0, -92.0, 181.0, -91.0, 20);
+    } catch(Exception e) {
+      return;
+    }
+    throw new Exception("GeoBoundingBox should not accept invalid lat/lon");
+  }
+
+  public void testGeoDistanceQuery() throws Exception {
+    TopDocs td = geoDistanceQuery(-96.4538113027811, 32.94823588839368, 6000, 20);
+    assertEquals("GeoDistanceQuery failed", 2, td.totalHits);
+  }
+
+  /** see https://issues.apache.org/jira/browse/LUCENE-6905 */
+  public void testNonEmptyTermsEnum() throws Exception {
+    TopDocs td = geoDistanceQuery(-177.23537676036358, -88.56029371730983, 7757.999232959935, 20);
+    assertEquals("GeoDistanceQuery failed", 2, td.totalHits);
+  }
+
+  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
+   */
+  public void testGeoDistanceQueryHuge() throws Exception {
+    TopDocs td = geoDistanceQuery(-96.4538113027811, 32.94823588839368, 6000000, 20);
+    assertEquals("GeoDistanceQuery failed", 16, td.totalHits);
+  }
+
+  public void testGeoDistanceQueryCrossDateline() throws Exception {
+    TopDocs td = geoDistanceQuery(-179.9538113027811, 32.94823588839368, 120000, 20);
+    assertEquals("GeoDistanceQuery failed", 3, td.totalHits);
+  }
+
+  public void testInvalidGeoDistanceQuery() throws Exception {
+    try {
+      geoDistanceQuery(181.0, 92.0, 120000, 20);
+    } catch (Exception e) {
+      return;
+    }
+    throw new Exception("GeoDistanceQuery should not accept invalid lat/lon as origin");
+  }
+
+  public void testMaxDistanceRangeQuery() throws Exception {
+    TopDocs td = geoDistanceRangeQuery(0.0, 0.0, 10, 20000000, 20);
+    assertEquals("GeoDistanceRangeQuery failed", 24, td.totalHits);
+  }
+
+  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);
+  }
+
+  public void testEncodeDecode() throws Exception {
+    int iters = atLeast(10000);
+    boolean small = random().nextBoolean();
+    for(int iter=0;iter<iters;iter++) {
+      double lat = randomLat(small);
+      double lon = randomLon(small);
+
+      long enc = GeoUtils.mortonHash(lon, lat);
+      double latEnc = GeoUtils.mortonUnhashLat(enc);
+      double lonEnc = GeoUtils.mortonUnhashLon(enc);
+
+      assertEquals("lat=" + lat + " latEnc=" + latEnc + " diff=" + (lat - latEnc), lat, latEnc, GeoUtils.TOLERANCE);
+      assertEquals("lon=" + lon + " lonEnc=" + lonEnc + " diff=" + (lon - lonEnc), lon, lonEnc, GeoUtils.TOLERANCE);
+    }
+  }
+
+  public void testScaleUnscaleIsStable() throws Exception {
+    int iters = atLeast(1000);
+    boolean small = random().nextBoolean();
+    for(int iter=0;iter<iters;iter++) {
+      double lat = randomLat(small);
+      double lon = randomLon(small);
+
+      long enc = GeoUtils.mortonHash(lon, lat);
+      double latEnc = GeoUtils.mortonUnhashLat(enc);
+      double lonEnc = GeoUtils.mortonUnhashLon(enc);
+
+      long enc2 = GeoUtils.mortonHash(lon, lat);
+      double latEnc2 = GeoUtils.mortonUnhashLat(enc2);
+      double lonEnc2 = GeoUtils.mortonUnhashLon(enc2);
+      assertEquals(latEnc, latEnc2, 0.0);
+      assertEquals(lonEnc, lonEnc2, 0.0);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial/src/test/org/apache/lucene/spatial/serialized/SerializedStrategyTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/serialized/SerializedStrategyTest.java b/lucene/spatial/src/test/org/apache/lucene/spatial/serialized/SerializedStrategyTest.java
deleted file mode 100644
index bed8339..0000000
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/serialized/SerializedStrategyTest.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.spatial.serialized;
-
-import java.io.IOException;
-
-import com.spatial4j.core.context.SpatialContext;
-import org.apache.lucene.spatial.SpatialMatchConcern;
-import org.apache.lucene.spatial.StrategyTestCase;
-import org.junit.Before;
-import org.junit.Test;
-
-public class SerializedStrategyTest extends StrategyTestCase {
-
-  @Before
-  @Override
-  public void setUp() throws Exception {
-    super.setUp();
-    this.ctx = SpatialContext.GEO;
-    this.strategy = new SerializedDVStrategy(ctx, "serialized");
-  }
-
-  @Override
-  protected boolean needsDocValues() {
-    return (strategy instanceof SerializedDVStrategy);
-  }
-
-  @Test
-  public void testBasicOperaions() throws IOException {
-    getAddAndVerifyIndexedDocuments(DATA_SIMPLE_BBOX);
-
-    executeQueries(SpatialMatchConcern.EXACT, QTEST_Simple_Queries_BBox);
-  }
-
-  @Test
-  public void testStatesBBox() throws IOException {
-    getAddAndVerifyIndexedDocuments(DATA_STATES_BBOX);
-
-    executeQueries(SpatialMatchConcern.FILTER, QTEST_States_IsWithin_BBox);
-    executeQueries(SpatialMatchConcern.FILTER, QTEST_States_Intersects_BBox);
-  }
-
-  @Test
-  public void testCitiesIntersectsBBox() throws IOException {
-    getAddAndVerifyIndexedDocuments(DATA_WORLD_CITIES_POINTS);
-
-    executeQueries(SpatialMatchConcern.FILTER, QTEST_Cities_Intersects_BBox);
-  }
-
-  //sorting is tested in DistanceStrategyTest
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dRptTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dRptTest.java b/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dRptTest.java
deleted file mode 100644
index 7c98d0244..0000000
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dRptTest.java
+++ /dev/null
@@ -1,227 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.spatial.spatial4j;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import com.carrotsearch.randomizedtesting.annotations.Repeat;
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import org.apache.lucene.spatial.composite.CompositeSpatialStrategy;
-import org.apache.lucene.spatial.prefix.RandomSpatialOpStrategyTestCase;
-import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;
-import org.apache.lucene.spatial.prefix.tree.GeohashPrefixTree;
-import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
-import org.apache.lucene.spatial.query.SpatialOperation;
-import org.apache.lucene.spatial.serialized.SerializedDVStrategy;
-import org.apache.lucene.geo3d.GeoBBoxFactory;
-import org.apache.lucene.geo3d.GeoStandardCircle;
-import org.apache.lucene.geo3d.GeoPath;
-import org.apache.lucene.geo3d.GeoPoint;
-import org.apache.lucene.geo3d.GeoPolygonFactory;
-import org.apache.lucene.geo3d.GeoShape;
-import org.apache.lucene.geo3d.PlanetModel;
-import org.junit.Test;
-
-import static com.spatial4j.core.distance.DistanceUtils.DEGREES_TO_RADIANS;
-
-public class Geo3dRptTest extends RandomSpatialOpStrategyTestCase {
-
-  private SpatialPrefixTree grid;
-  private RecursivePrefixTreeStrategy rptStrategy;
-  {
-    this.ctx = SpatialContext.GEO;
-  }
-
-  private void setupGeohashGrid() {
-    this.grid = new GeohashPrefixTree(ctx, 2);//A fairly shallow grid
-    this.rptStrategy = newRPT();
-  }
-
-  protected RecursivePrefixTreeStrategy newRPT() {
-    final RecursivePrefixTreeStrategy rpt = new RecursivePrefixTreeStrategy(this.grid,
-        getClass().getSimpleName() + "_rpt");
-    rpt.setDistErrPct(0.10);//not too many cells
-    return rpt;
-  }
-
-  @Override
-  protected boolean needsDocValues() {
-    return true;//due to SerializedDVStrategy
-  }
-
-  private void setupStrategy() {
-    //setup
-    setupGeohashGrid();
-
-    SerializedDVStrategy serializedDVStrategy = new SerializedDVStrategy(ctx, getClass().getSimpleName() + "_sdv");
-    this.strategy = new CompositeSpatialStrategy("composite_" + getClass().getSimpleName(),
-        rptStrategy, serializedDVStrategy);
-  }
-
-  @Test
-  public void testFailure1() throws IOException {
-    setupStrategy();
-    final List<GeoPoint> points = new ArrayList<GeoPoint>();
-    points.add(new GeoPoint(PlanetModel.SPHERE, 18 * DEGREES_TO_RADIANS, -27 * DEGREES_TO_RADIANS));
-    points.add(new GeoPoint(PlanetModel.SPHERE, -57 * DEGREES_TO_RADIANS, 146 * DEGREES_TO_RADIANS));
-    points.add(new GeoPoint(PlanetModel.SPHERE, 14 * DEGREES_TO_RADIANS, -180 * DEGREES_TO_RADIANS));
-    points.add(new GeoPoint(PlanetModel.SPHERE, -15 * DEGREES_TO_RADIANS, 153 * DEGREES_TO_RADIANS));
-    
-    final Shape triangle = new Geo3dShape(GeoPolygonFactory.makeGeoPolygon(PlanetModel.SPHERE, points,0),ctx);
-    final Rectangle rect = ctx.makeRectangle(-49, -45, 73, 86);
-    testOperation(rect,SpatialOperation.Intersects,triangle, false);
-  }
-
-  @Test
-  public void testFailureLucene6535() throws IOException {
-    setupStrategy();
-
-    final List<GeoPoint> points = new ArrayList<>();
-    points.add(new GeoPoint(PlanetModel.SPHERE, 18 * DEGREES_TO_RADIANS, -27 * DEGREES_TO_RADIANS));
-    points.add(new GeoPoint(PlanetModel.SPHERE, -57 * DEGREES_TO_RADIANS, 146 * DEGREES_TO_RADIANS));
-    points.add(new GeoPoint(PlanetModel.SPHERE, 14 * DEGREES_TO_RADIANS, -180 * DEGREES_TO_RADIANS));
-    points.add(new GeoPoint(PlanetModel.SPHERE, -15 * DEGREES_TO_RADIANS, 153 * DEGREES_TO_RADIANS));
-    final GeoPath path = new GeoPath(PlanetModel.SPHERE, 29 * DEGREES_TO_RADIANS);
-    path.addPoint(55.0 * DEGREES_TO_RADIANS, -26.0 * DEGREES_TO_RADIANS);
-    path.addPoint(-90.0 * DEGREES_TO_RADIANS, 0.0);
-    path.addPoint(54.0 * DEGREES_TO_RADIANS, 165.0 * DEGREES_TO_RADIANS);
-    path.addPoint(-90.0 * DEGREES_TO_RADIANS, 0.0);
-    path.done();
-    final Shape shape = new Geo3dShape(path,ctx);
-    final Rectangle rect = ctx.makeRectangle(131, 143, 39, 54);
-    testOperation(rect,SpatialOperation.Intersects,shape,true);
-  }
-
-  @Test
-  @Repeat(iterations = 10)
-  public void testOperations() throws IOException {
-    setupStrategy();
-
-    testOperationRandomShapes(SpatialOperation.Intersects);
-  }
-
-  private Shape makeTriangle(double x1, double y1, double x2, double y2, double x3, double y3) {
-    final List<GeoPoint> geoPoints = new ArrayList<>();
-    geoPoints.add(new GeoPoint(PlanetModel.SPHERE, y1 * DEGREES_TO_RADIANS, x1 * DEGREES_TO_RADIANS));
-    geoPoints.add(new GeoPoint(PlanetModel.SPHERE, y2 * DEGREES_TO_RADIANS, x2 * DEGREES_TO_RADIANS));
-    geoPoints.add(new GeoPoint(PlanetModel.SPHERE, y3 * DEGREES_TO_RADIANS, x3 * DEGREES_TO_RADIANS));
-    final int convexPointIndex = 0;
-    final GeoShape shape = GeoPolygonFactory.makeGeoPolygon(PlanetModel.SPHERE, geoPoints, convexPointIndex);
-    return new Geo3dShape(shape, ctx);
-  }
-
-  @Override
-  protected Shape randomIndexedShape() {
-    return randomRectangle();
-  }
-
-  @Override
-  protected Shape randomQueryShape() {
-    final int shapeType = random().nextInt(4);
-    switch (shapeType) {
-    case 0: {
-        // Polygons
-        final int vertexCount = random().nextInt(3) + 3;
-        while (true) {
-          final List<GeoPoint> geoPoints = new ArrayList<>();
-          while (geoPoints.size() < vertexCount) {
-            final Point point = randomPoint();
-            final GeoPoint gPt = new GeoPoint(PlanetModel.SPHERE, point.getY() * DEGREES_TO_RADIANS, point.getX() * DEGREES_TO_RADIANS);
-            geoPoints.add(gPt);
-          }
-          final int convexPointIndex = random().nextInt(vertexCount);       //If we get this wrong, hopefully we get IllegalArgumentException
-          try {
-            final GeoShape shape = GeoPolygonFactory.makeGeoPolygon(PlanetModel.SPHERE, geoPoints, convexPointIndex);
-            return new Geo3dShape(shape, ctx);
-          } catch (IllegalArgumentException e) {
-            // This is what happens when we create a shape that is invalid.  Although it is conceivable that there are cases where
-            // the exception is thrown incorrectly, we aren't going to be able to do that in this random test.
-            continue;
-          }
-        }
-      }
-    case 1: {
-        // Circles
-        while (true) {
-          final int circleRadius = random().nextInt(179) + 1;
-          final Point point = randomPoint();
-          try {
-            final GeoShape shape = new GeoStandardCircle(PlanetModel.SPHERE, point.getY() * DEGREES_TO_RADIANS, point.getX() * DEGREES_TO_RADIANS,
-              circleRadius * DEGREES_TO_RADIANS);
-            return new Geo3dShape(shape, ctx);
-          } catch (IllegalArgumentException e) {
-            // This is what happens when we create a shape that is invalid.  Although it is conceivable that there are cases where
-            // the exception is thrown incorrectly, we aren't going to be able to do that in this random test.
-            continue;
-          }
-        }
-      }
-    case 2: {
-        // Rectangles
-        while (true) {
-          Point ulhcPoint = randomPoint();
-          Point lrhcPoint = randomPoint();
-          if (ulhcPoint.getY() < lrhcPoint.getY()) {
-            //swap
-            Point temp = ulhcPoint;
-            ulhcPoint = lrhcPoint;
-            lrhcPoint = temp;
-          }
-          try {
-            final GeoShape shape = GeoBBoxFactory.makeGeoBBox(PlanetModel.SPHERE, ulhcPoint.getY() * DEGREES_TO_RADIANS,
-              lrhcPoint.getY() * DEGREES_TO_RADIANS,
-              ulhcPoint.getX() * DEGREES_TO_RADIANS,
-              lrhcPoint.getX() * DEGREES_TO_RADIANS);
-            //System.err.println("Trial rectangle shape: "+shape);
-            return new Geo3dShape(shape, ctx);
-          } catch (IllegalArgumentException e) {
-            // This is what happens when we create a shape that is invalid.  Although it is conceivable that there are cases where
-            // the exception is thrown incorrectly, we aren't going to be able to do that in this random test.
-            continue;
-          }
-        }
-      }
-    case 3: {
-        // Paths
-        final int pointCount = random().nextInt(5) + 1;
-        final double width = (random().nextInt(89)+1) * DEGREES_TO_RADIANS;
-        while (true) {
-          try {
-            final GeoPath path = new GeoPath(PlanetModel.SPHERE, width);
-            for (int i = 0; i < pointCount; i++) {
-              final Point nextPoint = randomPoint();
-              path.addPoint(nextPoint.getY() * DEGREES_TO_RADIANS, nextPoint.getX() * DEGREES_TO_RADIANS);
-            }
-            path.done();
-            return new Geo3dShape(path, ctx);
-          } catch (IllegalArgumentException e) {
-            // This is what happens when we create a shape that is invalid.  Although it is conceivable that there are cases where
-            // the exception is thrown incorrectly, we aren't going to be able to do that in this random test.
-            continue;
-          }
-        }
-      }
-    default:
-      throw new IllegalStateException("Unexpected shape type");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java b/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java
deleted file mode 100644
index 3113aed..0000000
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java
+++ /dev/null
@@ -1,264 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.spatial.spatial4j;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import com.carrotsearch.randomizedtesting.RandomizedContext;
-import com.spatial4j.core.TestLog;
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Circle;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.RectIntersectionTestHelper;
-import org.apache.lucene.geo3d.LatLonBounds;
-import org.apache.lucene.geo3d.GeoBBox;
-import org.apache.lucene.geo3d.GeoBBoxFactory;
-import org.apache.lucene.geo3d.GeoStandardCircle;
-import org.apache.lucene.geo3d.GeoPath;
-import org.apache.lucene.geo3d.GeoPoint;
-import org.apache.lucene.geo3d.GeoPolygonFactory;
-import org.apache.lucene.geo3d.GeoShape;
-import org.apache.lucene.geo3d.PlanetModel;
-import org.junit.Rule;
-import org.junit.Test;
-
-import static com.spatial4j.core.distance.DistanceUtils.DEGREES_TO_RADIANS;
-
-public abstract class Geo3dShapeRectRelationTestCase extends RandomizedShapeTestCase {
-  protected final static double RADIANS_PER_DEGREE = Math.PI/180.0;
-
-  @Rule
-  public final TestLog testLog = TestLog.instance;
-
-  protected final PlanetModel planetModel;
-
-  public Geo3dShapeRectRelationTestCase(PlanetModel planetModel) {
-    super(SpatialContext.GEO);
-    this.planetModel = planetModel;
-  }
-
-  protected GeoBBox getBoundingBox(final GeoShape path) {
-    LatLonBounds bounds = new LatLonBounds();
-    path.getBounds(bounds);
-
-    double leftLon;
-    double rightLon;
-    if (bounds.checkNoLongitudeBound()) {
-      leftLon = -Math.PI;
-      rightLon = Math.PI;
-    } else {
-      leftLon = bounds.getLeftLongitude().doubleValue();
-      rightLon = bounds.getRightLongitude().doubleValue();
-    }
-    double minLat;
-    if (bounds.checkNoBottomLatitudeBound()) {
-      minLat = -Math.PI * 0.5;
-    } else {
-      minLat = bounds.getMinLatitude().doubleValue();
-    }
-    double maxLat;
-    if (bounds.checkNoTopLatitudeBound()) {
-      maxLat = Math.PI * 0.5;
-    } else {
-      maxLat = bounds.getMaxLatitude().doubleValue();
-    }
-    return GeoBBoxFactory.makeGeoBBox(planetModel, maxLat, minLat, leftLon, rightLon);
-  }
-
-  abstract class Geo3dRectIntersectionTestHelper extends RectIntersectionTestHelper<Geo3dShape> {
-
-    public Geo3dRectIntersectionTestHelper(SpatialContext ctx) {
-      super(ctx);
-    }
-
-    //20 times each -- should be plenty
-
-    protected int getContainsMinimum(int laps) {
-      return 20;
-    }
-
-    protected int getIntersectsMinimum(int laps) {
-      return 20;
-    }
-
-    // producing "within" cases in Geo3D based on our random shapes doesn't happen often. It'd be nice to increase this.
-    protected int getWithinMinimum(int laps) {
-      return 2;
-    }
-
-    protected int getDisjointMinimum(int laps) {
-      return 20;
-    }
-
-    protected int getBoundingMinimum(int laps) {
-      return 20;
-    }
-  }
-
-  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-6867")
-  @Test
-  public void testGeoCircleRect() {
-    new Geo3dRectIntersectionTestHelper(ctx) {
-
-      @Override
-      protected Geo3dShape generateRandomShape(Point nearP) {
-        final int circleRadius = 180 - random().nextInt(180);//no 0-radius
-        final Point point = nearP;
-        final GeoShape shape = new GeoStandardCircle(planetModel, point.getY() * DEGREES_TO_RADIANS, point.getX() * DEGREES_TO_RADIANS,
-            circleRadius * DEGREES_TO_RADIANS);
-        return new Geo3dShape(planetModel, shape, ctx);
-      }
-
-      @Override
-      protected Point randomPointInEmptyShape(Geo3dShape shape) {
-        GeoPoint geoPoint = ((GeoStandardCircle)shape.shape).getCenter();
-        return geoPointToSpatial4jPoint(geoPoint);
-      }
-
-    }.testRelateWithRectangle();
-  }
-
-  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-6867")
-  @Test
-  public void testGeoBBoxRect() {
-    new Geo3dRectIntersectionTestHelper(ctx) {
-
-      @Override
-      protected boolean isRandomShapeRectangular() {
-        return true;
-      }
-
-      @Override
-      protected Geo3dShape generateRandomShape(Point nearP) {
-        // (ignoring nearP)
-        Point ulhcPoint = randomPoint();
-        Point lrhcPoint = randomPoint();
-        if (ulhcPoint.getY() < lrhcPoint.getY()) {
-          //swap
-          Point temp = ulhcPoint;
-          ulhcPoint = lrhcPoint;
-          lrhcPoint = temp;
-        }
-        final GeoShape shape = GeoBBoxFactory.makeGeoBBox(planetModel, ulhcPoint.getY() * DEGREES_TO_RADIANS,
-            lrhcPoint.getY() * DEGREES_TO_RADIANS,
-            ulhcPoint.getX() * DEGREES_TO_RADIANS,
-            lrhcPoint.getX() * DEGREES_TO_RADIANS);
-        return new Geo3dShape(planetModel, shape, ctx);
-      }
-
-      @Override
-      protected Point randomPointInEmptyShape(Geo3dShape shape) {
-        return shape.getBoundingBox().getCenter();
-      }
-    }.testRelateWithRectangle();
-  }
-
-  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-6867")
-  @Test
-  public void testGeoPolygonRect() {
-    new Geo3dRectIntersectionTestHelper(ctx) {
-
-      @Override
-      protected Geo3dShape generateRandomShape(Point nearP) {
-        final Point centerPoint = randomPoint();
-        final int maxDistance = random().nextInt(160) + 20;
-        final Circle pointZone = ctx.makeCircle(centerPoint, maxDistance);
-        final int vertexCount = random().nextInt(3) + 3;
-        while (true) {
-          final List<GeoPoint> geoPoints = new ArrayList<>();
-          while (geoPoints.size() < vertexCount) {
-            final Point point = randomPointIn(pointZone);
-            final GeoPoint gPt = new GeoPoint(planetModel, point.getY() * DEGREES_TO_RADIANS, point.getX() * DEGREES_TO_RADIANS);
-            geoPoints.add(gPt);
-          }
-          final int convexPointIndex = random().nextInt(vertexCount); //If we get this wrong, hopefully we get IllegalArgumentException
-          try {
-            final GeoShape shape = GeoPolygonFactory.makeGeoPolygon(planetModel, geoPoints, convexPointIndex);
-            return new Geo3dShape(planetModel, shape, ctx);
-          } catch (IllegalArgumentException e) {
-            // This is what happens when we create a shape that is invalid.  Although it is conceivable that there are cases where
-            // the exception is thrown incorrectly, we aren't going to be able to do that in this random test.
-            continue;
-          }
-        }
-      }
-
-      @Override
-      protected Point randomPointInEmptyShape(Geo3dShape shape) {
-        throw new IllegalStateException("unexpected; need to finish test code");
-      }
-
-      @Override
-      protected int getWithinMinimum(int laps) {
-        // Long/thin so lets just find 1.
-        return 1;
-      }
-
-    }.testRelateWithRectangle();
-  }
-
-  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-6867")
-  @Test
-  public void testGeoPathRect() {
-    new Geo3dRectIntersectionTestHelper(ctx) {
-
-      @Override
-      protected Geo3dShape generateRandomShape(Point nearP) {
-        final Point centerPoint = randomPoint();
-        final int maxDistance = random().nextInt(160) + 20;
-        final Circle pointZone = ctx.makeCircle(centerPoint, maxDistance);
-        final int pointCount = random().nextInt(5) + 1;
-        final double width = (random().nextInt(89)+1) * DEGREES_TO_RADIANS;
-        while (true) {
-          try {
-            final GeoPath path = new GeoPath(planetModel, width);
-            for (int i = 0; i < pointCount; i++) {
-              final Point nextPoint = randomPointIn(pointZone);
-              path.addPoint(nextPoint.getY() * DEGREES_TO_RADIANS, nextPoint.getX() * DEGREES_TO_RADIANS);
-            }
-            path.done();
-            return new Geo3dShape(planetModel, path, ctx);
-          } catch (IllegalArgumentException e) {
-            // This is what happens when we create a shape that is invalid.  Although it is conceivable that there are cases where
-            // the exception is thrown incorrectly, we aren't going to be able to do that in this random test.
-            continue;
-          }
-        }
-      }
-
-      @Override
-      protected Point randomPointInEmptyShape(Geo3dShape shape) {
-        throw new IllegalStateException("unexpected; need to finish test code");
-      }
-
-      @Override
-      protected int getWithinMinimum(int laps) {
-        // Long/thin so lets just find 1.
-        return 1;
-      }
-
-    }.testRelateWithRectangle();
-  }
-
-  private Point geoPointToSpatial4jPoint(GeoPoint geoPoint) {
-    return ctx.makePoint(geoPoint.getLongitude() * DistanceUtils.RADIANS_TO_DEGREES,
-        geoPoint.getLongitude() * DistanceUtils.RADIANS_TO_DEGREES);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeSphereModelRectRelationTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeSphereModelRectRelationTest.java b/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeSphereModelRectRelationTest.java
deleted file mode 100644
index aac0a0a..0000000
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeSphereModelRectRelationTest.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.spatial.spatial4j;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import com.spatial4j.core.shape.Rectangle;
-import org.apache.lucene.geo3d.GeoArea;
-import org.apache.lucene.geo3d.GeoBBox;
-import org.apache.lucene.geo3d.GeoBBoxFactory;
-import org.apache.lucene.geo3d.GeoStandardCircle;
-import org.apache.lucene.geo3d.GeoPoint;
-import org.apache.lucene.geo3d.GeoPolygonFactory;
-import org.apache.lucene.geo3d.GeoShape;
-import org.apache.lucene.geo3d.PlanetModel;
-import org.junit.Test;
-
-public class Geo3dShapeSphereModelRectRelationTest extends Geo3dShapeRectRelationTestCase {
-
-  public Geo3dShapeSphereModelRectRelationTest() {
-    super(PlanetModel.SPHERE);
-  }
-
-  @Test
-  public void testFailure1() {
-    final GeoBBox rect = GeoBBoxFactory.makeGeoBBox(planetModel, 88 * RADIANS_PER_DEGREE, 30 * RADIANS_PER_DEGREE, -30 * RADIANS_PER_DEGREE, 62 * RADIANS_PER_DEGREE);
-    final List<GeoPoint> points = new ArrayList<>();
-    points.add(new GeoPoint(planetModel, 66.2465299717 * RADIANS_PER_DEGREE, -29.1786158537 * RADIANS_PER_DEGREE));
-    points.add(new GeoPoint(planetModel, 43.684447915 * RADIANS_PER_DEGREE, 46.2210986329 * RADIANS_PER_DEGREE));
-    points.add(new GeoPoint(planetModel, 30.4579218227 * RADIANS_PER_DEGREE, 14.5238410082 * RADIANS_PER_DEGREE));
-    final GeoShape path = GeoPolygonFactory.makeGeoPolygon(planetModel, points,0);
-
-    final GeoPoint point = new GeoPoint(planetModel, 34.2730264413182 * RADIANS_PER_DEGREE, 82.75500168892472 * RADIANS_PER_DEGREE);
-
-    // Apparently the rectangle thinks the polygon is completely within it... "shape inside rectangle"
-    assertTrue(GeoArea.WITHIN == rect.getRelationship(path));
-
-    // Point is within path? Apparently not...
-    assertFalse(path.isWithin(point));
-
-    // If it is within the path, it must be within the rectangle, and similarly visa versa
-    assertFalse(rect.isWithin(point));
-
-  }
-
-  @Test
-  public void testFailure2_LUCENE6475() {
-    GeoShape geo3dCircle = new GeoStandardCircle(planetModel, 1.6282053147165243E-4 * RADIANS_PER_DEGREE,
-        -70.1600629789353 * RADIANS_PER_DEGREE, 86 * RADIANS_PER_DEGREE);
-    Geo3dShape geo3dShape = new Geo3dShape(planetModel, geo3dCircle, ctx);
-    Rectangle rect = ctx.makeRectangle(-118, -114, -2.0, 32.0);
-    assertTrue(geo3dShape.relate(rect).intersects());
-    // thus the bounding box must intersect too
-    assertTrue(geo3dShape.getBoundingBox().relate(rect).intersects());
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeWGS84ModelRectRelationTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeWGS84ModelRectRelationTest.java b/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeWGS84ModelRectRelationTest.java
deleted file mode 100644
index a9ff58d..0000000
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeWGS84ModelRectRelationTest.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.spatial.spatial4j;
-
-import org.apache.lucene.geo3d.GeoArea;
-import org.apache.lucene.geo3d.GeoBBox;
-import org.apache.lucene.geo3d.GeoBBoxFactory;
-import org.apache.lucene.geo3d.GeoCircle;
-import org.apache.lucene.geo3d.GeoStandardCircle;
-import org.apache.lucene.geo3d.GeoPath;
-import org.apache.lucene.geo3d.GeoPoint;
-import org.apache.lucene.geo3d.PlanetModel;
-import org.junit.Test;
-
-public class Geo3dShapeWGS84ModelRectRelationTest extends Geo3dShapeRectRelationTestCase {
-
-  public Geo3dShapeWGS84ModelRectRelationTest() {
-    super(PlanetModel.WGS84);
-  }
-
-  @Test
-  public void testFailure1() {
-    final GeoBBox rect = GeoBBoxFactory.makeGeoBBox(planetModel, 90 * RADIANS_PER_DEGREE, 74 * RADIANS_PER_DEGREE,
-        40 * RADIANS_PER_DEGREE, 60 * RADIANS_PER_DEGREE);
-    final GeoPath path = new GeoPath(planetModel, 4 * RADIANS_PER_DEGREE);
-    path.addPoint(84.4987594274 * RADIANS_PER_DEGREE, -22.8345484402 * RADIANS_PER_DEGREE);
-    path.done();
-    assertTrue(GeoArea.DISJOINT == rect.getRelationship(path));
-    // This is what the test failure claimed...
-    //assertTrue(GeoArea.CONTAINS == rect.getRelationship(path));
-    //final GeoBBox bbox = getBoundingBox(path);
-    //assertFalse(GeoArea.DISJOINT == rect.getRelationship(bbox));
-  }
-
-  @Test
-  public void testFailure2() {
-    final GeoBBox rect = GeoBBoxFactory.makeGeoBBox(planetModel, -74 * RADIANS_PER_DEGREE, -90 * RADIANS_PER_DEGREE,
-        0 * RADIANS_PER_DEGREE, 26 * RADIANS_PER_DEGREE);
-    final GeoCircle circle = new GeoStandardCircle(planetModel, -87.3647352103 * RADIANS_PER_DEGREE, 52.3769709972 * RADIANS_PER_DEGREE, 1 * RADIANS_PER_DEGREE);
-    assertTrue(GeoArea.DISJOINT == rect.getRelationship(circle));
-    // This is what the test failure claimed...
-    //assertTrue(GeoArea.CONTAINS == rect.getRelationship(circle));
-    //final GeoBBox bbox = getBoundingBox(circle);
-    //assertFalse(GeoArea.DISJOINT == rect.getRelationship(bbox));
-  }
-
-  @Test
-  public void testFailure3() {
-    /*
-   [junit4]   1> S-R Rel: {}, Shape {}, Rectangle {}    lap# {} [CONTAINS, Geo3dShape{planetmodel=PlanetModel: {ab=1.0011188180710464, c=0.9977622539852008}, shape=GeoPath: {planetmodel=PlanetModel: {ab=1.0011188180710464, c=0.9977622539852008}, width=1.53588974175501(87.99999999999999), 
-    points={[[X=0.12097657665150223, Y=-0.6754177666095532, Z=0.7265376136709238], [X=-0.3837892785614207, Y=0.4258049113530899, Z=0.8180007850434892]]}}}, 
-    Rect(minX=4.0,maxX=36.0,minY=16.0,maxY=16.0), 6981](no slf4j subst; sorry)
-   [junit4] FAILURE 0.59s | Geo3dWGS84ShapeRectRelationTest.testGeoPathRect <<<
-   [junit4]    > Throwable #1: java.lang.AssertionError: Geo3dShape{planetmodel=PlanetModel: {ab=1.0011188180710464, c=0.9977622539852008}, shape=GeoPath: {planetmodel=PlanetModel: {ab=1.0011188180710464, c=0.9977622539852008}, width=1.53588974175501(87.99999999999999), 
-    points={[[X=0.12097657665150223, Y=-0.6754177666095532, Z=0.7265376136709238], [X=-0.3837892785614207, Y=0.4258049113530899, Z=0.8180007850434892]]}}} intersect Pt(x=23.81626064835212,y=16.0)
-   [junit4]    >  at __randomizedtesting.SeedInfo.seed([2595268DA3F13FEA:6CC30D8C83453E5D]:0)
-   [junit4]    >  at org.apache.lucene.spatial.spatial4j.RandomizedShapeTestCase._assertIntersect(RandomizedShapeTestCase.java:168)
-   [junit4]    >  at org.apache.lucene.spatial.spatial4j.RandomizedShapeTestCase.assertRelation(RandomizedShapeTestCase.java:153)
-   [junit4]    >  at org.apache.lucene.spatial.spatial4j.RectIntersectionTestHelper.testRelateWithRectangle(RectIntersectionTestHelper.java:128)
-   [junit4]    >  at org.apache.lucene.spatial.spatial4j.Geo3dWGS84ShapeRectRelationTest.testGeoPathRect(Geo3dWGS84ShapeRectRelationTest.java:265)
-  */
-    final GeoBBox rect = GeoBBoxFactory.makeGeoBBox(planetModel, 16 * RADIANS_PER_DEGREE, 16 * RADIANS_PER_DEGREE, 4 * RADIANS_PER_DEGREE, 36 * RADIANS_PER_DEGREE);
-    final GeoPoint pt = new GeoPoint(planetModel, 16 * RADIANS_PER_DEGREE, 23.81626064835212 * RADIANS_PER_DEGREE);
-    final GeoPath path = new GeoPath(planetModel, 88 * RADIANS_PER_DEGREE);
-    path.addPoint(46.6369060853 * RADIANS_PER_DEGREE, -79.8452213228 * RADIANS_PER_DEGREE);
-    path.addPoint(54.9779334519 * RADIANS_PER_DEGREE, 132.029177424 * RADIANS_PER_DEGREE);
-    path.done();
-    System.out.println("rect=" + rect);
-    // Rectangle is within path (this is wrong; it's on the other side.  Should be OVERLAPS)
-    assertTrue(GeoArea.OVERLAPS == rect.getRelationship(path));
-    // Rectangle contains point
-    //assertTrue(rect.isWithin(pt));
-    // Path contains point (THIS FAILS)
-    //assertTrue(path.isWithin(pt));
-    // What happens: (1) The center point of the horizontal line is within the path, in fact within a radius of one of the endpoints.
-    // (2) The point mentioned is NOT inside either SegmentEndpoint.
-    // (3) The point mentioned is NOT inside the path segment, either.  (I think it should be...)
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/RandomizedShapeTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/RandomizedShapeTestCase.java b/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/RandomizedShapeTestCase.java
deleted file mode 100644
index db72520..0000000
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/RandomizedShapeTestCase.java
+++ /dev/null
@@ -1,289 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.spatial.spatial4j;
-
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Circle;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
-import com.spatial4j.core.shape.impl.Range;
-
-import static com.spatial4j.core.shape.SpatialRelation.CONTAINS;
-import static com.spatial4j.core.shape.SpatialRelation.WITHIN;
-
-import org.apache.lucene.util.LuceneTestCase;
-
-import static com.carrotsearch.randomizedtesting.RandomizedTest.*;
-
-/**
- * A base test class with utility methods to help test shapes.
- * Extends from RandomizedTest.
- */
-public abstract class RandomizedShapeTestCase extends LuceneTestCase {
-
-  protected static final double EPS = 10e-9;
-
-  protected SpatialContext ctx;//needs to be set ASAP
-
-  /** Used to reduce the space of numbers to increase the likelihood that
-   * random numbers become equivalent, and thus trigger different code paths.
-   * Also makes some random shapes easier to manually examine.
-   */
-  protected final double DIVISIBLE = 2;// even coordinates; (not always used)
-
-  protected RandomizedShapeTestCase() {
-  }
-
-  public RandomizedShapeTestCase(SpatialContext ctx) {
-    this.ctx = ctx;
-  }
-
-  @SuppressWarnings("unchecked")
-  public static void checkShapesImplementEquals( Class<?>[] classes ) {
-    for( Class<?> clazz : classes ) {
-      try {
-        clazz.getDeclaredMethod( "equals", Object.class );
-      } catch (Exception e) {
-        fail("Shape needs to define 'equals' : " + clazz.getName());
-      }
-      try {
-        clazz.getDeclaredMethod( "hashCode" );
-      } catch (Exception e) {
-        fail("Shape needs to define 'hashCode' : " + clazz.getName());
-      }
-    }
-  }
-
-  //These few norm methods normalize the arguments for creating a shape to
-  // account for the dateline. Some tests loop past the dateline or have offsets
-  // that go past it and it's easier to have them coded that way and correct for
-  // it here.  These norm methods should be used when needed, not frivolously.
-
-  protected double normX(double x) {
-    return ctx.isGeo() ? DistanceUtils.normLonDEG(x) : x;
-  }
-
-  protected double normY(double y) {
-    return ctx.isGeo() ? DistanceUtils.normLatDEG(y) : y;
-  }
-
-  protected Rectangle makeNormRect(double minX, double maxX, double minY, double maxY) {
-    if (ctx.isGeo()) {
-      if (Math.abs(maxX - minX) >= 360) {
-        minX = -180;
-        maxX = 180;
-      } else {
-        minX = DistanceUtils.normLonDEG(minX);
-        maxX = DistanceUtils.normLonDEG(maxX);
-      }
-
-    } else {
-      if (maxX < minX) {
-        double t = minX;
-        minX = maxX;
-        maxX = t;
-      }
-      minX = boundX(minX, ctx.getWorldBounds());
-      maxX = boundX(maxX, ctx.getWorldBounds());
-    }
-    if (maxY < minY) {
-      double t = minY;
-      minY = maxY;
-      maxY = t;
-    }
-    minY = boundY(minY, ctx.getWorldBounds());
-    maxY = boundY(maxY, ctx.getWorldBounds());
-    return ctx.makeRectangle(minX, maxX, minY, maxY);
-  }
-
-  public static double divisible(double v, double divisible) {
-    return (int) (Math.round(v / divisible) * divisible);
-  }
-
-  protected double divisible(double v) {
-    return divisible(v, DIVISIBLE);
-  }
-
-  /** reset()'s p, and confines to world bounds. Might not be divisible if
-   * the world bound isn't divisible too.
-   */
-  protected Point divisible(Point p) {
-    Rectangle bounds = ctx.getWorldBounds();
-    double newX = boundX( divisible(p.getX()), bounds );
-    double newY = boundY( divisible(p.getY()), bounds );
-    p.reset(newX, newY);
-    return p;
-  }
-
-  static double boundX(double i, Rectangle bounds) {
-    return bound(i, bounds.getMinX(), bounds.getMaxX());
-  }
-
-  static double boundY(double i, Rectangle bounds) {
-    return bound(i, bounds.getMinY(), bounds.getMaxY());
-  }
-
-  static double bound(double i, double min, double max) {
-    if (i < min) return min;
-    if (i > max) return max;
-    return i;
-  }
-
-  protected void assertRelation(SpatialRelation expected, Shape a, Shape b) {
-    assertRelation(null, expected, a, b);
-  }
-
-  protected void assertRelation(String msg, SpatialRelation expected, Shape a, Shape b) {
-    _assertIntersect(msg, expected, a, b);
-    //check flipped a & b w/ transpose(), while we're at it
-    _assertIntersect(msg, expected.transpose(), b, a);
-  }
-
-  private void _assertIntersect(String msg, SpatialRelation expected, Shape a, Shape b) {
-    SpatialRelation sect = a.relate(b);
-    if (sect == expected)
-      return;
-    msg = ((msg == null) ? "" : msg+"\r") + a +" intersect "+b;
-    if (expected == WITHIN || expected == CONTAINS) {
-      if (a.getClass().equals(b.getClass())) // they are the same shape type
-        assertEquals(msg,a,b);
-      else {
-        //they are effectively points or lines that are the same location
-        assertTrue(msg,!a.hasArea());
-        assertTrue(msg,!b.hasArea());
-
-        Rectangle aBBox = a.getBoundingBox();
-        Rectangle bBBox = b.getBoundingBox();
-        if (aBBox.getHeight() == 0 && bBBox.getHeight() == 0
-            && (aBBox.getMaxY() == 90 && bBBox.getMaxY() == 90
-            || aBBox.getMinY() == -90 && bBBox.getMinY() == -90))
-          ;//== a point at the pole
-        else
-          assertEquals(msg, aBBox, bBBox);
-      }
-    } else {
-      assertEquals(msg,expected,sect);//always fails
-    }
-  }
-
-  protected void assertEqualsRatio(String msg, double expected, double actual) {
-    double delta = Math.abs(actual - expected);
-    double base = Math.min(actual, expected);
-    double deltaRatio = base==0 ? delta : Math.min(delta,delta / base);
-    assertEquals(msg,0,deltaRatio, EPS);
-  }
-
-  protected int randomIntBetweenDivisible(int start, int end) {
-    return randomIntBetweenDivisible(start, end, (int)DIVISIBLE);
-  }
-  /** Returns a random integer between [start, end]. Integers between must be divisible by the 3rd argument. */
-  protected int randomIntBetweenDivisible(int start, int end, int divisible) {
-    // DWS: I tested this
-    int divisStart = (int) Math.ceil( (start+1) / (double)divisible );
-    int divisEnd = (int) Math.floor( (end-1) / (double)divisible );
-    int divisRange = Math.max(0,divisEnd - divisStart + 1);
-    int r = randomInt(1 + divisRange);//remember that '0' is counted
-    if (r == 0)
-      return start;
-    if (r == 1)
-      return end;
-    return (r-2 + divisStart)*divisible;
-  }
-
-  protected Rectangle randomRectangle(Point nearP) {
-    Rectangle bounds = ctx.getWorldBounds();
-    if (nearP == null)
-      nearP = randomPointIn(bounds);
-
-    Range xRange = randomRange(rarely() ? 0 : nearP.getX(), Range.xRange(bounds, ctx));
-    Range yRange = randomRange(rarely() ? 0 : nearP.getY(), Range.yRange(bounds, ctx));
-
-    return makeNormRect(
-        divisible(xRange.getMin()),
-        divisible(xRange.getMax()),
-        divisible(yRange.getMin()),
-        divisible(yRange.getMax()) );
-  }
-
-  private Range randomRange(double near, Range bounds) {
-    double mid = near + randomGaussian() * bounds.getWidth() / 6;
-    double width = Math.abs(randomGaussian()) * bounds.getWidth() / 6;//1/3rd
-    return new Range(mid - width / 2, mid + width / 2);
-  }
-
-  private double randomGaussianZeroTo(double max) {
-    if (max == 0)
-      return max;
-    assert max > 0;
-    double r;
-    do {
-      r = Math.abs(randomGaussian()) * (max * 0.50);
-    } while (r > max);
-    return r;
-  }
-
-  protected Rectangle randomRectangle(int divisible) {
-    double rX = randomIntBetweenDivisible(-180, 180, divisible);
-    double rW = randomIntBetweenDivisible(0, 360, divisible);
-    double rY1 = randomIntBetweenDivisible(-90, 90, divisible);
-    double rY2 = randomIntBetweenDivisible(-90, 90, divisible);
-    double rYmin = Math.min(rY1,rY2);
-    double rYmax = Math.max(rY1,rY2);
-    if (rW > 0 && rX == 180)
-      rX = -180;
-    return makeNormRect(rX, rX + rW, rYmin, rYmax);
-  }
-
-  protected Point randomPoint() {
-    return randomPointIn(ctx.getWorldBounds());
-  }
-
-  protected Point randomPointIn(Circle c) {
-    double d = c.getRadius() * randomDouble();
-    double angleDEG = 360 * randomDouble();
-    Point p = ctx.getDistCalc().pointOnBearing(c.getCenter(), d, angleDEG, ctx, null);
-    assertEquals(CONTAINS,c.relate(p));
-    return p;
-  }
-
-  protected Point randomPointIn(Rectangle r) {
-    double x = r.getMinX() + randomDouble()*r.getWidth();
-    double y = r.getMinY() + randomDouble()*r.getHeight();
-    x = normX(x);
-    y = normY(y);
-    Point p = ctx.makePoint(x,y);
-    assertEquals(CONTAINS,r.relate(p));
-    return p;
-  }
-
-  protected Point randomPointInOrNull(Shape shape) {
-    if (!shape.hasArea())// or try the center?
-      throw new UnsupportedOperationException("Need area to define shape!");
-    Rectangle bbox = shape.getBoundingBox();
-    for (int i = 0; i < 1000; i++) {
-      Point p = randomPointIn(bbox);
-      if (shape.relate(p).intersects()) {
-        return p;
-      }
-    }
-    return null;//tried too many times and failed
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/geo3d/GeoPointTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/geo3d/GeoPointTest.java b/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/geo3d/GeoPointTest.java
deleted file mode 100644
index e652581..0000000
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/spatial4j/geo3d/GeoPointTest.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.spatial.spatial4j.geo3d;
-
-import org.apache.lucene.geo3d.GeoPoint;
-import org.apache.lucene.geo3d.PlanetModel;
-import org.apache.lucene.util.LuceneTestCase;
-import org.junit.Test;
-
-import com.spatial4j.core.distance.DistanceUtils;
-
-import static com.carrotsearch.randomizedtesting.RandomizedTest.randomFloat;
-
-/**
- * Test basic GeoPoint functionality.
- */
-public class GeoPointTest extends LuceneTestCase {
-
-  @Test
-  public void testConversion() {
-    testPointRoundTrip(PlanetModel.SPHERE, 90 * DistanceUtils.DEGREES_TO_RADIANS, 0, 1e-6);
-    testPointRoundTrip(PlanetModel.SPHERE, -90 * DistanceUtils.DEGREES_TO_RADIANS, 0, 1e-6);
-    testPointRoundTrip(PlanetModel.WGS84, 90 * DistanceUtils.DEGREES_TO_RADIANS, 0, 1e-6);
-    testPointRoundTrip(PlanetModel.WGS84, -90 * DistanceUtils.DEGREES_TO_RADIANS, 0, 1e-6);
-
-    final int times = atLeast(100);
-    for (int i = 0; i < times; i++) {
-      final double pLat = (randomFloat() * 180.0 - 90.0) * DistanceUtils.DEGREES_TO_RADIANS;
-      final double pLon = (randomFloat() * 360.0 - 180.0) * DistanceUtils.DEGREES_TO_RADIANS;
-      testPointRoundTrip(PlanetModel.SPHERE, pLat, pLon, 1e-6);//1e-6 since there's a square root in there (Karl says)
-      testPointRoundTrip(PlanetModel.WGS84, pLat, pLon, 1e-6);
-    }
-  }
-
-  protected void testPointRoundTrip(PlanetModel planetModel, double pLat, double pLon, double epsilon) {
-    final GeoPoint p1 = new GeoPoint(planetModel, pLat, pLon);
-    // In order to force the reverse conversion, we have to construct a geopoint from just x,y,z
-    final GeoPoint p2 = new GeoPoint(p1.x, p1.y, p1.z);
-    // Now, construct the final point based on getLatitude() and getLongitude()
-    final GeoPoint p3 = new GeoPoint(planetModel, p2.getLatitude(), p2.getLongitude());
-    double dist = p1.arcDistance(p3);
-    assertEquals(0, dist, epsilon);
-  }
-
-  @Test
-  public void testSurfaceDistance() {
-    final int times = atLeast(100);
-    for (int i = 0; i < times; i++) {
-      final double p1Lat = (randomFloat() * 180.0 - 90.0) * DistanceUtils.DEGREES_TO_RADIANS;
-      final double p1Lon = (randomFloat() * 360.0 - 180.0) * DistanceUtils.DEGREES_TO_RADIANS;
-      final double p2Lat = (randomFloat() * 180.0 - 90.0) * DistanceUtils.DEGREES_TO_RADIANS;
-      final double p2Lon = (randomFloat() * 360.0 - 180.0) * DistanceUtils.DEGREES_TO_RADIANS;
-      final GeoPoint p1 = new GeoPoint(PlanetModel.SPHERE, p1Lat, p1Lon);
-      final GeoPoint p2 = new GeoPoint(PlanetModel.SPHERE, p2Lat, p2Lon);
-      final double arcDistance = p1.arcDistance(p2);
-      // Compute ellipsoid distance; it should agree for a sphere
-      final double surfaceDistance = PlanetModel.SPHERE.surfaceDistance(p1,p2);
-      assertEquals(arcDistance, surfaceDistance, 1e-6);
-    }
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testBadLatLon() {
-    new GeoPoint(PlanetModel.SPHERE, 50.0, 32.2);
-  }
-}
-
-