You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2018/08/09 19:27:12 UTC

[10/17] lucene-solr:jira/solr-12470: LUCENE-8447: Add DISJOINT and WITHIN support to LatLonShape queries

LUCENE-8447: Add DISJOINT and WITHIN support to LatLonShape queries


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/cbaedb47
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/cbaedb47
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/cbaedb47

Branch: refs/heads/jira/solr-12470
Commit: cbaedb470c8ddc7b21e8ff0b2729c6cf97fbd3d0
Parents: 9b418a4
Author: Nicholas Knize <nk...@gmail.com>
Authored: Mon Aug 6 16:31:21 2018 -0500
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Wed Aug 8 12:50:05 2018 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   2 +
 .../org/apache/lucene/document/LatLonShape.java |  13 +-
 .../document/LatLonShapeBoundingBoxQuery.java   | 420 +++++--------------
 .../document/LatLonShapePolygonQuery.java       | 301 ++-----------
 .../lucene/document/LatLonShapeQuery.java       | 394 +++++++++++++++++
 .../document/BaseLatLonShapeTestCase.java       |  46 +-
 .../document/TestLatLonLineShapeQueries.java    |  40 +-
 .../document/TestLatLonPointShapeQueries.java   |  20 +-
 .../document/TestLatLonPolygonShapeQueries.java |  36 +-
 .../apache/lucene/document/TestLatLonShape.java |   3 +-
 10 files changed, 659 insertions(+), 616 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cbaedb47/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 31c8988..f1e6ebb 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -225,6 +225,8 @@ Changes in Runtime Behavior:
 
 Improvements
 
+* LUCENE-8447: Add DISJOINT and WITHIN support to LatLonShape queries. (Nick Knize)
+
 * LUCENE-8440: Add support for indexing and searching Line and Point shapes using LatLonShape encoding (Nick Knize)
 
 * LUCENE-8435: Add new LatLonShapePolygonQuery for querying indexed LatLonShape fields by arbitrary polygons (Nick Knize)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cbaedb47/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShape.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShape.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShape.java
index 01a31ad..3ac171f9 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShape.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShape.java
@@ -131,12 +131,12 @@ public class LatLonShape {
    *  note: does not currently support dateline crossing boxes
    * todo split dateline crossing boxes into two queries like {@link LatLonPoint#newBoxQuery}
    **/
-  public static Query newBoxQuery(String field, double minLatitude, double maxLatitude, double minLongitude, double maxLongitude) {
-    return new LatLonShapeBoundingBoxQuery(field, minLatitude, maxLatitude, minLongitude, maxLongitude);
+  public static Query newBoxQuery(String field, QueryRelation queryRelation, double minLatitude, double maxLatitude, double minLongitude, double maxLongitude) {
+    return new LatLonShapeBoundingBoxQuery(field, queryRelation, minLatitude, maxLatitude, minLongitude, maxLongitude);
   }
 
-  public static Query newPolygonQuery(String field, Polygon... polygons) {
-    return new LatLonShapePolygonQuery(field, polygons);
+  public static Query newPolygonQuery(String field, QueryRelation queryRelation, Polygon... polygons) {
+    return new LatLonShapePolygonQuery(field, queryRelation, polygons);
   }
 
   /** polygons are decomposed into tessellated triangles using {@link org.apache.lucene.geo.Tessellator}
@@ -167,4 +167,9 @@ public class LatLonShape {
       NumericUtils.intToSortableBytes(cX, bytes, BYTES * 5);
     }
   }
+
+  /** Query Relation Types **/
+  public enum QueryRelation {
+    INTERSECTS, WITHIN, DISJOINT
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cbaedb47/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java
----------------------------------------------------------------------
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 9779210..e797043 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeBoundingBoxQuery.java
@@ -16,29 +16,11 @@
  */
 package org.apache.lucene.document;
 
-import java.io.IOException;
 import java.util.Arrays;
-import java.util.Objects;
 
 import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.geo.Tessellator;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.PointValues;
-import org.apache.lucene.index.PointValues.IntersectVisitor;
 import org.apache.lucene.index.PointValues.Relation;
-import org.apache.lucene.search.ConstantScoreScorer;
-import org.apache.lucene.search.ConstantScoreWeight;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ScoreMode;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.ScorerSupplier;
-import org.apache.lucene.search.Weight;
-import org.apache.lucene.util.BitSetIterator;
-import org.apache.lucene.util.DocIdSetBuilder;
-import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.FutureArrays;
 import org.apache.lucene.util.NumericUtils;
 
@@ -58,19 +40,18 @@ import static org.apache.lucene.geo.GeoUtils.orient;
  *
  *  @lucene.experimental
  **/
-class LatLonShapeBoundingBoxQuery extends Query {
-  final String field;
+final class LatLonShapeBoundingBoxQuery extends LatLonShapeQuery {
   final byte[] bbox;
   final int minX;
   final int maxX;
   final int minY;
   final int maxY;
 
-  public LatLonShapeBoundingBoxQuery(String field, double minLat, double maxLat, double minLon, double maxLon) {
+  public LatLonShapeBoundingBoxQuery(String field, LatLonShape.QueryRelation queryRelation, double minLat, double maxLat, double minLon, double maxLon) {
+    super(field, queryRelation);
     if (minLon > maxLon) {
       throw new IllegalArgumentException("dateline crossing bounding box queries are not supported for [" + field + "]");
     }
-    this.field = field;
     this.bbox = new byte[4 * LatLonPoint.BYTES];
     this.minX = encodeLongitudeCeil(minLon);
     this.maxX = encodeLongitude(maxLon);
@@ -83,301 +64,129 @@ class LatLonShapeBoundingBoxQuery extends Query {
   }
 
   @Override
-  public final Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
-    return new ConstantScoreWeight(this, boost) {
-
-      private boolean edgeIntersectsQuery(double ax, double ay, double bx, double by) {
-        // top
-        if (orient(ax, ay, bx, by, minX, maxY) * orient(ax, ay, bx, by, maxX, maxY) <= 0 &&
-            orient(minX, maxY, maxX, maxY, ax, ay) * orient(minX, maxY, maxX, maxY, bx, by) <= 0) {
-          return true;
-        }
-
-        // right
-        if (orient(ax, ay, bx, by, maxX, maxY) * orient(ax, ay, bx, by, maxX, minY) <= 0 &&
-            orient(maxX, maxY, maxX, minY, ax, ay) * orient(maxX, maxY, maxX, minY, bx, by) <= 0) {
-          return true;
-        }
-
-        // bottom
-        if (orient(ax, ay, bx, by, maxX, minY) * orient(ax, ay, bx, by, minX, minY) <= 0 &&
-            orient(maxX, minY, minX, minY, ax, ay) * orient(maxX, minY, minX, minY, bx, by) <= 0) {
-          return true;
-        }
-
-        // left
-        if (orient(ax, ay, bx, by, minX, minY) * orient(ax, ay, bx, by, minX, maxY) <= 0 &&
-            orient(minX, minY, minX, maxY, ax, ay) * orient(minX, minY, minX, maxY, bx, by) <= 0) {
-          return true;
-        }
-        return false;
-      }
-
-      private boolean queryContains(byte[] t, int point) {
-        final int yIdx = 2 * LatLonPoint.BYTES * point;
-        final int xIdx = yIdx + LatLonPoint.BYTES;
-
-        if (FutureArrays.compareUnsigned(t, yIdx, xIdx, bbox, 0, LatLonPoint.BYTES) < 0 ||                     //minY
-            FutureArrays.compareUnsigned(t, yIdx, xIdx, bbox, 2 * LatLonPoint.BYTES, 3 * LatLonPoint.BYTES) > 0 ||  //maxY
-            FutureArrays.compareUnsigned(t, xIdx, xIdx + LatLonPoint.BYTES, bbox, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES) < 0 || // minX
-            FutureArrays.compareUnsigned(t, xIdx, xIdx + LatLonPoint.BYTES, bbox, 3 * LatLonPoint.BYTES, bbox.length) > 0) {
-          return false;
-        }
-        return true;
-      }
-
-      private boolean queryIntersects(int ax, int ay, int bx, int by, int cx, int cy) {
-        // check each edge of the triangle against the query
-        if (edgeIntersectsQuery(ax, ay, bx, by) ||
-            edgeIntersectsQuery(bx, by, cx, cy) ||
-            edgeIntersectsQuery(cx, cy, ax, ay)) {
-          return true;
-        }
-        return false;
-      }
-
-      private boolean queryCrossesTriangle(byte[] t) {
-        // 1. query contains any triangle points
-        if (queryContains(t, 0) || queryContains(t, 1) || queryContains(t, 2)) {
-          return true;
-        }
-
-        int aY = NumericUtils.sortableBytesToInt(t, 0);
-        int aX = NumericUtils.sortableBytesToInt(t, LatLonPoint.BYTES);
-        int bY = NumericUtils.sortableBytesToInt(t, 2 * LatLonPoint.BYTES);
-        int bX = NumericUtils.sortableBytesToInt(t, 3 * LatLonPoint.BYTES);
-        int cY = NumericUtils.sortableBytesToInt(t, 4 * LatLonPoint.BYTES);
-        int cX = NumericUtils.sortableBytesToInt(t, 5 * LatLonPoint.BYTES);
-
-        int tMinX = StrictMath.min(StrictMath.min(aX, bX), cX);
-        int tMaxX = StrictMath.max(StrictMath.max(aX, bX), cX);
-        int tMinY = StrictMath.min(StrictMath.min(aY, bY), cY);
-        int tMaxY = StrictMath.max(StrictMath.max(aY, bY), cY);
-
-        // 2. check bounding boxes are disjoint
-        if (tMaxX < minX || tMinX > maxX || tMinY > maxY || tMaxY < minY) {
-          return false;
-        }
-
-        // 3. check triangle contains any query points
-        if (Tessellator.pointInTriangle(minX, minY, aX, aY, bX, bY, cX, cY)) {
-          return true;
-        } else if (Tessellator.pointInTriangle(maxX, minY, aX, aY, bX, bY, cX, cY)) {
-          return true;
-        } else if (Tessellator.pointInTriangle(maxX, maxY, aX, aY, bX, bY, cX, cY)) {
-          return true;
-        } else if (Tessellator.pointInTriangle(minX, maxY, aX, aY, bX, bY, cX, cY)) {
-          return true;
-        }
-
-
-        // 4. last ditch effort: check crossings
-        if (queryIntersects(aX, aY, bX, bY, cX, cY)) {
-          return true;
-        }
-        return false;
-      }
-
-      private Relation relateRangeToQuery(byte[] minTriangle, byte[] maxTriangle) {
-        // compute bounding box
-        int minXOfs = 0;
-        int minYOfs = 0;
-        int maxXOfs = 0;
-        int maxYOfs = 0;
-        for (int d = 1; d < 3; ++d) {
-          // check minX
-          int aOfs = (minXOfs * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
-          int bOfs = (d * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
-          if (FutureArrays.compareUnsigned(minTriangle, bOfs, bOfs + LatLonPoint.BYTES, minTriangle, aOfs, aOfs + LatLonPoint.BYTES) < 0) {
-            minXOfs = d;
-          }
-          // check maxX
-          aOfs = (maxXOfs * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
-          if (FutureArrays.compareUnsigned(maxTriangle, bOfs, bOfs + LatLonPoint.BYTES, maxTriangle, aOfs, aOfs + LatLonPoint.BYTES) > 0) {
-            maxXOfs = d;
-          }
-          // check minY
-          aOfs = minYOfs * 2 * LatLonPoint.BYTES;
-          bOfs = d * 2 * LatLonPoint.BYTES;
-          if (FutureArrays.compareUnsigned(minTriangle, bOfs, bOfs + LatLonPoint.BYTES, minTriangle, aOfs, aOfs + LatLonPoint.BYTES) < 0) {
-            minYOfs = d;
-          }
-          // check maxY
-          aOfs = maxYOfs * 2 * LatLonPoint.BYTES;
-          if (FutureArrays.compareUnsigned(maxTriangle, bOfs, bOfs + LatLonPoint.BYTES, maxTriangle, aOfs, aOfs + LatLonPoint.BYTES) > 0) {
-            maxYOfs = d;
-          }
-        }
-        minXOfs = (minXOfs * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
-        maxXOfs = (maxXOfs * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
-        minYOfs *= 2 * LatLonPoint.BYTES;
-        maxYOfs *= 2 * LatLonPoint.BYTES;
-
-        // check bounding box (DISJOINT)
-        if (FutureArrays.compareUnsigned(minTriangle, minXOfs, minXOfs + LatLonPoint.BYTES, bbox, 3 * LatLonPoint.BYTES, 4 * LatLonPoint.BYTES) > 0 ||
-            FutureArrays.compareUnsigned(maxTriangle, maxXOfs, maxXOfs + LatLonPoint.BYTES, bbox, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES) < 0 ||
-            FutureArrays.compareUnsigned(minTriangle, minYOfs, minYOfs + LatLonPoint.BYTES, bbox, 2 * LatLonPoint.BYTES, 3 * LatLonPoint.BYTES) > 0 ||
-            FutureArrays.compareUnsigned(maxTriangle, maxYOfs, maxYOfs + LatLonPoint.BYTES, bbox, 0, LatLonPoint.BYTES) < 0) {
-          return Relation.CELL_OUTSIDE_QUERY;
-        }
-
-        if (FutureArrays.compareUnsigned(minTriangle, minXOfs, minXOfs + LatLonPoint.BYTES, bbox, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES) > 0 &&
-            FutureArrays.compareUnsigned(maxTriangle, maxXOfs, maxXOfs + LatLonPoint.BYTES, bbox, 3 * LatLonPoint.BYTES, 4 * LatLonPoint.BYTES) < 0 &&
-            FutureArrays.compareUnsigned(minTriangle, minYOfs, minYOfs + LatLonPoint.BYTES, bbox, 0, LatLonPoint.BYTES) > 0 &&
-            FutureArrays.compareUnsigned(maxTriangle, maxYOfs, maxYOfs + LatLonPoint.BYTES, bbox, 2 * LatLonPoint.BYTES, 2 * LatLonPoint.BYTES) < 0) {
-          return Relation.CELL_INSIDE_QUERY;
-        }
-        return Relation.CELL_CROSSES_QUERY;
-      }
-
-      private IntersectVisitor getIntersectVisitor(DocIdSetBuilder result) {
-        return new IntersectVisitor() {
-
-          DocIdSetBuilder.BulkAdder adder;
-
-          @Override
-          public void grow(int count) {
-            adder = result.grow(count);
-          }
-
-          @Override
-          public void visit(int docID) throws IOException {
-            adder.add(docID);
-          }
-
-          @Override
-          public void visit(int docID, byte[] t) throws IOException {
-            if (queryCrossesTriangle(t)) {
-              adder.add(docID);
-            }
-          }
+  protected Relation relateRangeBBoxToQuery(int minXOffset, int minYOffset, byte[] minTriangle,
+                                            int maxXOffset, int maxYOffset, byte[] maxTriangle) {
+    // check bounding box (DISJOINT)
+    if (FutureArrays.compareUnsigned(minTriangle, minXOffset, minXOffset + LatLonPoint.BYTES, bbox, 3 * LatLonPoint.BYTES, 4 * LatLonPoint.BYTES) > 0 ||
+        FutureArrays.compareUnsigned(maxTriangle, maxXOffset, maxXOffset + LatLonPoint.BYTES, bbox, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES) < 0 ||
+        FutureArrays.compareUnsigned(minTriangle, minYOffset, minYOffset + LatLonPoint.BYTES, bbox, 2 * LatLonPoint.BYTES, 3 * LatLonPoint.BYTES) > 0 ||
+        FutureArrays.compareUnsigned(maxTriangle, maxYOffset, maxYOffset + LatLonPoint.BYTES, bbox, 0, LatLonPoint.BYTES) < 0) {
+      return Relation.CELL_OUTSIDE_QUERY;
+    }
 
-          @Override
-          public Relation compare(byte[] minTriangle, byte[] maxTriangle) {
-            return relateRangeToQuery(minTriangle, maxTriangle);
-          }
-        };
-      }
+    if (FutureArrays.compareUnsigned(minTriangle, minXOffset, minXOffset + LatLonPoint.BYTES, bbox, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES) > 0 &&
+        FutureArrays.compareUnsigned(maxTriangle, maxXOffset, maxXOffset + LatLonPoint.BYTES, bbox, 3 * LatLonPoint.BYTES, 4 * LatLonPoint.BYTES) < 0 &&
+        FutureArrays.compareUnsigned(minTriangle, minYOffset, minYOffset + LatLonPoint.BYTES, bbox, 0, LatLonPoint.BYTES) > 0 &&
+        FutureArrays.compareUnsigned(maxTriangle, maxYOffset, maxYOffset + LatLonPoint.BYTES, bbox, 2 * LatLonPoint.BYTES, 2 * LatLonPoint.BYTES) < 0) {
+      return Relation.CELL_INSIDE_QUERY;
+    }
+    return Relation.CELL_CROSSES_QUERY;
+  }
 
-      /**
-       * Create a visitor that clears documents that do NOT match the bounding box query.
-       */
-      private IntersectVisitor getInverseIntersectVisitor(FixedBitSet result, int[] cost) {
-        return new IntersectVisitor() {
+  /** returns true if the query matches the encoded triangle */
+  @Override
+  protected boolean queryMatches(byte[] t) {
+    if (queryRelation == LatLonShape.QueryRelation.WITHIN) {
+      return queryContains(t, 0) && queryContains(t, 1) && queryContains(t, 2);
+    }
+    return queryIntersects(t);
+  }
 
-          @Override
-          public void visit(int docID) {
-            result.clear(docID);
-            cost[0]--;
-          }
+  /** returns true if the query intersects the encoded triangle */
+  protected boolean queryIntersects(byte[] t) {
 
-          @Override
-          public void visit(int docID, byte[] packedTriangle) {
-            if (queryCrossesTriangle(packedTriangle) == false) {
-              result.clear(docID);
-              cost[0]--;
-            }
-          }
+    // 1. query contains any triangle points
+    if (queryContains(t, 0) || queryContains(t, 1) || queryContains(t, 2)) {
+      return true;
+    }
 
-          @Override
-          public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-            Relation r = relateRangeToQuery(minPackedValue, maxPackedValue);
-            if (r == Relation.CELL_OUTSIDE_QUERY) {
-              return Relation.CELL_INSIDE_QUERY;
-            } else if (r == Relation.CELL_INSIDE_QUERY) {
-              return Relation.CELL_OUTSIDE_QUERY;
-            }
-            return r;
-          }
-        };
-      }
+    int aY = NumericUtils.sortableBytesToInt(t, 0);
+    int aX = NumericUtils.sortableBytesToInt(t, LatLonPoint.BYTES);
+    int bY = NumericUtils.sortableBytesToInt(t, 2 * LatLonPoint.BYTES);
+    int bX = NumericUtils.sortableBytesToInt(t, 3 * LatLonPoint.BYTES);
+    int cY = NumericUtils.sortableBytesToInt(t, 4 * LatLonPoint.BYTES);
+    int cX = NumericUtils.sortableBytesToInt(t, 5 * LatLonPoint.BYTES);
+
+    int tMinX = StrictMath.min(StrictMath.min(aX, bX), cX);
+    int tMaxX = StrictMath.max(StrictMath.max(aX, bX), cX);
+    int tMinY = StrictMath.min(StrictMath.min(aY, bY), cY);
+    int tMaxY = StrictMath.max(StrictMath.max(aY, bY), cY);
+
+    // 2. check bounding boxes are disjoint
+    if (tMaxX < minX || tMinX > maxX || tMinY > maxY || tMaxY < minY) {
+      return false;
+    }
 
-      @Override
-      public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
-        LeafReader reader = context.reader();
-        PointValues values = reader.getPointValues(field);
-        if (values == null) {
-          return null;
-        }
+    // 3. check triangle contains any query points
+    if (Tessellator.pointInTriangle(minX, minY, aX, aY, bX, bY, cX, cY)) {
+      return true;
+    } else if (Tessellator.pointInTriangle(maxX, minY, aX, aY, bX, bY, cX, cY)) {
+      return true;
+    } else if (Tessellator.pointInTriangle(maxX, maxY, aX, aY, bX, bY, cX, cY)) {
+      return true;
+    } else if (Tessellator.pointInTriangle(minX, maxY, aX, aY, bX, bY, cX, cY)) {
+      return true;
+    }
 
-        boolean allDocsMatch = true;
-        if (values.getDocCount() != reader.maxDoc() ||
-            relateRangeToQuery(values.getMinPackedValue(), values.getMaxPackedValue()) != Relation.CELL_INSIDE_QUERY) {
-          allDocsMatch = false;
-        }
 
-        final Weight weight = this;
-        if (allDocsMatch) {
-          return new ScorerSupplier() {
-            @Override
-            public Scorer get(long leadCost) throws IOException {
-              return new ConstantScoreScorer(weight, score(),
-                  DocIdSetIterator.all(reader.maxDoc()));
-            }
+    // 4. last ditch effort: check crossings
+    if (queryIntersects(aX, aY, bX, bY, cX, cY)) {
+      return true;
+    }
+    return false;
+  }
 
-            @Override
-            public long cost() {
-              return reader.maxDoc();
-            }
-          };
-        } else {
-          return new ScorerSupplier() {
-            final DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
-            final IntersectVisitor visitor = getIntersectVisitor(result);
-            long cost = -1;
+  /** returns true if the edge (defined by (ax, ay) (bx, by)) intersects the query */
+  private boolean edgeIntersectsQuery(double ax, double ay, double bx, double by) {
+    // top
+    if (orient(ax, ay, bx, by, minX, maxY) * orient(ax, ay, bx, by, maxX, maxY) <= 0 &&
+        orient(minX, maxY, maxX, maxY, ax, ay) * orient(minX, maxY, maxX, maxY, bx, by) <= 0) {
+      return true;
+    }
 
-            @Override
-            public Scorer get(long leadCost) throws IOException {
-              if (values.getDocCount() == reader.maxDoc()
-                  && values.getDocCount() == values.size()
-                  && cost() > reader.maxDoc() / 2) {
-                // If all docs have exactly one value and the cost is greater
-                // than half the leaf size then maybe we can make things faster
-                // by computing the set of documents that do NOT match the query
-                final FixedBitSet result = new FixedBitSet(reader.maxDoc());
-                result.set(0, reader.maxDoc());
-                int[] cost = new int[]{reader.maxDoc()};
-                values.intersect(getInverseIntersectVisitor(result, cost));
-                final DocIdSetIterator iterator = new BitSetIterator(result, cost[0]);
-                return new ConstantScoreScorer(weight, score(), iterator);
-              }
+    // right
+    if (orient(ax, ay, bx, by, maxX, maxY) * orient(ax, ay, bx, by, maxX, minY) <= 0 &&
+        orient(maxX, maxY, maxX, minY, ax, ay) * orient(maxX, maxY, maxX, minY, bx, by) <= 0) {
+      return true;
+    }
 
-              values.intersect(visitor);
-              DocIdSetIterator iterator = result.build().iterator();
-              return new ConstantScoreScorer(weight, score(), iterator);
-            }
+    // bottom
+    if (orient(ax, ay, bx, by, maxX, minY) * orient(ax, ay, bx, by, minX, minY) <= 0 &&
+        orient(maxX, minY, minX, minY, ax, ay) * orient(maxX, minY, minX, minY, bx, by) <= 0) {
+      return true;
+    }
 
-            @Override
-            public long cost() {
-              if (cost == -1) {
-                // Computing the cost may be expensive, so only do it if necessary
-                cost = values.estimatePointCount(visitor);
-                assert cost >= 0;
-              }
-              return cost;
-            }
-          };
-        }
-      }
+    // left
+    if (orient(ax, ay, bx, by, minX, minY) * orient(ax, ay, bx, by, minX, maxY) <= 0 &&
+        orient(minX, minY, minX, maxY, ax, ay) * orient(minX, minY, minX, maxY, bx, by) <= 0) {
+      return true;
+    }
+    return false;
+  }
 
-      @Override
-      public Scorer scorer(LeafReaderContext context) throws IOException {
-        ScorerSupplier scorerSupplier = scorerSupplier(context);
-        if (scorerSupplier == null) {
-          return null;
-        }
-        return scorerSupplier.get(Long.MAX_VALUE);
-      }
+  /** returns true if the query contains the triangle vertex */
+  private boolean queryContains(byte[] t, int point) {
+    final int yIdx = 2 * LatLonPoint.BYTES * point;
+    final int xIdx = yIdx + LatLonPoint.BYTES;
 
-      @Override
-      public boolean isCacheable(LeafReaderContext ctx) {
-        return true;
-      }
-    };
+    if (FutureArrays.compareUnsigned(t, yIdx, xIdx, bbox, 0, LatLonPoint.BYTES) < 0 ||                     //minY
+        FutureArrays.compareUnsigned(t, yIdx, xIdx, bbox, 2 * LatLonPoint.BYTES, 3 * LatLonPoint.BYTES) > 0 ||  //maxY
+        FutureArrays.compareUnsigned(t, xIdx, xIdx + LatLonPoint.BYTES, bbox, LatLonPoint.BYTES, 2 * LatLonPoint.BYTES) < 0 || // minX
+        FutureArrays.compareUnsigned(t, xIdx, xIdx + LatLonPoint.BYTES, bbox, 3 * LatLonPoint.BYTES, bbox.length) > 0) {
+      return false;
+    }
+    return true;
   }
 
-  public String getField() {
-    return field;
+  /** returns true if the query intersects the provided triangle (in encoded space) */
+  private boolean queryIntersects(int ax, int ay, int bx, int by, int cx, int cy) {
+    // check each edge of the triangle against the query
+    if (edgeIntersectsQuery(ax, ay, bx, by) ||
+        edgeIntersectsQuery(bx, by, cx, cy) ||
+        edgeIntersectsQuery(cx, cy, ax, ay)) {
+      return true;
+    }
+    return false;
   }
 
   @Override
@@ -385,15 +194,14 @@ class LatLonShapeBoundingBoxQuery extends Query {
     return sameClassAs(o) && equalsTo(getClass().cast(o));
   }
 
-  private boolean equalsTo(LatLonShapeBoundingBoxQuery o) {
-    return Objects.equals(field, o.field) &&
-        Arrays.equals(bbox, o.bbox);
+  @Override
+  protected boolean equalsTo(Object o) {
+    return super.equalsTo(o) && Arrays.equals(bbox, ((LatLonShapeBoundingBoxQuery)o).bbox);
   }
 
   @Override
   public int hashCode() {
-    int hash = classHash();
-    hash = 31 * hash + field.hashCode();
+    int hash = super.hashCode();
     hash = 31 * hash + Arrays.hashCode(bbox);
     return hash;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cbaedb47/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapePolygonQuery.java
----------------------------------------------------------------------
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 be47971..755946e 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapePolygonQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapePolygonQuery.java
@@ -16,39 +16,13 @@
  */
 package org.apache.lucene.document;
 
-import java.io.IOException;
 import java.util.Arrays;
-import java.util.Objects;
 
+import org.apache.lucene.document.LatLonShape.QueryRelation;
 import org.apache.lucene.geo.GeoEncodingUtils;
 import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.geo.Polygon2D;
-import org.apache.lucene.geo.Rectangle;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.PointValues;
-import org.apache.lucene.index.PointValues.IntersectVisitor;
 import org.apache.lucene.index.PointValues.Relation;
-import org.apache.lucene.search.ConstantScoreScorer;
-import org.apache.lucene.search.ConstantScoreWeight;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ScoreMode;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.ScorerSupplier;
-import org.apache.lucene.search.Weight;
-import org.apache.lucene.util.BitSetIterator;
-import org.apache.lucene.util.DocIdSetBuilder;
-import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.FutureArrays;
-import org.apache.lucene.util.NumericUtils;
-
-import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitudeCeil;
-import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitudeCeil;
 
 /**
  * Finds all previously indexed shapes that intersect the specified arbitrary.
@@ -58,15 +32,15 @@ import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitudeCeil;
  *
  *  @lucene.experimental
  **/
-public class LatLonShapePolygonQuery extends Query {
-  final String field;
+final class LatLonShapePolygonQuery extends LatLonShapeQuery {
   final Polygon[] polygons;
+  final private Polygon2D poly2D;
 
-
-  public LatLonShapePolygonQuery(String field, Polygon... polygons) {
-    if (field == null) {
-      throw new IllegalArgumentException("field must not be null");
-    }
+  /**
+   * Creates a query that matches all indexed shapes to the provided polygons
+   */
+  public LatLonShapePolygonQuery(String field, QueryRelation queryRelation, Polygon... polygons) {
+    super(field, queryRelation);
     if (polygons == null) {
       throw new IllegalArgumentException("polygons must not be null");
     }
@@ -76,236 +50,40 @@ public class LatLonShapePolygonQuery extends Query {
     for (int i = 0; i < polygons.length; i++) {
       if (polygons[i] == null) {
         throw new IllegalArgumentException("polygon[" + i + "] must not be null");
+      } else if (polygons[i].minLon > polygons[i].maxLon) {
+        throw new IllegalArgumentException("LatLonShapePolygonQuery does not currently support querying across dateline.");
       }
     }
-    this.field = field;
     this.polygons = polygons.clone();
+    this.poly2D = Polygon2D.create(polygons);
   }
 
   @Override
-  public final Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
-    final Rectangle box = Rectangle.fromPolygon(polygons);
-    final byte minLat[] = new byte[Integer.BYTES];
-    final byte maxLat[] = new byte[Integer.BYTES];
-    final byte minLon[] = new byte[Integer.BYTES];
-    final byte maxLon[] = new byte[Integer.BYTES];
-    NumericUtils.intToSortableBytes(encodeLatitudeCeil(box.minLat), minLat, 0);
-    NumericUtils.intToSortableBytes(encodeLatitude(box.maxLat), maxLat, 0);
-    NumericUtils.intToSortableBytes(encodeLongitudeCeil(box.minLon), minLon, 0);
-    NumericUtils.intToSortableBytes(encodeLongitude(box.maxLon), maxLon, 0);
-
-    final Polygon2D polygon = Polygon2D.create(polygons);
-
-    return new ConstantScoreWeight(this, boost) {
-
-      private Relation relateRangeToQuery(byte[] minTriangle, byte[] maxTriangle) {
-        // compute bounding box
-        int minXOfs = 0;
-        int minYOfs = 0;
-        int maxXOfs = 0;
-        int maxYOfs = 0;
-        for (int d = 1; d < 3; ++d) {
-          // check minX
-          int aOfs = (minXOfs * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
-          int bOfs = (d * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
-          if (FutureArrays.compareUnsigned(minTriangle, bOfs, bOfs + LatLonPoint.BYTES, minTriangle, aOfs, aOfs + LatLonPoint.BYTES) < 0) {
-            minXOfs = d;
-          }
-          // check maxX
-          aOfs = (maxXOfs * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
-          if (FutureArrays.compareUnsigned(maxTriangle, bOfs, bOfs + LatLonPoint.BYTES, maxTriangle, aOfs, aOfs + LatLonPoint.BYTES) > 0) {
-            maxXOfs = d;
-          }
-          // check minY
-          aOfs = minYOfs * 2 * LatLonPoint.BYTES;
-          bOfs = d * 2 * LatLonPoint.BYTES;
-          if (FutureArrays.compareUnsigned(minTriangle, bOfs, bOfs + LatLonPoint.BYTES, minTriangle, aOfs, aOfs + LatLonPoint.BYTES) < 0) {
-            minYOfs = d;
-          }
-          // check maxY
-          aOfs = maxYOfs * 2 * LatLonPoint.BYTES;
-          if (FutureArrays.compareUnsigned(maxTriangle, bOfs, bOfs + LatLonPoint.BYTES, maxTriangle, aOfs, aOfs + LatLonPoint.BYTES) > 0) {
-            maxYOfs = d;
-          }
-        }
-        minXOfs = (minXOfs * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
-        maxXOfs = (maxXOfs * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
-        minYOfs *= 2 * LatLonPoint.BYTES;
-        maxYOfs *= 2 * LatLonPoint.BYTES;
-
-        double minLat = GeoEncodingUtils.decodeLatitude(minTriangle, minYOfs);
-        double minLon = GeoEncodingUtils.decodeLongitude(minTriangle, minXOfs);
-        double maxLat = GeoEncodingUtils.decodeLatitude(maxTriangle, maxYOfs);
-        double maxLon = GeoEncodingUtils.decodeLongitude(maxTriangle, maxXOfs);
-
-        // check internal node against query
-        return polygon.relate(minLat, maxLat, minLon, maxLon);
-      }
-
-      private boolean queryCrossesTriangle(byte[] t) {
-        double ay = GeoEncodingUtils.decodeLatitude(t, 0);
-        double ax = GeoEncodingUtils.decodeLongitude(t, LatLonPoint.BYTES);
-        double by = GeoEncodingUtils.decodeLatitude(t, 2 * LatLonPoint.BYTES);
-        double bx = GeoEncodingUtils.decodeLongitude(t, 3 * LatLonPoint.BYTES);
-        double cy = GeoEncodingUtils.decodeLatitude(t, 4 * LatLonPoint.BYTES);
-        double cx = GeoEncodingUtils.decodeLongitude(t, 5 * LatLonPoint.BYTES);
-        return polygon.relateTriangle(ax, ay, bx, by, cx, cy) != Relation.CELL_OUTSIDE_QUERY;
-      }
-
-      private IntersectVisitor getIntersectVisitor(DocIdSetBuilder result) {
-        return new IntersectVisitor() {
-
-          DocIdSetBuilder.BulkAdder adder;
-
-          @Override
-          public void grow(int count) {
-            adder = result.grow(count);
-          }
-
-          @Override
-          public void visit(int docID) throws IOException {
-            adder.add(docID);
-          }
-
-          @Override
-          public void visit(int docID, byte[] t) throws IOException {
-            if (queryCrossesTriangle(t)) {
-              adder.add(docID);
-            }
-          }
-
-          @Override
-          public Relation compare(byte[] minTriangle, byte[] maxTriangle) {
-            return relateRangeToQuery(minTriangle, maxTriangle);
-          }
-        };
-      }
-
-      /**
-       * Create a visitor that clears documents that do NOT match the polygon query.
-       */
-      private IntersectVisitor getInverseIntersectVisitor(FixedBitSet result, int[] cost) {
-        return new IntersectVisitor() {
-
-          @Override
-          public void visit(int docID) {
-            result.clear(docID);
-            cost[0]--;
-          }
-
-          @Override
-          public void visit(int docID, byte[] packedTriangle) {
-            if (queryCrossesTriangle(packedTriangle) == false) {
-              result.clear(docID);
-              cost[0]--;
-            }
-          }
-
-          @Override
-          public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-            Relation r = relateRangeToQuery(minPackedValue, maxPackedValue);
-            if (r == Relation.CELL_OUTSIDE_QUERY) {
-              return Relation.CELL_INSIDE_QUERY;
-            } else if (r == Relation.CELL_INSIDE_QUERY) {
-              return Relation.CELL_OUTSIDE_QUERY;
-            }
-            return r;
-          }
-        };
-      }
-
-      @Override
-      public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
-        LeafReader reader = context.reader();
-        PointValues values = reader.getPointValues(field);
-        if (values == null) {
-          // No docs in this segment had any points fields
-          return null;
-        }
-        FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field);
-        if (fieldInfo == null) {
-          // No docs in this segment indexed this field at all
-          return null;
-        }
-
-        boolean allDocsMatch = true;
-        if (values.getDocCount() != reader.maxDoc() ||
-            relateRangeToQuery(values.getMinPackedValue(), values.getMaxPackedValue()) != Relation.CELL_INSIDE_QUERY) {
-          allDocsMatch = false;
-        }
-
-        final Weight weight = this;
-        if (allDocsMatch) {
-          return new ScorerSupplier() {
-            @Override
-            public Scorer get(long leadCost) throws IOException {
-              return new ConstantScoreScorer(weight, score(),
-                  DocIdSetIterator.all(reader.maxDoc()));
-            }
-
-            @Override
-            public long cost() {
-              return reader.maxDoc();
-            }
-          };
-        } else {
-          return new ScorerSupplier() {
-            final DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
-            final IntersectVisitor visitor = getIntersectVisitor(result);
-            long cost = -1;
-
-            @Override
-            public Scorer get(long leadCost) throws IOException {
-              if (values.getDocCount() == reader.maxDoc()
-                  && values.getDocCount() == values.size()
-                  && cost() > reader.maxDoc() / 2) {
-                // If all docs have exactly one value and the cost is greater
-                // than half the leaf size then maybe we can make things faster
-                // by computing the set of documents that do NOT match the query
-                final FixedBitSet result = new FixedBitSet(reader.maxDoc());
-                result.set(0, reader.maxDoc());
-                int[] cost = new int[]{reader.maxDoc()};
-                values.intersect(getInverseIntersectVisitor(result, cost));
-                final DocIdSetIterator iterator = new BitSetIterator(result, cost[0]);
-                return new ConstantScoreScorer(weight, score(), iterator);
-              }
-
-              values.intersect(visitor);
-              DocIdSetIterator iterator = result.build().iterator();
-              return new ConstantScoreScorer(weight, score(), iterator);
-            }
-
-            @Override
-            public long cost() {
-              if (cost == -1) {
-                // Computing the cost may be expensive, so only do it if necessary
-                cost = values.estimatePointCount(visitor);
-                assert cost >= 0;
-              }
-              return cost;
-            }
-          };
-        }
-      }
-
-      @Override
-      public Scorer scorer(LeafReaderContext context) throws IOException {
-        ScorerSupplier scorerSupplier = scorerSupplier(context);
-        if (scorerSupplier == null) {
-          return null;
-        }
-        return scorerSupplier.get(Long.MAX_VALUE);
-      }
-
-      @Override
-      public boolean isCacheable(LeafReaderContext ctx) {
-        return true;
-      }
-    };
+  protected Relation relateRangeBBoxToQuery(int minXOffset, int minYOffset, byte[] minTriangle,
+                                            int maxXOffset, int maxYOffset, byte[] maxTriangle) {
+    double minLat = GeoEncodingUtils.decodeLatitude(minTriangle, minYOffset);
+    double minLon = GeoEncodingUtils.decodeLongitude(minTriangle, minXOffset);
+    double maxLat = GeoEncodingUtils.decodeLatitude(maxTriangle, maxYOffset);
+    double maxLon = GeoEncodingUtils.decodeLongitude(maxTriangle, maxXOffset);
+
+    // check internal node against query
+    return poly2D.relate(minLat, maxLat, minLon, maxLon);
   }
 
-  public String getField() {
-    return field;
+  @Override
+  protected boolean queryMatches(byte[] triangle) {
+    double ay = GeoEncodingUtils.decodeLatitude(triangle, 0);
+    double ax = GeoEncodingUtils.decodeLongitude(triangle, LatLonPoint.BYTES);
+    double by = GeoEncodingUtils.decodeLatitude(triangle, 2 * LatLonPoint.BYTES);
+    double bx = GeoEncodingUtils.decodeLongitude(triangle, 3 * LatLonPoint.BYTES);
+    double cy = GeoEncodingUtils.decodeLatitude(triangle, 4 * LatLonPoint.BYTES);
+    double cx = GeoEncodingUtils.decodeLongitude(triangle, 5 * LatLonPoint.BYTES);
+
+    if (queryRelation == QueryRelation.WITHIN) {
+      return poly2D.relateTriangle(ax, ay, bx, by, cx, cy) == Relation.CELL_INSIDE_QUERY;
+    }
+    // INTERSECTS
+    return poly2D.relateTriangle(ax, ay, bx, by, cx, cy) != Relation.CELL_OUTSIDE_QUERY;
   }
 
   @Override
@@ -323,18 +101,13 @@ public class LatLonShapePolygonQuery extends Query {
   }
 
   @Override
-  public boolean equals(Object o) {
-    return sameClassAs(o) && equalsTo(getClass().cast(o));
-  }
-
-  private boolean equalsTo(LatLonShapePolygonQuery o) {
-    return Objects.equals(field, o.field) && Arrays.equals(polygons, o.polygons);
+  protected boolean equalsTo(Object o) {
+    return super.equalsTo(o) && Arrays.equals(polygons, ((LatLonShapePolygonQuery)o).polygons);
   }
 
   @Override
   public int hashCode() {
-    int hash = classHash();
-    hash = 31 * hash + field.hashCode();
+    int hash = super.hashCode();
     hash = 31 * hash + Arrays.hashCode(polygons);
     return hash;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cbaedb47/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeQuery.java
new file mode 100644
index 0000000..c678941
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeQuery.java
@@ -0,0 +1,394 @@
+/*
+ * 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.io.IOException;
+import java.util.Objects;
+
+import org.apache.lucene.document.LatLonShape.QueryRelation;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.ScorerSupplier;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.BitSetIterator;
+import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.FutureArrays;
+
+/**
+ * Base LatLonShape Query class providing common query logic for
+ * {@link LatLonShapeBoundingBoxQuery} and {@link LatLonShapePolygonQuery}
+ *
+ * Note: this class implements the majority of the INTERSECTS, WITHIN, DISJOINT relation logic
+ *
+ * @lucene.experimental
+ **/
+abstract class LatLonShapeQuery extends Query {
+  /** field name */
+  final String field;
+  /** query relation
+   * disjoint: {@code CELL_OUTSIDE_QUERY}
+   * intersects: {@code CELL_CROSSES_QUERY},
+   * within: {@code CELL_WITHIN_QUERY} */
+  final LatLonShape.QueryRelation queryRelation;
+
+  protected LatLonShapeQuery(String field, final QueryRelation queryType) {
+    if (field == null) {
+      throw new IllegalArgumentException("field must not be null");
+    }
+    this.field = field;
+    this.queryRelation = queryType;
+  }
+
+  /**
+   *   relates an internal node (bounding box of a range of triangles) to the target query
+   *   Note: logic is specific to query type
+   *   see {@link LatLonShapeBoundingBoxQuery#relateRangeToQuery} and {@link LatLonShapePolygonQuery#relateRangeToQuery}
+   */
+  protected abstract Relation relateRangeBBoxToQuery(int minXOffset, int minYOffset, byte[] minTriangle,
+                                                     int maxXOffset, int maxYOffset, byte[] maxTriangle);
+
+  /** returns true if the provided triangle matches the query */
+  protected abstract boolean queryMatches(byte[] triangle);
+
+  /** relates a range of triangles (internal node) to the query */
+  protected Relation relateRangeToQuery(byte[] minTriangle, byte[] maxTriangle) {
+    // compute bounding box of internal node
+    int minXOfs = 0;
+    int minYOfs = 0;
+    int maxXOfs = 0;
+    int maxYOfs = 0;
+    for (int d = 1; d < 3; ++d) {
+      // check minX
+      int aOfs = (minXOfs * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
+      int bOfs = (d * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
+      if (FutureArrays.compareUnsigned(minTriangle, bOfs, bOfs + LatLonPoint.BYTES, minTriangle, aOfs, aOfs + LatLonPoint.BYTES) < 0) {
+        minXOfs = d;
+      }
+      // check maxX
+      aOfs = (maxXOfs * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
+      if (FutureArrays.compareUnsigned(maxTriangle, bOfs, bOfs + LatLonPoint.BYTES, maxTriangle, aOfs, aOfs + LatLonPoint.BYTES) > 0) {
+        maxXOfs = d;
+      }
+      // check minY
+      aOfs = minYOfs * 2 * LatLonPoint.BYTES;
+      bOfs = d * 2 * LatLonPoint.BYTES;
+      if (FutureArrays.compareUnsigned(minTriangle, bOfs, bOfs + LatLonPoint.BYTES, minTriangle, aOfs, aOfs + LatLonPoint.BYTES) < 0) {
+        minYOfs = d;
+      }
+      // check maxY
+      aOfs = maxYOfs * 2 * LatLonPoint.BYTES;
+      if (FutureArrays.compareUnsigned(maxTriangle, bOfs, bOfs + LatLonPoint.BYTES, maxTriangle, aOfs, aOfs + LatLonPoint.BYTES) > 0) {
+        maxYOfs = d;
+      }
+    }
+    minXOfs = (minXOfs * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
+    maxXOfs = (maxXOfs * 2 * LatLonPoint.BYTES) + LatLonPoint.BYTES;
+    minYOfs *= 2 * LatLonPoint.BYTES;
+    maxYOfs *= 2 * LatLonPoint.BYTES;
+
+    Relation r = relateRangeBBoxToQuery(minXOfs, minYOfs, minTriangle, maxXOfs, maxYOfs, maxTriangle);
+
+    if (queryRelation == QueryRelation.DISJOINT) {
+      return transposeRelation(r);
+    }
+    return r;
+  }
+
+  @Override
+  public final Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+
+    return new ConstantScoreWeight(this, boost) {
+
+      /** create a visitor that adds documents that match the query using a sparse bitset. (Used by INTERSECT) */
+      protected IntersectVisitor getSparseIntersectVisitor(DocIdSetBuilder result) {
+        return new IntersectVisitor() {
+          DocIdSetBuilder.BulkAdder adder;
+
+          @Override
+          public void grow(int count) {
+            adder = result.grow(count);
+          }
+
+          @Override
+          public void visit(int docID) throws IOException {
+            adder.add(docID);
+          }
+
+          @Override
+          public void visit(int docID, byte[] t) throws IOException {
+            if (queryMatches(t)) {
+              adder.add(docID);
+            }
+          }
+
+          @Override
+          public Relation compare(byte[] minTriangle, byte[] maxTriangle) {
+            return relateRangeToQuery(minTriangle, maxTriangle);
+          }
+        };
+      }
+
+      /** 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) {
+        return new IntersectVisitor() {
+
+          @Override
+          public void visit(int docID) throws IOException {
+            if (queryRelation == QueryRelation.DISJOINT) {
+              // if DISJOINT query set the doc in the disjoint bitset
+              disjoint.set(docID);
+            } else {
+              // for INTERSECT, and WITHIN queries we set the intersect bitset
+              intersect.set(docID);
+            }
+          }
+
+          @Override
+          public void visit(int docID, byte[] t) throws IOException {
+            if (queryMatches(t)) {
+              intersect.set(docID);
+            } else {
+              disjoint.set(docID);
+            }
+          }
+
+          @Override
+          public Relation compare(byte[] minTriangle, byte[] maxTriangle) {
+            return relateRangeToQuery(minTriangle, maxTriangle);
+          }
+        };
+      }
+
+      /** get a scorer supplier for INTERSECT queries */
+      protected ScorerSupplier getIntersectScorerSupplier(LeafReader reader, PointValues values, Weight weight) throws IOException {
+        DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
+        IntersectVisitor visitor = getSparseIntersectVisitor(result);
+        return new RelationScorerSupplier(values, visitor) {
+          @Override
+          public Scorer get(long leadCost) throws IOException {
+            return getIntersectsScorer(LatLonShapeQuery.this, reader, weight, result, score());
+          }
+        };
+      }
+
+      /** get a scorer supplier for all other queries (DISJOINT, WITHIN) */
+      protected ScorerSupplier getScorerSupplier(LeafReader reader, PointValues values, Weight weight) throws IOException {
+        if (queryRelation == QueryRelation.INTERSECTS) {
+          return getIntersectScorerSupplier(reader, values, weight);
+        }
+
+        FixedBitSet intersect = new FixedBitSet(reader.maxDoc());
+        FixedBitSet disjoint = new FixedBitSet(reader.maxDoc());
+        IntersectVisitor visitor = getDenseIntersectVisitor(intersect, disjoint);
+        return new RelationScorerSupplier(values, visitor) {
+          @Override
+          public Scorer get(long leadCost) throws IOException {
+            return getScorer(LatLonShapeQuery.this, weight, intersect, disjoint, score());
+          }
+        };
+      }
+
+      @Override
+      public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
+        LeafReader reader = context.reader();
+        PointValues values = reader.getPointValues(field);
+        if (values == null) {
+          // No docs in this segment had any points fields
+          return null;
+        }
+        FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field);
+        if (fieldInfo == null) {
+          // No docs in this segment indexed this field at all
+          return null;
+        }
+
+        boolean allDocsMatch = true;
+        if (values.getDocCount() != reader.maxDoc() ||
+            relateRangeToQuery(values.getMinPackedValue(), values.getMaxPackedValue()) != Relation.CELL_INSIDE_QUERY) {
+          allDocsMatch = false;
+        }
+
+        final Weight weight = this;
+        if (allDocsMatch) {
+          return new ScorerSupplier() {
+            @Override
+            public Scorer get(long leadCost) throws IOException {
+              return new ConstantScoreScorer(weight, score(),
+                  DocIdSetIterator.all(reader.maxDoc()));
+            }
+
+            @Override
+            public long cost() {
+              return reader.maxDoc();
+            }
+          };
+        } else {
+          return getScorerSupplier(reader, values, weight);
+        }
+      }
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        ScorerSupplier scorerSupplier = scorerSupplier(context);
+        if (scorerSupplier == null) {
+          return null;
+        }
+        return scorerSupplier.get(Long.MAX_VALUE);
+      }
+
+      @Override
+      public boolean isCacheable(LeafReaderContext ctx) {
+        return true;
+      }
+    };
+  }
+
+  /** returns the field name */
+  public String getField() {
+    return field;
+  }
+
+  /** returns the query relation */
+  public QueryRelation getQueryRelation() {
+    return queryRelation;
+  }
+
+  @Override
+  public int hashCode() {
+    int hash = classHash();
+    hash = 31 * hash + field.hashCode();
+    hash = 31 * hash + queryRelation.hashCode();
+    return hash;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    return sameClassAs(o) && equalsTo(o);
+  }
+
+  protected boolean equalsTo(Object o) {
+    return Objects.equals(field, ((LatLonShapeQuery)o).field) && this.queryRelation == ((LatLonShapeQuery)o).queryRelation;
+  }
+
+  /** transpose the relation; INSIDE becomes OUTSIDE, OUTSIDE becomes INSIDE, CROSSES remains unchanged */
+  private static Relation transposeRelation(Relation r) {
+    if (r == Relation.CELL_INSIDE_QUERY) {
+      return Relation.CELL_OUTSIDE_QUERY;
+    } else if (r == Relation.CELL_OUTSIDE_QUERY) {
+      return Relation.CELL_INSIDE_QUERY;
+    }
+    return Relation.CELL_CROSSES_QUERY;
+  }
+
+  /** utility class for implementing constant score logic specifig to INTERSECT, WITHIN, and DISJOINT */
+  protected static abstract class RelationScorerSupplier extends ScorerSupplier {
+    PointValues values;
+    IntersectVisitor visitor;
+    long cost = -1;
+
+    RelationScorerSupplier(PointValues values, IntersectVisitor visitor) {
+      this.values = values;
+      this.visitor = visitor;
+    }
+
+    /** create a visitor that clears documents that do NOT match the polygon query; used with INTERSECTS */
+    private IntersectVisitor getInverseIntersectVisitor(LatLonShapeQuery query, FixedBitSet result, int[] cost) {
+      return new IntersectVisitor() {
+
+        @Override
+        public void visit(int docID) {
+          result.clear(docID);
+          cost[0]--;
+        }
+
+        @Override
+        public void visit(int docID, byte[] packedTriangle) {
+          if (query.queryMatches(packedTriangle) == false) {
+            result.clear(docID);
+            cost[0]--;
+          }
+        }
+
+        @Override
+        public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+          return transposeRelation(query.relateRangeToQuery(minPackedValue, maxPackedValue));
+        }
+      };
+    }
+
+    /** returns a Scorer for INTERSECT queries that uses a sparse bitset */
+    protected Scorer getIntersectsScorer(LatLonShapeQuery query, LeafReader reader, Weight weight,
+                                         DocIdSetBuilder docIdSetBuilder, final float boost) throws IOException {
+      if (values.getDocCount() == reader.maxDoc()
+          && values.getDocCount() == values.size()
+          && cost() > reader.maxDoc() / 2) {
+        // If all docs have exactly one value and the cost is greater
+        // than half the leaf size then maybe we can make things faster
+        // by computing the set of documents that do NOT match the query
+        final FixedBitSet result = new FixedBitSet(reader.maxDoc());
+        result.set(0, reader.maxDoc());
+        int[] cost = new int[]{reader.maxDoc()};
+        values.intersect(getInverseIntersectVisitor(query, result, cost));
+        final DocIdSetIterator iterator = new BitSetIterator(result, cost[0]);
+        return new ConstantScoreScorer(weight, boost, iterator);
+      }
+
+      values.intersect(visitor);
+      DocIdSetIterator iterator = docIdSetBuilder.build().iterator();
+      return new ConstantScoreScorer(weight, boost, iterator);
+    }
+
+    /** returns a Scorer for all other (non INTERSECT) queries */
+    protected Scorer getScorer(LatLonShapeQuery query, Weight weight,
+                               FixedBitSet intersect, FixedBitSet disjoint, final float boost) throws IOException {
+      values.intersect(visitor);
+      DocIdSetIterator iterator;
+      if (query.queryRelation == QueryRelation.DISJOINT) {
+        disjoint.andNot(intersect);
+        iterator = new BitSetIterator(disjoint, cost());
+      } else if (query.queryRelation == QueryRelation.WITHIN) {
+        intersect.andNot(disjoint);
+        iterator = new BitSetIterator(intersect, cost());
+      } else {
+        iterator = new BitSetIterator(intersect, cost());
+      }
+      return new ConstantScoreScorer(weight, boost, iterator);
+    }
+
+    @Override
+    public long cost() {
+      if (cost == -1) {
+        // Computing the cost may be expensive, so only do it if necessary
+        cost = values.estimatePointCount(visitor);
+        assert cost >= 0;
+      }
+      return cost;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cbaedb47/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java
----------------------------------------------------------------------
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 a7560ee..191e2cb 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/BaseLatLonShapeTestCase.java
@@ -22,6 +22,7 @@ import java.util.HashSet;
 import java.util.Set;
 
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import org.apache.lucene.document.LatLonShape.QueryRelation;
 import org.apache.lucene.geo.GeoTestUtil;
 import org.apache.lucene.geo.Line;
 import org.apache.lucene.geo.Polygon;
@@ -94,6 +95,16 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
     return new Polygon(lats, lons);
   }
 
+  protected Line quantizeLine(Line line) {
+    double[] lats = new double[line.numPoints()];
+    double[] lons = new double[line.numPoints()];
+    for (int i = 0; i < lats.length; ++i) {
+      lats[i] = quantizeLat(line.getLat(i));
+      lons[i] = quantizeLon(line.getLon(i));
+    }
+    return new Line(lats, lons);
+  }
+
   protected abstract Field[] createIndexableFields(String field, Object shape);
 
   private void addShapeToDoc(String field, Document doc, Object shape) {
@@ -103,12 +114,12 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
     }
   }
 
-  protected Query newRectQuery(String field, double minLat, double maxLat, double minLon, double maxLon) {
-    return LatLonShape.newBoxQuery(field, minLat, maxLat, minLon, maxLon);
+  protected Query newRectQuery(String field, QueryRelation queryRelation, double minLat, double maxLat, double minLon, double maxLon) {
+    return LatLonShape.newBoxQuery(field, queryRelation, minLat, maxLat, minLon, maxLon);
   }
 
-  protected Query newPolygonQuery(String field, Polygon... polygons) {
-    return LatLonShape.newPolygonQuery(field, polygons);
+  protected Query newPolygonQuery(String field, QueryRelation queryRelation, Polygon... polygons) {
+    return LatLonShape.newPolygonQuery(field, queryRelation, polygons);
   }
 
   // A particularly tricky adversary for BKD tree:
@@ -240,7 +251,8 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
           break;
         }
       }
-      Query query = newRectQuery(FIELD_NAME, rect.minLat, rect.maxLat, rect.minLon, rect.maxLon);
+      QueryRelation queryRelation = RandomPicks.randomFrom(random(), QueryRelation.values());
+      Query query = newRectQuery(FIELD_NAME, queryRelation, rect.minLat, rect.maxLat, rect.minLon, rect.maxLon);
 
       if (VERBOSE) {
         System.out.println("  query=" + query);
@@ -280,7 +292,7 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
           expected = false;
         } else {
           // check quantized poly against quantized query
-          expected = getValidator().testBBoxQuery(quantizeLatCeil(rect.minLat), quantizeLat(rect.maxLat),
+          expected = getValidator(queryRelation).testBBoxQuery(quantizeLatCeil(rect.minLat), quantizeLat(rect.maxLat),
               quantizeLonCeil(rect.minLon), quantizeLon(rect.maxLon), shapes[id]);
         }
 
@@ -292,10 +304,11 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
           } else {
             b.append("FAIL: id=" + id + " should not match but did\n");
           }
+          b.append("  relation=" + queryRelation + "\n");
           b.append("  query=" + query + " docID=" + docID + "\n");
           b.append("  shape=" + shapes[id] + "\n");
           b.append("  deleted?=" + (liveDocs != null && liveDocs.get(docID) == false));
-          b.append("  rect=Rectangle(" + quantizeLatCeil(rect.minLat) + " TO " + quantizeLat(rect.maxLat) + " lon=" + quantizeLonCeil(rect.minLon) + " TO " + quantizeLon(rect.maxLon) + ")");
+          b.append("  rect=Rectangle(" + quantizeLatCeil(rect.minLat) + " TO " + quantizeLat(rect.maxLat) + " lon=" + quantizeLonCeil(rect.minLon) + " TO " + quantizeLon(rect.maxLon) + ")\n");
           if (true) {
             fail("wrong hit (first of possibly more):\n\n" + b);
           } else {
@@ -326,7 +339,8 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
       // Polygon
       Polygon queryPolygon = GeoTestUtil.nextPolygon();
       Polygon2D queryPoly2D = Polygon2D.create(queryPolygon);
-      Query query = newPolygonQuery(FIELD_NAME, queryPolygon);
+      QueryRelation queryRelation = RandomPicks.randomFrom(random(), QueryRelation.values());
+      Query query = newPolygonQuery(FIELD_NAME, queryRelation, queryPolygon);
 
       if (VERBOSE) {
         System.out.println("  query=" + query);
@@ -365,7 +379,7 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
         } else if (shapes[id] == null) {
           expected = false;
         } else {
-          expected = getValidator().testPolygonQuery(queryPoly2D, shapes[id]);
+          expected = getValidator(queryRelation).testPolygonQuery(queryPoly2D, shapes[id]);
         }
 
         if (hits.get(docID) != expected) {
@@ -376,6 +390,7 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
           } else {
             b.append("FAIL: id=" + id + " should not match but did\n");
           }
+          b.append("  relation=" + queryRelation + "\n");
           b.append("  query=" + query + " docID=" + docID + "\n");
           b.append("  shape=" + shapes[id] + "\n");
           b.append("  deleted?=" + (liveDocs != null && liveDocs.get(docID) == false));
@@ -394,7 +409,7 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
     }
   }
 
-  protected abstract Validator getValidator();
+  protected abstract Validator getValidator(QueryRelation relation);
 
   /** internal point class for testing point shapes */
   protected static class Point {
@@ -466,8 +481,13 @@ public abstract class BaseLatLonShapeTestCase extends LuceneTestCase {
     }
   }
 
-  protected interface Validator {
-    boolean testBBoxQuery(double minLat, double maxLat, double minLon, double maxLon, Object shape);
-    boolean testPolygonQuery(Polygon2D poly2d, Object shape);
+  protected 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 testPolygonQuery(Polygon2D poly2d, Object shape);
+
+    public void setRelation(QueryRelation relation) {
+      this.queryRelation = relation;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cbaedb47/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonLineShapeQueries.java
----------------------------------------------------------------------
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 21367dc..9a91232 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonLineShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonLineShapeQueries.java
@@ -16,16 +16,12 @@
  */
 package org.apache.lucene.document;
 
+import org.apache.lucene.document.LatLonShape.QueryRelation;
 import org.apache.lucene.geo.Line;
 import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.geo.Polygon2D;
 import org.apache.lucene.index.PointValues.Relation;
 
-import static org.apache.lucene.geo.GeoEncodingUtils.decodeLatitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.decodeLongitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.encodeLatitude;
-import static org.apache.lucene.geo.GeoEncodingUtils.encodeLongitude;
-
 /** random bounding box and polygon query tests for random generated {@link Line} types */
 public class TestLatLonLineShapeQueries extends BaseLatLonShapeTestCase {
 
@@ -42,17 +38,25 @@ public class TestLatLonLineShapeQueries extends BaseLatLonShapeTestCase {
   }
 
   @Override
-  protected Validator getValidator() {
+  protected Validator getValidator(QueryRelation queryRelation) {
+    VALIDATOR.setRelation(queryRelation);
     return VALIDATOR;
   }
 
-  protected class LineValidator implements Validator {
+  protected class LineValidator extends Validator {
     @Override
     public boolean testBBoxQuery(double minLat, double maxLat, double minLon, double maxLon, Object shape) {
+      Line l = (Line)shape;
+      if (queryRelation == QueryRelation.WITHIN) {
+        // within: bounding box of shape should be within query box
+        return minLat <= quantizeLat(l.minLat) && maxLat >= quantizeLat(l.maxLat)
+            && minLon <= quantizeLon(l.minLon) && maxLon >= quantizeLon(l.maxLon);
+      }
+
       // to keep it simple we convert the bbox into a polygon and use poly2d
       Polygon2D p = Polygon2D.create(new Polygon[] {new Polygon(new double[] {minLat, minLat, maxLat, maxLat, minLat},
           new double[] {minLon, maxLon, maxLon, minLon, minLon})});
-      return testLine(p, (Line)shape);
+      return testLine(p, l);
     }
 
     @Override
@@ -62,11 +66,12 @@ public class TestLatLonLineShapeQueries extends BaseLatLonShapeTestCase {
 
     private boolean testLine(Polygon2D queryPoly, Line line) {
       double ax, ay, bx, by, temp;
+      Relation r;
       for (int i = 0, j = 1; j < line.numPoints(); ++i, ++j) {
-        ay = decodeLatitude(encodeLatitude(line.getLat(i)));
-        ax = decodeLongitude(encodeLongitude(line.getLon(i)));
-        by = decodeLatitude(encodeLatitude(line.getLat(j)));
-        bx = decodeLongitude(encodeLongitude(line.getLon(j)));
+        ay = quantizeLat(line.getLat(i));
+        ax = quantizeLon(line.getLon(i));
+        by = quantizeLat(line.getLat(j));
+        bx = quantizeLon(line.getLon(j));
         if (ay > by) {
           temp = ay;
           ay = by;
@@ -84,11 +89,16 @@ public class TestLatLonLineShapeQueries extends BaseLatLonShapeTestCase {
             bx = temp;
           }
         }
-        if (queryPoly.relateTriangle(ax, ay, bx, by, ax, ay) != Relation.CELL_OUTSIDE_QUERY) {
-          return true;
+        r = queryPoly.relateTriangle(ax, ay, bx, by, ax, ay);
+        if (queryRelation == QueryRelation.DISJOINT) {
+          if (r != Relation.CELL_OUTSIDE_QUERY) return false;
+        } else if (queryRelation == QueryRelation.WITHIN) {
+          if (r != Relation.CELL_INSIDE_QUERY) return false;
+        } else {
+          if (r != Relation.CELL_OUTSIDE_QUERY) return true;
         }
       }
-      return false;
+      return queryRelation == QueryRelation.INTERSECTS ? false : true;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cbaedb47/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java
----------------------------------------------------------------------
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 3adb26b..df924fe 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointShapeQueries.java
@@ -16,6 +16,7 @@
  */
 package org.apache.lucene.document;
 
+import org.apache.lucene.document.LatLonShape.QueryRelation;
 import org.apache.lucene.geo.Polygon2D;
 import org.apache.lucene.index.PointValues.Relation;
 
@@ -41,17 +42,22 @@ public class TestLatLonPointShapeQueries extends BaseLatLonShapeTestCase {
   }
 
   @Override
-  protected Validator getValidator() {
+  protected Validator getValidator(QueryRelation relation) {
+    VALIDATOR.setRelation(relation);
     return VALIDATOR;
   }
 
-  protected class PointValidator implements Validator {
+  protected class PointValidator extends Validator {
     @Override
     public boolean testBBoxQuery(double minLat, double maxLat, double minLon, double maxLon, Object shape) {
       Point p = (Point)shape;
       double lat = decodeLatitude(encodeLatitude(p.lat));
       double lon = decodeLongitude(encodeLongitude(p.lon));
-      return (lat < minLat || lat > maxLat || lon < minLon || lon > maxLon) == false;
+      boolean isDisjoint = lat < minLat || lat > maxLat || lon < minLon || lon > maxLon;
+      if (queryRelation == QueryRelation.DISJOINT) {
+        return isDisjoint;
+      }
+      return isDisjoint == false;
     }
 
     @Override
@@ -60,7 +66,13 @@ public class TestLatLonPointShapeQueries extends BaseLatLonShapeTestCase {
       double lat = decodeLatitude(encodeLatitude(p.lat));
       double lon = decodeLongitude(encodeLongitude(p.lon));
       // for consistency w/ the query we test the point as a triangle
-      return poly2d.relateTriangle(lon, lat, lon, lat, lon, lat) != Relation.CELL_OUTSIDE_QUERY;
+      Relation r = poly2d.relateTriangle(lon, lat, lon, lat, lon, lat);
+      if (queryRelation == QueryRelation.WITHIN) {
+        return r == Relation.CELL_INSIDE_QUERY;
+      } else if (queryRelation == QueryRelation.DISJOINT) {
+        return r == Relation.CELL_OUTSIDE_QUERY;
+      }
+      return r != Relation.CELL_OUTSIDE_QUERY;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cbaedb47/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPolygonShapeQueries.java
----------------------------------------------------------------------
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 17eb6e8..9b844ab 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPolygonShapeQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPolygonShapeQueries.java
@@ -18,11 +18,13 @@ package org.apache.lucene.document;
 
 import java.util.List;
 
+import org.apache.lucene.document.LatLonShape.QueryRelation;
 import org.apache.lucene.geo.Polygon;
 import org.apache.lucene.geo.Polygon2D;
 import org.apache.lucene.geo.Tessellator;
 import org.apache.lucene.index.PointValues.Relation;
 
+/** random bounding box and polygon query tests for random indexed {@link Polygon} types */
 public class TestLatLonPolygonShapeQueries extends BaseLatLonShapeTestCase {
 
   protected final PolygonValidator VALIDATOR = new PolygonValidator();
@@ -53,30 +55,46 @@ public class TestLatLonPolygonShapeQueries extends BaseLatLonShapeTestCase {
   }
 
   @Override
-  protected Validator getValidator() {
+  protected Validator getValidator(QueryRelation relation) {
+    VALIDATOR.setRelation(relation);
     return VALIDATOR;
   }
 
-  protected class PolygonValidator implements Validator {
+  protected class PolygonValidator extends Validator {
     @Override
     public boolean testBBoxQuery(double minLat, double maxLat, double minLon, double maxLon, Object shape) {
-      Polygon2D poly = Polygon2D.create(quantizePolygon((Polygon)shape));
-      return poly.relate(minLat, maxLat, minLon, maxLon) != Relation.CELL_OUTSIDE_QUERY;
+      Polygon p = (Polygon)shape;
+      if (queryRelation == QueryRelation.WITHIN) {
+        // within: bounding box of shape should be within query box
+        return minLat <= quantizeLat(p.minLat) && maxLat >= quantizeLat(p.maxLat)
+            && minLon <= quantizeLon(p.minLon) && maxLon >= quantizeLon(p.maxLon);
+      }
+
+      Polygon2D poly = Polygon2D.create(quantizePolygon(p));
+      Relation r = poly.relate(minLat, maxLat, minLon, maxLon);
+      if (queryRelation == QueryRelation.DISJOINT) {
+        return r == Relation.CELL_OUTSIDE_QUERY;
+      }
+      return r != Relation.CELL_OUTSIDE_QUERY;
     }
 
     @Override
     public boolean testPolygonQuery(Polygon2D query, Object shape) {
-
       List<Tessellator.Triangle> tessellation = Tessellator.tessellate((Polygon) shape);
       for (Tessellator.Triangle t : tessellation) {
         // we quantize the triangle for consistency with the index
-        if (query.relateTriangle(quantizeLon(t.getLon(0)), quantizeLat(t.getLat(0)),
+        Relation r = query.relateTriangle(quantizeLon(t.getLon(0)), quantizeLat(t.getLat(0)),
             quantizeLon(t.getLon(1)), quantizeLat(t.getLat(1)),
-            quantizeLon(t.getLon(2)), quantizeLat(t.getLat(2))) != Relation.CELL_OUTSIDE_QUERY) {
-          return true;
+            quantizeLon(t.getLon(2)), quantizeLat(t.getLat(2)));
+        if (queryRelation == QueryRelation.DISJOINT) {
+          if (r != Relation.CELL_OUTSIDE_QUERY) return false;
+        } else if (queryRelation == QueryRelation.WITHIN) {
+          if (r != Relation.CELL_INSIDE_QUERY) return false;
+        } else {
+          if (r != Relation.CELL_OUTSIDE_QUERY) return true;
         }
       }
-      return false;
+      return queryRelation == QueryRelation.INTERSECTS ? false : true;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cbaedb47/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShape.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShape.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShape.java
index 3aa5ace..2f6e5e1 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShape.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonShape.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.document;
 
 import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
+import org.apache.lucene.document.LatLonShape.QueryRelation;
 import org.apache.lucene.geo.GeoTestUtil;
 import org.apache.lucene.geo.Line;
 import org.apache.lucene.geo.Polygon;
@@ -52,7 +53,7 @@ public class TestLatLonShape extends LuceneTestCase {
   }
 
   protected Query newRectQuery(String field, double minLat, double maxLat, double minLon, double maxLon) {
-    return LatLonShape.newBoxQuery(field, minLat, maxLat, minLon, maxLon);
+    return LatLonShape.newBoxQuery(field, QueryRelation.INTERSECTS, minLat, maxLat, minLon, maxLon);
   }
 
   @Ignore