You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by iv...@apache.org on 2019/01/21 15:03:11 UTC

[lucene-solr] branch master updated: LUCENE-8649: LatLonShape's within and disjoint queries can return false positives with indexed multi-shapes

This is an automated email from the ASF dual-hosted git repository.

ivera pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 01dfe7b  LUCENE-8649: LatLonShape's within and disjoint queries can return false positives with indexed multi-shapes
01dfe7b is described below

commit 01dfe7bf4b2bd05326c66cc6297300f3dd321547
Author: iverase <iv...@apache.org>
AuthorDate: Mon Jan 21 16:00:28 2019 +0100

    LUCENE-8649: LatLonShape's within and disjoint queries can return false positives with indexed multi-shapes
---
 lucene/CHANGES.txt                                 |   3 +
 .../document/LatLonShapeBoundingBoxQuery.java      |   2 +-
 .../lucene/document/LatLonShapeLineQuery.java      |   2 +-
 .../lucene/document/LatLonShapePolygonQuery.java   |   2 +-
 .../apache/lucene/document/LatLonShapeQuery.java   |  48 +++++---
 .../lucene/document/BaseLatLonShapeTestCase.java   |  32 +++--
 .../document/TestLatLonLineShapeQueries.java       |   2 +-
 .../document/TestLatLonMultiLineShapeQueries.java  | 124 +++++++++++++++++++
 .../document/TestLatLonMultiPointShapeQueries.java | 124 +++++++++++++++++++
 .../TestLatLonMultiPolygonShapeQueries.java        | 136 +++++++++++++++++++++
 .../document/TestLatLonPointShapeQueries.java      |   2 +-
 .../document/TestLatLonPolygonShapeQueries.java    |   2 +-
 12 files changed, 445 insertions(+), 34 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index b6a600d..7e38150 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -274,6 +274,9 @@ Bug fixes:
 * LUCENE-8639: Newly created threadstates while flushing / refreshing can cause duplicated
   sequence IDs on IndexWriter. (Simon Willnauer)
 
+* LUCENE-8649: LatLonShape's within and disjoint queries can return false positives with
+  indexed multi-shapes. (Ignacio Vera)
+
 New Features
 
 * LUCENE-8026: ExitableDirectoryReader may now time out queries that run on
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java
index 79d4d07..21babd7 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java
@@ -45,7 +45,7 @@ final class LatLonShapeBoundingBoxQuery extends LatLonShapeQuery {
 
   /** returns true if the query matches the encoded triangle */
   @Override
-  protected boolean queryMatches(byte[] t, int[] scratchTriangle) {
+  protected boolean queryMatches(byte[] t, int[] scratchTriangle, LatLonShape.QueryRelation queryRelation) {
     // decode indexed triangle
     LatLonShape.decodeTriangle(t, scratchTriangle);
 
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeLineQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeLineQuery.java
index fc6836e..e793345 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeLineQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeLineQuery.java
@@ -84,7 +84,7 @@ final class LatLonShapeLineQuery extends LatLonShapeQuery {
   }
 
   @Override
-  protected boolean queryMatches(byte[] t, int[] scratchTriangle) {
+  protected boolean queryMatches(byte[] t, int[] scratchTriangle, QueryRelation queryRelation) {
     LatLonShape.decodeTriangle(t, scratchTriangle);
 
     double alat = GeoEncodingUtils.decodeLatitude(scratchTriangle[0]);
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapePolygonQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapePolygonQuery.java
index f1ac299..94dde0d 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapePolygonQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapePolygonQuery.java
@@ -73,7 +73,7 @@ final class LatLonShapePolygonQuery extends LatLonShapeQuery {
   }
 
   @Override
-  protected boolean queryMatches(byte[] t, int[] scratchTriangle) {
+  protected boolean queryMatches(byte[] t, int[] scratchTriangle, QueryRelation queryRelation) {
     LatLonShape.decodeTriangle(t, scratchTriangle);
 
     double alat = GeoEncodingUtils.decodeLatitude(scratchTriangle[0]);
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeQuery.java
index 847043a..538f4a3 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeQuery.java
@@ -73,10 +73,10 @@ abstract class LatLonShapeQuery extends Query {
                                                      int maxXOffset, int maxYOffset, byte[] maxTriangle);
 
   /** returns true if the provided triangle matches the query */
-  protected abstract boolean queryMatches(byte[] triangle, int[] scratchTriangle);
+  protected abstract boolean queryMatches(byte[] triangle, int[] scratchTriangle, QueryRelation queryRelation);
 
   /** relates a range of triangles (internal node) to the query */
-  protected Relation relateRangeToQuery(byte[] minTriangle, byte[] maxTriangle) {
+  protected Relation relateRangeToQuery(byte[] minTriangle, byte[] maxTriangle, QueryRelation queryRelation) {
     // compute bounding box of internal node
     Relation r = relateRangeBBoxToQuery(LatLonShape.BYTES, 0, minTriangle, 3 * LatLonShape.BYTES, 2 * LatLonShape.BYTES, maxTriangle);
     if (queryRelation == QueryRelation.DISJOINT) {
@@ -108,20 +108,20 @@ abstract class LatLonShapeQuery extends Query {
 
           @Override
           public void visit(int docID, byte[] t) throws IOException {
-            if (queryMatches(t, scratchTriangle)) {
+            if (queryMatches(t, scratchTriangle, QueryRelation.INTERSECTS)) {
               adder.add(docID);
             }
           }
 
           @Override
           public Relation compare(byte[] minTriangle, byte[] maxTriangle) {
-            return relateRangeToQuery(minTriangle, maxTriangle);
+            return relateRangeToQuery(minTriangle, maxTriangle, QueryRelation.INTERSECTS);
           }
         };
       }
 
       /** create a visitor that adds documents that match the query using a dense bitset. (Used by WITHIN, DISJOINT) */
-      protected IntersectVisitor getDenseIntersectVisitor(FixedBitSet intersect, FixedBitSet disjoint) {
+      protected IntersectVisitor getDenseIntersectVisitor(FixedBitSet intersect, FixedBitSet disjoint, QueryRelation queryRelation) {
         return new IntersectVisitor() {
           final int[] scratchTriangle = new int[6];
           @Override
@@ -137,7 +137,7 @@ abstract class LatLonShapeQuery extends Query {
 
           @Override
           public void visit(int docID, byte[] t) throws IOException {
-            if (queryMatches(t, scratchTriangle)) {
+            if (queryMatches(t, scratchTriangle, queryRelation)) {
               intersect.set(docID);
             } else {
               disjoint.set(docID);
@@ -146,7 +146,7 @@ abstract class LatLonShapeQuery extends Query {
 
           @Override
           public Relation compare(byte[] minTriangle, byte[] maxTriangle) {
-            return relateRangeToQuery(minTriangle, maxTriangle);
+            return relateRangeToQuery(minTriangle, maxTriangle, queryRelation);
           }
         };
       }
@@ -155,7 +155,7 @@ abstract class LatLonShapeQuery extends Query {
       protected ScorerSupplier getIntersectScorerSupplier(LeafReader reader, PointValues values, Weight weight, ScoreMode scoreMode) throws IOException {
         DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
         IntersectVisitor visitor = getSparseIntersectVisitor(result);
-        return new RelationScorerSupplier(values, visitor) {
+        return new RelationScorerSupplier(values, visitor, null, queryRelation) {
           @Override
           public Scorer get(long leadCost) throws IOException {
             return getIntersectsScorer(LatLonShapeQuery.this, reader, weight, result, score(), scoreMode);
@@ -168,14 +168,15 @@ abstract class LatLonShapeQuery extends Query {
         if (queryRelation == QueryRelation.INTERSECTS) {
           return getIntersectScorerSupplier(reader, values, weight, scoreMode);
         }
-
-        FixedBitSet intersect = new FixedBitSet(reader.maxDoc());
+        //For within and disjoint we need two passes to remove false positives in case of multi-shapes.
+        FixedBitSet within = new FixedBitSet(reader.maxDoc());
         FixedBitSet disjoint = new FixedBitSet(reader.maxDoc());
-        IntersectVisitor visitor = getDenseIntersectVisitor(intersect, disjoint);
-        return new RelationScorerSupplier(values, visitor) {
+        IntersectVisitor withinVisitor = getDenseIntersectVisitor(within, disjoint, QueryRelation.WITHIN);
+        IntersectVisitor disjointVisitor = getDenseIntersectVisitor(within, disjoint, QueryRelation.DISJOINT);
+        return new RelationScorerSupplier(values, withinVisitor, disjointVisitor, queryRelation) {
           @Override
           public Scorer get(long leadCost) throws IOException {
-            return getScorer(LatLonShapeQuery.this, weight, intersect, disjoint, score(), scoreMode);
+            return getScorer(LatLonShapeQuery.this, weight, within, disjoint, score(), scoreMode);
           }
         };
       }
@@ -196,7 +197,7 @@ abstract class LatLonShapeQuery extends Query {
 
         boolean allDocsMatch = true;
         if (values.getDocCount() != reader.maxDoc() ||
-            relateRangeToQuery(values.getMinPackedValue(), values.getMaxPackedValue()) != Relation.CELL_INSIDE_QUERY) {
+            relateRangeToQuery(values.getMinPackedValue(), values.getMaxPackedValue(), queryRelation) != Relation.CELL_INSIDE_QUERY) {
           allDocsMatch = false;
         }
 
@@ -275,11 +276,15 @@ abstract class LatLonShapeQuery extends Query {
   private static abstract class RelationScorerSupplier extends ScorerSupplier {
     PointValues values;
     IntersectVisitor visitor;
+    IntersectVisitor disjointVisitor;//it can be null
+    QueryRelation queryRelation;
     long cost = -1;
 
-    RelationScorerSupplier(PointValues values, IntersectVisitor visitor) {
+    RelationScorerSupplier(PointValues values, IntersectVisitor visitor, IntersectVisitor disjointVisitor, QueryRelation queryRelation) {
       this.values = values;
       this.visitor = visitor;
+      this.disjointVisitor = disjointVisitor;
+      this.queryRelation = queryRelation;
     }
 
     /** create a visitor that clears documents that do NOT match the polygon query; used with INTERSECTS */
@@ -294,7 +299,7 @@ abstract class LatLonShapeQuery extends Query {
 
         @Override
         public void visit(int docID, byte[] packedTriangle) {
-          if (query.queryMatches(packedTriangle, scratchTriangle) == false) {
+          if (query.queryMatches(packedTriangle, scratchTriangle, QueryRelation.INTERSECTS) == false) {
             result.clear(docID);
             cost[0]--;
           }
@@ -302,7 +307,7 @@ abstract class LatLonShapeQuery extends Query {
 
         @Override
         public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-          return transposeRelation(query.relateRangeToQuery(minPackedValue, maxPackedValue));
+          return transposeRelation(query.relateRangeToQuery(minPackedValue, maxPackedValue, QueryRelation.INTERSECTS));
         }
       };
     }
@@ -333,6 +338,9 @@ abstract class LatLonShapeQuery extends Query {
     protected Scorer getScorer(LatLonShapeQuery query, Weight weight,
                                FixedBitSet intersect, FixedBitSet disjoint, final float boost, ScoreMode scoreMode) throws IOException {
       values.intersect(visitor);
+      if (disjointVisitor != null) {
+        values.intersect(disjointVisitor);
+      }
       DocIdSetIterator iterator;
       if (query.queryRelation == QueryRelation.DISJOINT) {
         disjoint.andNot(intersect);
@@ -350,7 +358,11 @@ abstract class LatLonShapeQuery extends Query {
     public long cost() {
       if (cost == -1) {
         // Computing the cost may be expensive, so only do it if necessary
-        cost = values.estimatePointCount(visitor);
+        if (queryRelation == QueryRelation.DISJOINT) {
+          cost = values.estimatePointCount(disjointVisitor);
+        } else {
+          cost = values.estimatePointCount(visitor);
+        }
         assert cost >= 0;
       }
       return cost;
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java
index f26c79a..1631373 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java
@@ -75,33 +75,33 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
   }
 
   /** quantizes a latitude value to be consistent with index encoding */
-  protected double quantizeLat(double rawLat) {
+  protected static double quantizeLat(double rawLat) {
     return decodeLatitude(encodeLatitude(rawLat));
   }
 
   /** quantizes a provided latitude value rounded up to the nearest encoded integer */
-  protected double quantizeLatCeil(double rawLat) {
+  protected static double quantizeLatCeil(double rawLat) {
     return decodeLatitude(encodeLatitudeCeil(rawLat));
   }
 
   /** quantizes a longitude value to be consistent with index encoding */
-  protected double quantizeLon(double rawLon) {
+  protected static double quantizeLon(double rawLon) {
     return decodeLongitude(encodeLongitude(rawLon));
   }
 
   /** quantizes a provided longitude value rounded up to the nearest encoded integer */
-  protected double quantizeLonCeil(double rawLon) {
+  protected static double quantizeLonCeil(double rawLon) {
     return decodeLongitude(encodeLongitudeCeil(rawLon));
   }
 
   /** quantizes a triangle to be consistent with index encoding */
-  protected double[] quantizeTriangle(double ax, double ay, double bx, double by, double cx, double cy) {
+  protected static double[] quantizeTriangle(double ax, double ay, double bx, double by, double cx, double cy) {
     int[] decoded = encodeDecodeTriangle(ax, ay, bx, by, cx, cy);
     return new double[]{decodeLatitude(decoded[0]), decodeLongitude(decoded[1]), decodeLatitude(decoded[2]), decodeLongitude(decoded[3]), decodeLatitude(decoded[4]), decodeLongitude(decoded[5])};
   }
 
   /** encode/decode a triangle */
-  protected int[] encodeDecodeTriangle(double ax, double ay, double bx, double by, double cx, double cy) {
+  protected static int[] encodeDecodeTriangle(double ax, double ay, double bx, double by, double cx, double cy) {
     byte[] encoded = new byte[7 * LatLonShape.BYTES];
     LatLonShape.encodeTriangle(encoded, encodeLatitude(ay), encodeLongitude(ax), encodeLatitude(by), encodeLongitude(bx), encodeLatitude(cy), encodeLongitude(cx));
     int[] decoded = new int[6];
@@ -347,7 +347,11 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
           }
           b.append("  relation=" + queryRelation + "\n");
           b.append("  query=" + query + " docID=" + docID + "\n");
-          b.append("  shape=" + shapes[id] + "\n");
+          if (shapes[id] instanceof Object[]) {
+            b.append("  shape=" + Arrays.toString((Object[]) shapes[id]) + "\n");
+          } else {
+            b.append("  shape=" + shapes[id] + "\n");
+          }
           b.append("  deleted?=" + (liveDocs != null && liveDocs.get(docID) == false));
           b.append("  rect=Rectangle(lat=" + quantizeLatCeil(rect.minLat) + " TO " + quantizeLat(rect.maxLat) + " lon=" + qMinLon + " TO " + quantizeLon(rect.maxLon) + ")\n");          if (true) {
             fail("wrong hit (first of possibly more):\n\n" + b);
@@ -445,7 +449,11 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
           }
           b.append("  relation=" + queryRelation + "\n");
           b.append("  query=" + query + " docID=" + docID + "\n");
-          b.append("  shape=" + shapes[id] + "\n");
+          if (shapes[id] instanceof Object[]) {
+            b.append("  shape=" + Arrays.toString((Object[]) shapes[id]) + "\n");
+          } else {
+            b.append("  shape=" + shapes[id] + "\n");
+          }
           b.append("  deleted?=" + (liveDocs != null && liveDocs.get(docID) == false));
           b.append("  queryPolygon=" + queryLine.toGeoJSON());
           if (true) {
@@ -532,7 +540,11 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
           }
           b.append("  relation=" + queryRelation + "\n");
           b.append("  query=" + query + " docID=" + docID + "\n");
-          b.append("  shape=" + shapes[id] + "\n");
+          if (shapes[id] instanceof Object[]) {
+            b.append("  shape=" + Arrays.toString((Object[]) shapes[id]) + "\n");
+          } else {
+            b.append("  shape=" + shapes[id] + "\n");
+          }
           b.append("  deleted?=" + (liveDocs != null && liveDocs.get(docID) == false));
           b.append("  queryPolygon=" + queryPolygon.toGeoJSON());
           if (true) {
@@ -622,7 +634,7 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
   }
 
   /** validator class used to test query results against "ground truth" */
-  protected abstract class Validator {
+  protected static abstract class Validator {
     protected QueryRelation queryRelation = QueryRelation.INTERSECTS;
     public abstract boolean testBBoxQuery(double minLat, double maxLat, double minLon, double maxLon, Object shape);
     public abstract boolean testLineQuery(Line2D line2d, Object shape);
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonLineShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonLineShapeQueries.java
index 87a2c94..7e6d995 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonLineShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonLineShapeQueries.java
@@ -76,7 +76,7 @@ public class TestLatLonLineShapeQueries extends BaseLatLonShapeTestCase {
     return VALIDATOR;
   }
 
-  protected class LineValidator extends Validator {
+  protected static class LineValidator extends Validator {
     @Override
     public boolean testBBoxQuery(double minLat, double maxLat, double minLon, double maxLon, Object shape) {
       Line line = (Line)shape;
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiLineShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiLineShapeQueries.java
new file mode 100644
index 0000000..5c9c42e
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiLineShapeQueries.java
@@ -0,0 +1,124 @@
+/*
+ * 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.document;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.document.LatLonShape.QueryRelation;
+import org.apache.lucene.geo.Line;
+import org.apache.lucene.geo.Line2D;
+import org.apache.lucene.geo.Polygon2D;
+
+/** random bounding box and polygon query tests for random indexed arrays of {@link Line} types */
+public class TestLatLonMultiLineShapeQueries extends BaseLatLonShapeTestCase {
+
+  protected final MultiLineValidator VALIDATOR = new MultiLineValidator();
+  protected final TestLatLonLineShapeQueries.LineValidator LINEVALIDATOR = new TestLatLonLineShapeQueries.LineValidator();
+
+  @Override
+  protected ShapeType getShapeType() {
+    return ShapeType.LINE;
+  }
+
+  @Override
+  protected Line[] nextShape() {
+    int n = random().nextInt(4) + 1;
+    Line[] lines = new Line[n];
+    for (int i =0; i < n; i++) {
+      lines[i] = nextLine();
+    }
+    return lines;
+  }
+
+  @Override
+  protected Field[] createIndexableFields(String name, Object o) {
+    Line[] lines = (Line[]) o;
+    List<Field> allFields = new ArrayList<>();
+    for (Line line : lines) {
+      Field[] fields = LatLonShape.createIndexableFields(name, line);
+      for (Field field : fields) {
+        allFields.add(field);
+      }
+    }
+    return allFields.toArray(new Field[allFields.size()]);
+  }
+
+  @Override
+  protected Validator getValidator(QueryRelation relation) {
+    VALIDATOR.setRelation(relation);
+    LINEVALIDATOR.setRelation(relation);
+    return VALIDATOR;
+  }
+
+  protected class MultiLineValidator extends Validator {
+    @Override
+    public boolean testBBoxQuery(double minLat, double maxLat, double minLon, double maxLon, Object shape) {
+      Line[] lines = (Line[])shape;
+      for (Line l : lines) {
+        boolean b = LINEVALIDATOR.testBBoxQuery(minLat, maxLat, minLon, maxLon, l);
+        if (b == true && queryRelation == QueryRelation.INTERSECTS) {
+          return true;
+        } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
+          return false;
+        } else if (b == false && queryRelation == QueryRelation.WITHIN) {
+          return false;
+        }
+      }
+      return queryRelation != QueryRelation.INTERSECTS;
+    }
+
+    @Override
+    public boolean testLineQuery(Line2D query, Object shape) {
+      Line[] lines = (Line[])shape;
+      for (Line l : lines) {
+        boolean b = LINEVALIDATOR.testLineQuery(query, l);
+        if (b == true && queryRelation == QueryRelation.INTERSECTS) {
+          return true;
+        } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
+          return false;
+        } else if (b == false && queryRelation == QueryRelation.WITHIN) {
+          return false;
+        }
+      }
+      return queryRelation != QueryRelation.INTERSECTS;
+    }
+
+    @Override
+    public boolean testPolygonQuery(Polygon2D query, Object shape) {
+      Line[] lines = (Line[])shape;
+      for (Line l : lines) {
+        boolean b = LINEVALIDATOR.testPolygonQuery(query, l);
+        if (b == true && queryRelation == QueryRelation.INTERSECTS) {
+          return true;
+        } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
+          return false;
+        } else if (b == false && queryRelation == QueryRelation.WITHIN) {
+          return false;
+        }
+      }
+      return queryRelation != QueryRelation.INTERSECTS;
+    }
+  }
+
+  @Slow
+  @Nightly
+  @Override
+  public void testRandomBig() throws Exception {
+    doTestRandom(10000);
+  }
+}
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiPointShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiPointShapeQueries.java
new file mode 100644
index 0000000..44d095d
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiPointShapeQueries.java
@@ -0,0 +1,124 @@
+/*
+ * 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.document;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.document.LatLonShape.QueryRelation;
+import org.apache.lucene.geo.GeoTestUtil;
+import org.apache.lucene.geo.Line2D;
+import org.apache.lucene.geo.Polygon2D;
+
+/** random bounding box and polygon query tests for random indexed arrays of {@code latitude, longitude} points */
+public class TestLatLonMultiPointShapeQueries extends BaseLatLonShapeTestCase {
+
+  protected final MultiPointValidator VALIDATOR = new MultiPointValidator();
+  protected final TestLatLonPointShapeQueries.PointValidator POINTVALIDATOR = new TestLatLonPointShapeQueries.PointValidator();
+
+  @Override
+  protected ShapeType getShapeType() {
+    return ShapeType.POINT;
+  }
+
+  @Override
+  protected Point[] nextShape() {
+    int n = random().nextInt(4) + 1;
+    Point[] points = new Point[n];
+    for (int i =0; i < n; i++) {
+      points[i] = new Point(GeoTestUtil.nextLatitude(), GeoTestUtil.nextLongitude());
+    }
+    return points;
+  }
+
+  @Override
+  protected Field[] createIndexableFields(String name, Object o) {
+    Point[] points = (Point[]) o;
+    List<Field> allFields = new ArrayList<>();
+    for (Point point : points) {
+      Field[] fields = LatLonShape.createIndexableFields(name, point.lat, point.lon);
+      for (Field field : fields) {
+        allFields.add(field);
+      }
+    }
+    return allFields.toArray(new Field[allFields.size()]);
+  }
+
+  @Override
+  protected Validator getValidator(QueryRelation relation) {
+    VALIDATOR.setRelation(relation);
+    POINTVALIDATOR.setRelation(relation);
+    return VALIDATOR;
+  }
+
+  protected class MultiPointValidator extends Validator {
+    @Override
+    public boolean testBBoxQuery(double minLat, double maxLat, double minLon, double maxLon, Object shape) {
+      Point[] points = (Point[]) shape;
+      for (Point p : points) {
+        boolean b = POINTVALIDATOR.testBBoxQuery(minLat, maxLat, minLon, maxLon, p);
+        if (b == true && queryRelation == QueryRelation.INTERSECTS) {
+          return true;
+        } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
+          return false;
+        } else if (b == false && queryRelation == QueryRelation.WITHIN) {
+          return false;
+        }
+      }
+      return queryRelation != QueryRelation.INTERSECTS;
+    }
+
+    @Override
+    public boolean testLineQuery(Line2D query, Object shape) {
+      Point[] points = (Point[]) shape;
+      for (Point p : points) {
+        boolean b = POINTVALIDATOR.testLineQuery(query, p);
+        if (b == true && queryRelation == QueryRelation.INTERSECTS) {
+          return true;
+        } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
+          return false;
+        } else if (b == false && queryRelation == QueryRelation.WITHIN) {
+          return false;
+        }
+      }
+      return queryRelation != QueryRelation.INTERSECTS;
+    }
+
+    @Override
+    public boolean testPolygonQuery(Polygon2D query, Object shape) {
+      Point[] points = (Point[]) shape;
+      for (Point p : points) {
+        boolean b = POINTVALIDATOR.testPolygonQuery(query, p);
+        if (b == true && queryRelation == QueryRelation.INTERSECTS) {
+          return true;
+        } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
+          return false;
+        } else if (b == false && queryRelation == QueryRelation.WITHIN) {
+          return false;
+        }
+      }
+      return queryRelation != QueryRelation.INTERSECTS;
+    }
+  }
+
+  @Slow
+  @Nightly
+  @Override
+  public void testRandomBig() throws Exception {
+    doTestRandom(10000);
+  }
+}
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiPolygonShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiPolygonShapeQueries.java
new file mode 100644
index 0000000..3729bba
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonMultiPolygonShapeQueries.java
@@ -0,0 +1,136 @@
+/*
+ * 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.document;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.document.LatLonShape.QueryRelation;
+import org.apache.lucene.geo.Line2D;
+import org.apache.lucene.geo.Polygon;
+import org.apache.lucene.geo.Polygon2D;
+import org.apache.lucene.geo.Tessellator;
+
+/** random bounding box and polygon query tests for random indexed arrays of {@link Polygon} types */
+public class TestLatLonMultiPolygonShapeQueries extends BaseLatLonShapeTestCase {
+
+  protected final MultiPolygonValidator VALIDATOR = new MultiPolygonValidator();
+  protected final TestLatLonPolygonShapeQueries.PolygonValidator POLYGONVALIDATOR = new TestLatLonPolygonShapeQueries.PolygonValidator();
+
+  @Override
+  protected ShapeType getShapeType() {
+    return ShapeType.POLYGON;
+  }
+
+  @Override
+  protected Polygon[] nextShape() {
+
+    int n = random().nextInt(4) + 1;
+    Polygon[] polygons = new Polygon[n];
+    for (int i =0; i < n; i++) {
+      while (true) {
+        // if we can't tessellate; then random polygon generator created a malformed shape
+        Polygon p = (Polygon) getShapeType().nextShape();
+        try {
+          Tessellator.tessellate(p);
+          polygons[i] = p;
+          break;
+        } catch (IllegalArgumentException e) {
+          continue;
+        }
+      }
+    }
+    return polygons;
+  }
+
+  @Override
+  protected Field[] createIndexableFields(String name, Object o) {
+    Polygon[] polygons = (Polygon[]) o;
+    List<Field> allFields = new ArrayList<>();
+    for (Polygon polygon : polygons) {
+      Field[] fields = LatLonShape.createIndexableFields(name, polygon);
+      for (Field field : fields) {
+        allFields.add(field);
+      }
+    }
+    return allFields.toArray(new Field[allFields.size()]);
+  }
+
+  @Override
+  protected Validator getValidator(QueryRelation relation) {
+    VALIDATOR.setRelation(relation);
+    POLYGONVALIDATOR.setRelation(relation);
+    return VALIDATOR;
+  }
+
+  protected class MultiPolygonValidator extends Validator {
+    @Override
+    public boolean testBBoxQuery(double minLat, double maxLat, double minLon, double maxLon, Object shape) {
+      Polygon[] polygons = (Polygon[])shape;
+      for (Polygon p : polygons) {
+        boolean b = POLYGONVALIDATOR.testBBoxQuery(minLat, maxLat, minLon, maxLon, p);
+        if (b == true && queryRelation == QueryRelation.INTERSECTS) {
+          return true;
+        } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
+          return false;
+        } else if (b == false && queryRelation == QueryRelation.WITHIN) {
+          return false;
+        }
+      }
+      return queryRelation != QueryRelation.INTERSECTS;
+    }
+
+    @Override
+    public boolean testLineQuery(Line2D query, Object shape) {
+      Polygon[] polygons = (Polygon[])shape;
+      for (Polygon p : polygons) {
+        boolean b = POLYGONVALIDATOR.testLineQuery(query, p);
+        if (b == true && queryRelation == QueryRelation.INTERSECTS) {
+          return true;
+        } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
+          return false;
+        } else if (b == false && queryRelation == QueryRelation.WITHIN) {
+          return false;
+        }
+      }
+      return queryRelation != QueryRelation.INTERSECTS;
+    }
+
+    @Override
+    public boolean testPolygonQuery(Polygon2D query, Object shape) {
+      Polygon[] polygons = (Polygon[])shape;
+      for (Polygon p : polygons) {
+        boolean b = POLYGONVALIDATOR.testPolygonQuery(query, p);
+        if (b == true && queryRelation == QueryRelation.INTERSECTS) {
+          return true;
+        } else if (b == false && queryRelation == QueryRelation.DISJOINT) {
+          return false;
+        } else if (b == false && queryRelation == QueryRelation.WITHIN) {
+          return false;
+        }
+      }
+      return queryRelation != QueryRelation.INTERSECTS;
+    }
+  }
+
+  @Slow
+  @Nightly
+  @Override
+  public void testRandomBig() throws Exception {
+    doTestRandom(10000);
+  }
+}
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java
index 6020617..d894aed 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java
@@ -72,7 +72,7 @@ public class TestLatLonPointShapeQueries extends BaseLatLonShapeTestCase {
     return VALIDATOR;
   }
 
-  protected class PointValidator extends Validator {
+  protected static class PointValidator extends Validator {
     @Override
     public boolean testBBoxQuery(double minLat, double maxLat, double minLon, double maxLon, Object shape) {
       Point p = (Point)shape;
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPolygonShapeQueries.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPolygonShapeQueries.java
index 9bc9b48..a3e060c 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPolygonShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPolygonShapeQueries.java
@@ -64,7 +64,7 @@ public class TestLatLonPolygonShapeQueries extends BaseLatLonShapeTestCase {
     return VALIDATOR;
   }
 
-  protected class PolygonValidator extends Validator {
+  protected static class PolygonValidator extends Validator {
     @Override
     public boolean testBBoxQuery(double minLat, double maxLat, double minLon, double maxLon, Object shape) {
       Polygon p = (Polygon)shape;