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 2020/02/21 10:33:56 UTC

[lucene-solr] branch branch_8x updated: LUCENE-9238: Add new XYPointField, queries and sorting capabilities (#1272)

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 53cb6ac  LUCENE-9238: Add new XYPointField, queries and sorting capabilities (#1272)
53cb6ac is described below

commit 53cb6acf912f54a07b870fe6726bfee329b62d06
Author: Ignacio Vera <iv...@apache.org>
AuthorDate: Fri Feb 21 11:26:30 2020 +0100

    LUCENE-9238: Add new XYPointField, queries and sorting capabilities (#1272)
    
    New XYPointField field and Queries for indexing, searching and sorting cartesian points.
---
 lucene/CHANGES.txt                                 |    3 +
 .../apache/lucene/document/XYDocValuesField.java   |  178 +++
 .../document/XYDocValuesPointInGeometryQuery.java  |  143 ++
 .../lucene/document/XYPointDistanceComparator.java |  207 +++
 .../org/apache/lucene/document/XYPointField.java   |  172 +++
 .../lucene/document/XYPointInGeometryQuery.java    |  227 ++++
 .../apache/lucene/document/XYPointSortField.java   |   96 ++
 .../lucene/search/TestXYDocValuesQueries.java      |   45 +
 .../lucene/search/TestXYPointDistanceSort.java     |  265 ++++
 .../apache/lucene/search/TestXYPointQueries.java   |   45 +
 .../org/apache/lucene/geo/BaseXYPointTestCase.java | 1424 ++++++++++++++++++++
 .../java}/org/apache/lucene/geo/ShapeTestUtil.java |   77 +-
 12 files changed, 2871 insertions(+), 11 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index b499cd9..15f53c2 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -31,6 +31,9 @@ New Features
 
 * LUCENE-8707: Add LatLonShape and XYShape distance query. (Ignacio Vera)
 
+* LUCENE-9238: New XYPointField field and Queries for indexing, searching and sorting
+  cartesian points. (Ignacio Vera)
+
 Improvements
 ---------------------
 
diff --git a/lucene/core/src/java/org/apache/lucene/document/XYDocValuesField.java b/lucene/core/src/java/org/apache/lucene/document/XYDocValuesField.java
new file mode 100644
index 0000000..3f1bb66
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/XYDocValuesField.java
@@ -0,0 +1,178 @@
+/*
+ * 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 org.apache.lucene.geo.XYCircle;
+import org.apache.lucene.geo.XYEncodingUtils;
+import org.apache.lucene.geo.XYPolygon;
+import org.apache.lucene.geo.XYRectangle;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.search.FieldDoc;
+import org.apache.lucene.search.IndexOrDocValuesQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SortField;
+
+
+/** 
+ * An per-document location field.
+ * <p>
+ * Sorting by distance is efficient. Multiple values for the same field in one document
+ * is allowed. 
+ * <p>
+ * This field defines static factory methods for common operations:
+ * <ul>
+ *   <li>{@link #newDistanceSort newDistanceSort()} for ordering documents by distance from a specified location.
+ * </ul>
+ * <p>
+ * If you also need query operations, you should add a separate {@link XYPointField} instance.
+ * If you also need to store the value, you should add a separate {@link StoredField} instance.
+ *
+ * @see XYPointField
+ */
+public class XYDocValuesField extends Field {
+
+  /**
+   * Type for a XYDocValuesField
+   * <p>
+   * Each value stores a 64-bit long where the upper 32 bits are the encoded x value,
+   * and the lower 32 bits are the encoded y value.
+   * @see org.apache.lucene.geo.XYEncodingUtils#decode(int)
+   */
+  public static final FieldType TYPE = new FieldType();
+  static {
+    TYPE.setDocValuesType(DocValuesType.SORTED_NUMERIC);
+    TYPE.freeze();
+  }
+
+  /**
+   * Creates a new XYDocValuesField with the specified x and y
+   * @param name field name
+   * @param x x value.
+   * @param y y values.
+   * @throws IllegalArgumentException if the field name is null or x or y are infinite or NaN.
+   */
+  public XYDocValuesField(String name, float x, float y) {
+    super(name, TYPE);
+    setLocationValue(x, y);
+  }
+  
+  /**
+   * Change the values of this field
+   * @param x x value.
+   * @param y y value.
+   * @throws IllegalArgumentException  if x or y are infinite or NaN.
+   */
+  public void setLocationValue(float x, float y) {
+    int xEncoded = XYEncodingUtils.encode(x);
+    int yEncoded = XYEncodingUtils.encode(y);
+    fieldsData = Long.valueOf((((long) xEncoded) << 32) | (yEncoded & 0xFFFFFFFFL));
+  }
+
+  /** helper: checks a fieldinfo and throws exception if its definitely not a XYDocValuesField */
+  static void checkCompatible(FieldInfo fieldInfo) {
+    // dv properties could be "unset", if you e.g. used only StoredField with this same name in the segment.
+    if (fieldInfo.getDocValuesType() != DocValuesType.NONE && fieldInfo.getDocValuesType() != TYPE.docValuesType()) {
+      throw new IllegalArgumentException("field=\"" + fieldInfo.name + "\" was indexed with docValuesType=" + fieldInfo.getDocValuesType() + 
+                                         " but this type has docValuesType=" + TYPE.docValuesType() + 
+                                         ", is the field really a XYDocValuesField?");
+    }
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder result = new StringBuilder();
+    result.append(getClass().getSimpleName());
+    result.append(" <");
+    result.append(name);
+    result.append(':');
+
+    long currentValue = (Long)fieldsData;
+    result.append(XYEncodingUtils.decode((int)(currentValue >> 32)));
+    result.append(',');
+    result.append(XYEncodingUtils.decode((int)(currentValue & 0xFFFFFFFF)));
+
+    result.append('>');
+    return result.toString();
+  }
+
+  /**
+   * Creates a SortField for sorting by distance from a location.
+   * <p>
+   * This sort orders documents by ascending distance from the location. The value returned in {@link FieldDoc} for
+   * the hits contains a Double instance with the distance in meters.
+   * <p>
+   * If a document is missing the field, then by default it is treated as having {@link Double#POSITIVE_INFINITY} distance
+   * (missing values sort last).
+   * <p>
+   * If a document contains multiple values for the field, the <i>closest</i> distance to the location is used.
+   * 
+   * @param field field name. must not be null.
+   * @param x x at the center.
+   * @param y y at the center.
+   * @return SortField ordering documents by distance
+   * @throws IllegalArgumentException if {@code field} is null or location has invalid coordinates.
+   */
+  public static SortField newDistanceSort(String field, float x, float y) {
+    return new XYPointSortField(field, x, y);
+  }
+
+  /**
+   * Create a query for matching a bounding box using doc values.
+   * This query is usually slow as it does not use an index structure and needs
+   * to verify documents one-by-one in order to know whether they match. It is
+   * best used wrapped in an {@link IndexOrDocValuesQuery} alongside a
+   * {@link XYPointField#newBoxQuery}.
+   */
+  public static Query newSlowBoxQuery(String field, float minX, float maxX, float minY, float maxY) {
+    XYRectangle rectangle = new XYRectangle(minX, maxX, minY, maxY);
+    return new XYDocValuesPointInGeometryQuery(field, rectangle);
+  }
+
+  /**
+   * Create a query for matching points within the specified distance of the supplied location.
+   * This query is usually slow as it does not use an index structure and needs
+   * to verify documents one-by-one in order to know whether they match. It is
+   * best used wrapped in an {@link IndexOrDocValuesQuery} alongside a
+   * {@link XYPointField#newDistanceQuery}.
+   * @param field field name. must not be null.
+   * @param x x at the center.
+   * @param y y at the center: must be within standard +/-180 coordinate bounds.
+   * @param radius maximum distance from the center in cartesian distance: must be non-negative and finite.
+   * @return query matching points within this distance
+   * @throws IllegalArgumentException if {@code field} is null, location has invalid coordinates, or radius is invalid.
+   */
+  public static Query newSlowDistanceQuery(String field, float x, float y, float radius) {
+    XYCircle circle = new XYCircle(x, y, radius);
+    return new XYDocValuesPointInGeometryQuery(field, circle);
+  }
+
+  /**
+   * Create a query for matching points within the supplied polygons.
+   * This query is usually slow as it does not use an index structure and needs
+   * to verify documents one-by-one in order to know whether they match. It is
+   * best used wrapped in an {@link IndexOrDocValuesQuery} alongside a
+   * {@link XYPointField#newPolygonQuery(String, XYPolygon...)}.
+   * @param field field name. must not be null.
+   * @param polygons array of polygons. must not be null or empty.
+   * @return query matching points within the given polygons.
+   * @throws IllegalArgumentException if {@code field} is null or polygons is empty or contain a null polygon.
+   */
+  public static Query newSlowPolygonQuery(String field, XYPolygon... polygons) {
+    return new XYDocValuesPointInGeometryQuery(field, polygons);
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/XYDocValuesPointInGeometryQuery.java b/lucene/core/src/java/org/apache/lucene/document/XYDocValuesPointInGeometryQuery.java
new file mode 100644
index 0000000..e5f8095
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/XYDocValuesPointInGeometryQuery.java
@@ -0,0 +1,143 @@
+/*
+ * 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.Arrays;
+import org.apache.lucene.geo.Component2D;
+import org.apache.lucene.geo.XYEncodingUtils;
+import org.apache.lucene.geo.XYGeometry;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TwoPhaseIterator;
+import org.apache.lucene.search.Weight;
+
+/** XYGeometry query for {@link XYDocValuesField}. */
+public class XYDocValuesPointInGeometryQuery extends Query {
+
+  private final String field;
+  private final XYGeometry[] geometries;
+
+
+  XYDocValuesPointInGeometryQuery(String field, XYGeometry... geometries) {
+    if (field == null) {
+      throw new IllegalArgumentException("field must not be null");
+    }
+    if (geometries == null) {
+      throw new IllegalArgumentException("geometries must not be null");
+    }
+    if (geometries.length == 0) {
+      throw new IllegalArgumentException("geometries must not be empty");
+    }
+    for (int i = 0; i < geometries.length; i++) {
+      if (geometries[i] == null) {
+        throw new IllegalArgumentException("geometries[" + i + "] must not be null");
+      }
+    }
+    this.field = field;
+    this.geometries = geometries.clone();
+  }
+
+  @Override
+  public String toString(String field) {
+    StringBuilder sb = new StringBuilder();
+    if (!this.field.equals(field)) {
+      sb.append(this.field);
+      sb.append(':');
+    }
+    sb.append("geometries(").append(Arrays.toString(geometries));
+    return sb.append(")").toString();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (sameClassAs(obj) == false) {
+      return false;
+    }
+    XYDocValuesPointInGeometryQuery other = (XYDocValuesPointInGeometryQuery) obj;
+    return field.equals(other.field) &&
+           Arrays.equals(geometries, other.geometries);
+  }
+
+  @Override
+  public int hashCode() {
+    int h = classHash();
+    h = 31 * h + field.hashCode();
+    h = 31 * h + Arrays.hashCode(geometries);
+    return h;
+  }
+
+  @Override
+  public void visit(QueryVisitor visitor) {
+    if (visitor.acceptField(field)) {
+      visitor.visitLeaf(this);
+    }
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+
+    return new ConstantScoreWeight(this, boost) {
+
+      final Component2D component2D = XYGeometry.create(geometries);
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        final SortedNumericDocValues values = context.reader().getSortedNumericDocValues(field);
+        if (values == null) {
+          return null;
+        }
+
+        final TwoPhaseIterator iterator = new TwoPhaseIterator(values) {
+
+          @Override
+          public boolean matches() throws IOException {
+            for (int i = 0, count = values.docValueCount(); i < count; ++i) {
+              final long value = values.nextValue();
+              final double x = XYEncodingUtils.decode((int) (value >>> 32));
+              final double y = XYEncodingUtils.decode((int) (value & 0xFFFFFFFF));
+              if (component2D.contains(x, y)) {
+                return true;
+              }
+            }
+            return false;
+          }
+
+          @Override
+          public float matchCost() {
+            return 1000f; // TODO: what should it be?
+          }
+        };
+        return new ConstantScoreScorer(this, boost, scoreMode, iterator);
+      }
+
+      @Override
+      public boolean isCacheable(LeafReaderContext ctx) {
+        return DocValues.isCacheable(ctx, field);
+      }
+    };
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/XYPointDistanceComparator.java b/lucene/core/src/java/org/apache/lucene/document/XYPointDistanceComparator.java
new file mode 100644
index 0000000..2555411
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/XYPointDistanceComparator.java
@@ -0,0 +1,207 @@
+/*
+ * 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 org.apache.lucene.geo.XYEncodingUtils;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.Scorable;
+import org.apache.lucene.util.ArrayUtil;
+
+/**
+ * Compares documents by distance from an origin point
+ * <p>
+ * When the least competitive item on the priority queue changes (setBottom), we recompute
+ * a bounding box representing competitive distance to the top-N. Then in compareBottom, we can
+ * quickly reject hits based on bounding box alone without computing distance for every element.
+ */
+class XYPointDistanceComparator extends FieldComparator<Double> implements LeafFieldComparator {
+  final String field;
+  final double x;
+  final double y;
+
+  // distances needs to be calculated with square root to
+  // avoid numerical issues (square distances are different but
+  // actual distances are equal)
+  final double[] values;
+  double bottom;
+  double topValue;
+  SortedNumericDocValues currentDocs;
+
+  // current bounding box(es) for the bottom distance on the PQ.
+  // these are pre-encoded with XYPoint's encoding and
+  // used to exclude uncompetitive hits faster.
+  int minX = Integer.MIN_VALUE;
+  int maxX = Integer.MAX_VALUE;
+  int minY = Integer.MIN_VALUE;
+  int maxY = Integer.MAX_VALUE;
+
+  // the number of times setBottom has been called (adversary protection)
+  int setBottomCounter = 0;
+
+  private long[] currentValues = new long[4];
+  private int valuesDocID = -1;
+
+  public XYPointDistanceComparator(String field, float x, float y, int numHits) {
+    this.field = field;
+    this.x = x;
+    this.y = y;
+    this.values = new double[numHits];
+  }
+  
+  @Override
+  public void setScorer(Scorable scorer) {}
+
+  @Override
+  public int compare(int slot1, int slot2) {
+    return Double.compare(values[slot1], values[slot2]);
+  }
+  
+  @Override
+  public void setBottom(int slot) {
+    bottom = values[slot];
+    // make bounding box(es) to exclude non-competitive hits, but start
+    // sampling if we get called way too much: don't make gobs of bounding
+    // boxes if comparator hits a worst case order (e.g. backwards distance order)
+    if (setBottomCounter < 1024 || (setBottomCounter & 0x3F) == 0x3F) {
+
+      // pre-encode our box to our integer encoding, so we don't have to decode
+      // to double values for uncompetitive hits. This has some cost!
+      this.minX = XYEncodingUtils.encode((float) Math.max(-Float.MAX_VALUE, x - bottom));
+      this.maxX = XYEncodingUtils.encode((float) Math.min(Float.MAX_VALUE, x + bottom));
+      this.minY = XYEncodingUtils.encode((float) Math.max(-Float.MAX_VALUE, y - bottom));
+      this.maxY = XYEncodingUtils.encode((float) Math.min(Float.MAX_VALUE, y + bottom));
+    }
+    setBottomCounter++;
+  }
+  
+  @Override
+  public void setTopValue(Double value) {
+    topValue = value.doubleValue();
+  }
+
+  private void setValues() throws IOException {
+    if (valuesDocID != currentDocs.docID()) {
+      assert valuesDocID < currentDocs.docID(): " valuesDocID=" + valuesDocID + " vs " + currentDocs.docID();
+      valuesDocID = currentDocs.docID();
+      int count = currentDocs.docValueCount();
+      if (count > currentValues.length) {
+        currentValues = new long[ArrayUtil.oversize(count, Long.BYTES)];
+      }
+      for(int i=0;i<count;i++) {
+        currentValues[i] = currentDocs.nextValue();
+      }
+    }
+  }
+  
+  @Override
+  public int compareBottom(int doc) throws IOException {
+    if (doc > currentDocs.docID()) {
+      currentDocs.advance(doc);
+    }
+    if (doc < currentDocs.docID()) {
+      return Double.compare(bottom, Double.POSITIVE_INFINITY);
+    }
+
+    setValues();
+
+    int numValues = currentDocs.docValueCount();
+
+    int cmp = -1;
+    for (int i = 0; i < numValues; i++) {
+      long encoded = currentValues[i];
+
+      // test bounding box
+      int xBits = (int)(encoded >> 32);
+      if (xBits < minX || xBits > maxX) {
+        continue;
+      }
+      int yBits = (int)(encoded & 0xFFFFFFFF);
+      if (yBits < minY || yBits > maxY) {
+        continue;
+      }
+
+      // only compute actual distance if its inside "competitive bounding box"
+      double docX = XYEncodingUtils.decode(xBits);
+      double docY = XYEncodingUtils.decode(yBits);
+      final double diffX = x - docX;
+      final double diffY = y - docY;
+      double distance =  Math.sqrt(diffX * diffX + diffY * diffY);
+      cmp = Math.max(cmp, Double.compare(bottom, distance));
+      // once we compete in the PQ, no need to continue.
+      if (cmp > 0) {
+        return cmp;
+      }
+    }
+    return cmp;
+  }
+  
+  @Override
+  public void copy(int slot, int doc) throws IOException {
+    values[slot] = sortKey(doc);
+  }
+  
+  @Override
+  public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException {
+    LeafReader reader = context.reader();
+    FieldInfo info = reader.getFieldInfos().fieldInfo(field);
+    if (info != null) {
+      XYDocValuesField.checkCompatible(info);
+    }
+    currentDocs = DocValues.getSortedNumeric(reader, field);
+    valuesDocID = -1;
+    return this;
+  }
+  
+  @Override
+  public Double value(int slot) {
+    return values[slot];
+  }
+  
+  @Override
+  public int compareTop(int doc) throws IOException {
+    return Double.compare(topValue, sortKey(doc));
+  }
+
+  double sortKey(int doc) throws IOException {
+    if (doc > currentDocs.docID()) {
+      currentDocs.advance(doc);
+    }
+    double minValue = Double.POSITIVE_INFINITY;
+    if (doc == currentDocs.docID()) {
+      setValues();
+      int numValues = currentDocs.docValueCount();
+      for (int i = 0; i < numValues; i++) {
+        long encoded = currentValues[i];
+        double docX = XYEncodingUtils.decode((int)(encoded >> 32));
+        double docY = XYEncodingUtils.decode((int)(encoded & 0xFFFFFFFF));
+        final double diffX = x - docX;
+        final double diffY = y - docY;
+        double distance =  Math.sqrt(diffX * diffX + diffY * diffY);
+        minValue = Math.min(minValue, distance);
+      }
+    }
+    return minValue;
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/XYPointField.java b/lucene/core/src/java/org/apache/lucene/document/XYPointField.java
new file mode 100644
index 0000000..1c43774
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/XYPointField.java
@@ -0,0 +1,172 @@
+/*
+ * 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 org.apache.lucene.geo.Polygon;
+import org.apache.lucene.geo.XYCircle;
+import org.apache.lucene.geo.XYEncodingUtils;
+import org.apache.lucene.geo.XYPolygon;
+import org.apache.lucene.geo.XYRectangle;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
+
+
+/** 
+ * An indexed XY position field.
+ * <p>
+ * Finding all documents within a range at search time is
+ * efficient.  Multiple values for the same field in one document
+ * is allowed. 
+ * <p>
+ * This field defines static factory methods for common operations:
+ * <ul>
+ *   <li>{@link #newBoxQuery newBoxQuery()} for matching points within a bounding box.
+ *   <li>{@link #newDistanceQuery newDistanceQuery()} for matching points within a specified distance.
+ *   <li>{@link #newPolygonQuery newPolygonQuery()} for matching points within an arbitrary polygon.
+ * </ul>
+ * <p>
+ * If you also need per-document operations such as sort by distance, add a separate {@link XYDocValuesField} instance.
+ * If you also need to store the value, you should add a separate {@link StoredField} instance.
+ *
+ * @see PointValues
+ * @see XYDocValuesField
+ */
+
+public class XYPointField extends Field {
+  /** XYPoint is encoded as integer values so number of bytes is 4 */
+  public static final int BYTES = Integer.BYTES;
+  /**
+   * Type for an indexed XYPoint
+   * <p>
+   * Each point stores two dimensions with 4 bytes per dimension.
+   */
+  public static final FieldType TYPE = new FieldType();
+  static {
+    TYPE.setDimensions(2, Integer.BYTES);
+    TYPE.freeze();
+  }
+
+  /**
+   * Change the values of this field
+   * @param x x value.
+   * @param y y value.
+   */
+  public void setLocationValue(float x, float y) {
+    final byte[] bytes;
+    if (fieldsData == null) {
+      bytes = new byte[8];
+      fieldsData = new BytesRef(bytes);
+    } else {
+      bytes = ((BytesRef) fieldsData).bytes;
+    }
+    int xEncoded = XYEncodingUtils.encode(x);
+    int yEncoded = XYEncodingUtils.encode(y);
+    NumericUtils.intToSortableBytes(xEncoded, bytes, 0);
+    NumericUtils.intToSortableBytes(yEncoded, bytes, Integer.BYTES);
+  }
+
+  /**
+   * Creates a new XYPoint with the specified x and y
+   * @param name field name
+   * @param x x value.
+   * @param y y value.
+   */
+  public XYPointField(String name, float x, float y) {
+    super(name, TYPE);
+    setLocationValue(x, y);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder result = new StringBuilder();
+    result.append(getClass().getSimpleName());
+    result.append(" <");
+    result.append(name);
+    result.append(':');
+
+    byte bytes[] = ((BytesRef) fieldsData).bytes;
+    result.append(XYEncodingUtils.decode(bytes, 0));
+    result.append(',');
+    result.append(XYEncodingUtils.decode(bytes, Integer.BYTES));
+
+    result.append('>');
+    return result.toString();
+  }
+
+
+  /** helper: checks a fieldinfo and throws exception if its definitely not a XYPoint */
+  static void checkCompatible(FieldInfo fieldInfo) {
+    // point/dv properties could be "unset", if you e.g. used only StoredField with this same name in the segment.
+    if (fieldInfo.getPointDimensionCount() != 0 && fieldInfo.getPointDimensionCount() != TYPE.pointDimensionCount()) {
+      throw new IllegalArgumentException("field=\"" + fieldInfo.name + "\" was indexed with numDims=" + fieldInfo.getPointDimensionCount() +
+          " but this point type has numDims=" + TYPE.pointDimensionCount() +
+                                         ", is the field really a XYPoint?");
+    }
+    if (fieldInfo.getPointNumBytes() != 0 && fieldInfo.getPointNumBytes() != TYPE.pointNumBytes()) {
+      throw new IllegalArgumentException("field=\"" + fieldInfo.name + "\" was indexed with bytesPerDim=" + fieldInfo.getPointNumBytes() +
+                                         " but this point type has bytesPerDim=" + TYPE.pointNumBytes() +
+                                         ", is the field really a XYPoint?");
+    }
+  }
+
+  // static methods for generating queries
+
+  /**
+   * Create a query for matching a bounding box.
+   * <p>
+   * @param field field name. must not be null.
+   * @param minX x lower bound.
+   * @param maxX x upper bound.
+   * @param minY y lower bound.
+   * @param maxY y upper bound.
+   * @return query matching points within this box
+   * @throws IllegalArgumentException if {@code field} is null, or the box has invalid coordinates.
+   */
+  public static Query newBoxQuery(String field, float minX, float maxX, float minY, float maxY) {
+    XYRectangle rectangle = new XYRectangle(minX, maxX, minY, maxY);
+    return new XYPointInGeometryQuery(field, rectangle);
+  }
+
+  /**
+   * Create a query for matching points within the specified distance of the supplied location.
+   * @param field field name. must not be null.
+   * @param x x at the center.
+   * @param y y at the center.
+   * @param radius maximum distance from the center in cartesian units: must be non-negative and finite.
+   * @return query matching points within this distance
+   * @throws IllegalArgumentException if {@code field} is null, location has invalid coordinates, or radius is invalid.
+   */
+  public static Query newDistanceQuery(String field, float x, float y, float radius) {
+    XYCircle circle = new XYCircle(x, y, radius);
+    return new XYPointInGeometryQuery(field, circle);
+  }
+  
+  /** 
+   * Create a query for matching one or more polygons.
+   * @param field field name. must not be null.
+   * @param polygons array of polygons. must not be null or empty
+   * @return query matching points within this polygon
+   * @throws IllegalArgumentException if {@code field} is null, {@code polygons} is null or empty
+   * @see Polygon
+   */
+  public static Query newPolygonQuery(String field, XYPolygon... polygons) {
+    return new XYPointInGeometryQuery(field, polygons);
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/XYPointInGeometryQuery.java b/lucene/core/src/java/org/apache/lucene/document/XYPointInGeometryQuery.java
new file mode 100644
index 0000000..00189bd
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/XYPointInGeometryQuery.java
@@ -0,0 +1,227 @@
+/*
+ * 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.Arrays;
+
+import org.apache.lucene.geo.Component2D;
+import org.apache.lucene.geo.XYEncodingUtils;
+import org.apache.lucene.geo.XYGeometry;
+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.QueryVisitor;
+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.DocIdSetBuilder;
+
+/** Finds all previously indexed points that fall within the specified XY geometries.
+ *
+ *  <p>The field must be indexed with using {@link XYPointField} added per document.
+ *
+ *  @lucene.experimental */
+
+final class XYPointInGeometryQuery extends Query {
+  final String field;
+  final XYGeometry[] xyGeometries;
+
+  XYPointInGeometryQuery(String field, XYGeometry... xyGeometries) {
+    if (field == null) {
+      throw new IllegalArgumentException("field must not be null");
+    }
+    if (xyGeometries == null) {
+      throw new IllegalArgumentException("geometries must not be null");
+    }
+    if (xyGeometries.length == 0) {
+      throw new IllegalArgumentException("geometries must not be empty");
+    }
+    this.field = field;
+    this.xyGeometries = xyGeometries.clone();
+  }
+
+  @Override
+  public void visit(QueryVisitor visitor) {
+    if (visitor.acceptField(field)) {
+      visitor.visitLeaf(this);
+    }
+  }
+
+  private IntersectVisitor getIntersectVisitor(DocIdSetBuilder result, Component2D tree) {
+    return new IntersectVisitor() {
+          DocIdSetBuilder.BulkAdder adder;
+
+          @Override
+          public void grow(int count) {
+            adder = result.grow(count);
+          }
+
+          @Override
+          public void visit(int docID) {
+            adder.add(docID);
+          }
+
+          @Override
+          public void visit(int docID, byte[] packedValue) {
+            double x = XYEncodingUtils.decode(packedValue, 0);
+            double y = XYEncodingUtils.decode(packedValue, Integer.BYTES);
+            if (tree.contains(x, y)) {
+              visit(docID);
+            }
+          }
+
+          @Override
+          public void visit(DocIdSetIterator iterator, byte[] packedValue) throws IOException {
+            double x = XYEncodingUtils.decode(packedValue, 0);
+            double y = XYEncodingUtils.decode(packedValue, Integer.BYTES);
+            if (tree.contains(x, y)) {
+              int docID;
+              while ((docID = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+                visit(docID);
+              }
+            }
+          }
+
+          @Override
+          public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+            double cellMinX = XYEncodingUtils.decode(minPackedValue, 0);
+            double cellMinY = XYEncodingUtils.decode(minPackedValue, Integer.BYTES);
+            double cellMaxX = XYEncodingUtils.decode(maxPackedValue, 0);
+            double cellMaxY = XYEncodingUtils.decode(maxPackedValue, Integer.BYTES);
+            return tree.relate(cellMinX, cellMaxX, cellMinY, cellMaxY);
+          }
+        };
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+
+    final Component2D tree = XYGeometry.create(xyGeometries);
+
+    return new ConstantScoreWeight(this, boost) {
+
+      @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;
+        }
+        XYPointField.checkCompatible(fieldInfo);
+        final Weight weight = this;
+
+        return new ScorerSupplier() {
+
+          long cost = -1;
+          DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
+          final IntersectVisitor visitor = getIntersectVisitor(result, tree);
+
+          @Override
+          public Scorer get(long leadCost) throws IOException {
+            values.intersect(visitor);
+            return new ConstantScoreScorer(weight, score(), scoreMode, result.build().iterator());
+          }
+
+          @Override
+          public long cost() {
+            if (cost == -1) {
+               // Computing the cost may be expensive, so only do it if necessary
+              cost = values.estimateDocCount(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;
+      }
+    };
+
+  }
+
+  /** Returns the query field */
+  public String getField() {
+    return field;
+  }
+
+  /** Returns a copy of the internal geometries array */
+  public XYGeometry[] getGeometries() {
+    return xyGeometries.clone();
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = classHash();
+    result = prime * result + field.hashCode();
+    result = prime * result + Arrays.hashCode(xyGeometries);
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    return sameClassAs(other) &&
+           equalsTo(getClass().cast(other));
+  }
+
+  private boolean equalsTo(XYPointInGeometryQuery other) {
+    return field.equals(other.field) &&
+           Arrays.equals(xyGeometries, other.xyGeometries);
+  }
+
+  @Override
+  public String toString(String field) {
+    final StringBuilder sb = new StringBuilder();
+    sb.append(getClass().getSimpleName());
+    sb.append(':');
+    if (this.field.equals(field) == false) {
+      sb.append(" field=");
+      sb.append(this.field);
+      sb.append(':');
+    }
+    sb.append(Arrays.toString(xyGeometries));
+    return sb.toString();
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/XYPointSortField.java b/lucene/core/src/java/org/apache/lucene/document/XYPointSortField.java
new file mode 100644
index 0000000..361e1bd
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/XYPointSortField.java
@@ -0,0 +1,96 @@
+/*
+ * 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 org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.SortField;
+
+/**
+ * Sorts by distance from an origin location.
+ */
+final class XYPointSortField extends SortField {
+  final float x;
+  final float y;
+
+  XYPointSortField(String field, float x, float y) {
+    super(field, Type.CUSTOM);
+    if (field == null) {
+      throw new IllegalArgumentException("field must not be null");
+    }
+    this.x = x;
+    this.y = y;
+    setMissingValue(Double.POSITIVE_INFINITY);
+  }
+  
+  @Override
+  public FieldComparator<?> getComparator(int numHits, int sortPos) {
+    return new XYPointDistanceComparator(getField(), x, y, numHits);
+  }
+
+  @Override
+  public Double getMissingValue() {
+    return (Double) super.getMissingValue();
+  }
+
+  @Override
+  public void setMissingValue(Object missingValue) {
+    if (Double.valueOf(Double.POSITIVE_INFINITY).equals(missingValue) == false) {
+      throw new IllegalArgumentException("Missing value can only be Double.POSITIVE_INFINITY (missing values last), but got " + missingValue);
+    }
+    this.missingValue = missingValue;
+  }
+  
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = super.hashCode();
+    long temp;
+    temp = Float.floatToIntBits(x);
+    result = prime * result + (int) (temp ^ (temp >>> 32));
+    temp = Float.floatToIntBits(y);
+    result = prime * result + (int) (temp ^ (temp >>> 32));
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) return true;
+    if (!super.equals(obj)) return false;
+    if (getClass() != obj.getClass()) return false;
+    XYPointSortField other = (XYPointSortField) obj;
+    if (x != other.x || y != other.y) return false;
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("<distance:");
+    builder.append('"');
+    builder.append(getField());
+    builder.append('"');
+    builder.append(" x=");
+    builder.append(x);
+    builder.append(" y=");
+    builder.append(y);
+    if (Double.POSITIVE_INFINITY != getMissingValue()) {
+      builder.append(" missingValue=").append(getMissingValue());
+    }
+    builder.append('>');
+    return builder.toString();
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestXYDocValuesQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestXYDocValuesQueries.java
new file mode 100644
index 0000000..4ca1e5e
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestXYDocValuesQueries.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.XYDocValuesField;
+import org.apache.lucene.geo.BaseXYPointTestCase;
+import org.apache.lucene.geo.XYPolygon;
+
+public class TestXYDocValuesQueries extends BaseXYPointTestCase {
+
+  @Override
+  protected void addPointToDoc(String field, Document doc, float x, float y) {
+    doc.add(new XYDocValuesField(field, x, y));
+  }
+
+  @Override
+  protected Query newRectQuery(String field, float minX, float maxX, float minY, float maxY) {
+    return XYDocValuesField.newSlowBoxQuery(field, minX, maxX, minY, maxY);
+  }
+
+  @Override
+  protected Query newDistanceQuery(String field, float centerX, float centerY, float radius) {
+    return XYDocValuesField.newSlowDistanceQuery(field, centerX, centerY, radius);
+  }
+
+  @Override
+  protected Query newPolygonQuery(String field, XYPolygon... polygons) {
+    return XYDocValuesField.newSlowPolygonQuery(field, polygons);
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestXYPointDistanceSort.java b/lucene/core/src/test/org/apache/lucene/search/TestXYPointDistanceSort.java
new file mode 100644
index 0000000..782bf77
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestXYPointDistanceSort.java
@@ -0,0 +1,265 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.document.XYDocValuesField;
+import org.apache.lucene.geo.ShapeTestUtil;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+/** Simple tests for {@link XYDocValuesField#newDistanceSort} */
+public class TestXYPointDistanceSort extends LuceneTestCase {
+
+  private double cartesianDistance(double x1, double y1, double x2, double y2) {
+    final double diffX = x1 - x2;
+    final double diffY = y1 - y2;
+    return Math.sqrt(diffX * diffX + diffY * diffY);
+  }
+
+  /** Add three points and sort by distance */
+  public void testDistanceSort() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    
+    // add some docs
+    Document doc = new Document();
+    doc.add(new XYDocValuesField("location", 40.759011f, -73.9844722f));
+    iw.addDocument(doc);
+    double d1 = cartesianDistance(40.759011f, -73.9844722f, 40.7143528f, -74.0059731f);
+    
+    doc = new Document();
+    doc.add(new XYDocValuesField("location", 40.718266f, -74.007819f));
+    iw.addDocument(doc);
+    double d2 = cartesianDistance(40.718266f, -74.007819f, 40.7143528f, -74.0059731f);
+    
+    doc = new Document();
+    doc.add(new XYDocValuesField("location", 40.7051157f, -74.0088305f));
+    iw.addDocument(doc);
+    double d3 = cartesianDistance(40.7051157f, -74.0088305f, 40.7143528f, -74.0059731f);
+
+    IndexReader reader = iw.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+
+    Sort sort = new Sort(XYDocValuesField.newDistanceSort("location", 40.7143528f, -74.0059731f));
+    TopDocs td = searcher.search(new MatchAllDocsQuery(), 3, sort);
+    
+    FieldDoc d = (FieldDoc) td.scoreDocs[0];
+    assertEquals(d2, (Double)d.fields[0], 0.0D);
+    
+    d = (FieldDoc) td.scoreDocs[1];
+    assertEquals(d3, (Double)d.fields[0], 0.0D);
+    
+    d = (FieldDoc) td.scoreDocs[2];
+    assertEquals(d1, (Double)d.fields[0], 0.0D);
+    
+    reader.close();
+    dir.close();
+  }
+  
+  /** Add two points (one doc missing) and sort by distance */
+  public void testMissingLast() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    
+    // missing
+    Document doc = new Document();
+    iw.addDocument(doc);
+    
+    doc = new Document();
+    doc.add(new XYDocValuesField("location", 40.718266f, -74.007819f));
+    iw.addDocument(doc);
+    double d2 = cartesianDistance(40.718266f, -74.007819f, 40.7143528f, -74.0059731f);
+    
+    doc = new Document();
+    doc.add(new XYDocValuesField("location", 40.7051157f, -74.0088305f));
+    iw.addDocument(doc);
+    double d3 = cartesianDistance(40.7051157f, -74.0088305f, 40.7143528f, -74.0059731f);
+
+
+    IndexReader reader = iw.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+
+    Sort sort = new Sort(XYDocValuesField.newDistanceSort("location", 40.7143528f, -74.0059731f));
+    TopDocs td = searcher.search(new MatchAllDocsQuery(), 3, sort);
+    
+    FieldDoc d = (FieldDoc) td.scoreDocs[0];
+    assertEquals(d2, (Double)d.fields[0], 0.0D);
+    
+    d = (FieldDoc) td.scoreDocs[1];
+    assertEquals(d3, (Double)d.fields[0], 0.0D);
+    
+    d = (FieldDoc) td.scoreDocs[2];
+    assertEquals(Double.POSITIVE_INFINITY, (Double)d.fields[0], 0.0D);
+    
+    reader.close();
+    dir.close();
+  }
+
+  /** Run a few iterations with just 10 docs, hopefully easy to debug */
+  public void testRandom() throws Exception {
+    for (int iters = 0; iters < 100; iters++) {
+      doRandomTest(10, 100);
+    }
+  }
+  
+  /** Runs with thousands of docs */
+  @Nightly
+  public void testRandomHuge() throws Exception {
+    for (int iters = 0; iters < 10; iters++) {
+      doRandomTest(2000, 100);
+    }
+  }
+
+  // result class used for testing. holds an id+distance.
+  // we sort these with Arrays.sort and compare with lucene's results
+  static class Result implements Comparable<Result> {
+    int id;
+    double distance;
+    
+    Result(int id, double distance) {
+      this.id = id;
+      this.distance = distance;
+    }
+
+    @Override
+    public int compareTo(Result o) {
+      int cmp = Double.compare(distance, o.distance);
+      if (cmp == 0) {
+        return Integer.compare(id, o.id);
+      }
+      return cmp;
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      long temp;
+      temp = Double.doubleToLongBits(distance);
+      result = prime * result + (int) (temp ^ (temp >>> 32));
+      result = prime * result + id;
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) return true;
+      if (obj == null) return false;
+      if (getClass() != obj.getClass()) return false;
+      Result other = (Result) obj;
+      if (Double.doubleToLongBits(distance) != Double.doubleToLongBits(other.distance)) return false;
+      if (id != other.id) return false;
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      return "Result [id=" + id + ", distance=" + distance + "]";
+    }
+  }
+  
+  private void doRandomTest(int numDocs, int numQueries) throws IOException {
+    Directory dir = newDirectory();    
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    // else seeds may not to reproduce:
+    iwc.setMergeScheduler(new SerialMergeScheduler());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      doc.add(new StoredField("id", i));
+      doc.add(new NumericDocValuesField("id", i));
+      if (random().nextInt(10) > 7) {
+        float x = ShapeTestUtil.nextFloat(random());
+        float y = ShapeTestUtil.nextFloat(random());
+
+        doc.add(new XYDocValuesField("field", x, y));
+        doc.add(new StoredField("x", x));
+        doc.add(new StoredField("y", y));
+      } // otherwise "missing"
+      writer.addDocument(doc);
+    }
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+
+    for (int i = 0; i < numQueries; i++) {
+      float x = ShapeTestUtil.nextFloat(random());
+      float y = ShapeTestUtil.nextFloat(random());
+      double missingValue = Double.POSITIVE_INFINITY;
+
+      Result expected[] = new Result[reader.maxDoc()];
+      
+      for (int doc = 0; doc < reader.maxDoc(); doc++) {
+        Document targetDoc = reader.document(doc);
+        final double distance;
+        if (targetDoc.getField("x") == null) {
+          distance = missingValue; // missing
+        } else {
+          double docX = targetDoc.getField("x").numericValue().floatValue();
+          double docY = targetDoc.getField("y").numericValue().floatValue();
+          distance = cartesianDistance(x, y, docX, docY);
+        }
+        int id = targetDoc.getField("id").numericValue().intValue();
+        expected[doc] = new Result(id, distance);
+      }
+      
+      Arrays.sort(expected);
+      
+      // randomize the topN a bit
+      int topN = TestUtil.nextInt(random(), 1, reader.maxDoc());
+      // sort by distance, then ID
+      SortField distanceSort = XYDocValuesField.newDistanceSort("field", x, y);
+      distanceSort.setMissingValue(missingValue);
+      Sort sort = new Sort(distanceSort, 
+                           new SortField("id", SortField.Type.INT));
+
+      TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), topN, sort);
+      for (int resultNumber = 0; resultNumber < topN; resultNumber++) {
+        FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[resultNumber];
+        Result actual = new Result((Integer) fieldDoc.fields[1], (Double) fieldDoc.fields[0]);
+        assertEquals(expected[resultNumber], actual);
+      }
+
+      // get page2 with searchAfter()
+      if (topN < reader.maxDoc()) {
+        int page2 = TestUtil.nextInt(random(), 1, reader.maxDoc() - topN);
+        TopDocs topDocs2 = searcher.searchAfter(topDocs.scoreDocs[topN - 1], new MatchAllDocsQuery(), page2, sort);
+        for (int resultNumber = 0; resultNumber < page2; resultNumber++) {
+          FieldDoc fieldDoc = (FieldDoc) topDocs2.scoreDocs[resultNumber];
+          Result actual = new Result((Integer) fieldDoc.fields[1], (Double) fieldDoc.fields[0]);
+          assertEquals(expected[topN + resultNumber], actual);
+        }
+      }
+    }
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestXYPointQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestXYPointQueries.java
new file mode 100644
index 0000000..208c3bd
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestXYPointQueries.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.XYPointField;
+import org.apache.lucene.geo.BaseXYPointTestCase;
+import org.apache.lucene.geo.XYPolygon;
+
+public class TestXYPointQueries extends BaseXYPointTestCase  {
+
+  @Override
+  protected void addPointToDoc(String field, Document doc, float x, float y) {
+    doc.add(new XYPointField(field, x, y));
+  }
+
+  @Override
+  protected Query newRectQuery(String field, float minX, float maxX, float minY, float maxY) {
+    return XYPointField.newBoxQuery(field, minX, maxX, minY, maxY);
+  }
+
+  @Override
+  protected Query newDistanceQuery(String field, float centerX, float centerY, float radius) {
+    return XYPointField.newDistanceQuery(field, centerX, centerY, radius);
+  }
+
+  @Override
+  protected Query newPolygonQuery(String field, XYPolygon... polygons) {
+    return XYPointField.newPolygonQuery(field, polygons);
+  }
+}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/geo/BaseXYPointTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseXYPointTestCase.java
new file mode 100644
index 0000000..a597ca5
--- /dev/null
+++ b/lucene/test-framework/src/java/org/apache/lucene/geo/BaseXYPointTestCase.java
@@ -0,0 +1,1424 @@
+/*
+ * 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.geo;
+
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.text.DecimalFormatSymbols;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.Set;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.PointsFormat;
+import org.apache.lucene.codecs.PointsReader;
+import org.apache.lucene.codecs.PointsWriter;
+import org.apache.lucene.codecs.lucene60.Lucene60PointsReader;
+import org.apache.lucene.codecs.lucene60.Lucene60PointsWriter;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.MultiBits;
+import org.apache.lucene.index.MultiDocValues;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.SerialMergeScheduler;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.bkd.BKDWriter;
+
+/**
+ * Abstract class to do basic tests for a xy spatial impl (high level
+ * fields and queries) */
+public abstract class BaseXYPointTestCase extends LuceneTestCase {
+
+  protected static final String FIELD_NAME = "point";
+
+  // TODO: remove these hooks once all subclasses can pass with new random!
+
+  protected float nextX() {
+    return ShapeTestUtil.nextFloat(random());
+  }
+
+  protected float nextY() {
+    return ShapeTestUtil.nextFloat(random());
+  }
+
+  protected XYRectangle nextBox() {
+    return ShapeTestUtil.nextBox(random());
+  }
+
+  protected XYPolygon nextPolygon() {
+    return ShapeTestUtil.nextPolygon();
+  }
+
+  /** Whether this impl supports polygons. */
+  protected boolean supportsPolygons() {
+    return true;
+  }
+
+  /** Valid values that should not cause exception */
+  public void testIndexExtremeValues() {
+    Document document = new Document();
+    addPointToDoc("foo", document, Float.MAX_VALUE, Float.MAX_VALUE);
+    addPointToDoc("foo", document, Float.MAX_VALUE, -Float.MAX_VALUE);
+    addPointToDoc("foo", document, -Float.MAX_VALUE, Float.MAX_VALUE);
+    addPointToDoc("foo", document, -Float.MAX_VALUE, -Float.MAX_VALUE);
+  }
+  
+  /** NaN: illegal */
+  public void testIndexNaNValues() {
+    Document document = new Document();
+    IllegalArgumentException expected;
+
+    expected = expectThrows(IllegalArgumentException.class, () -> {
+      addPointToDoc("foo", document, Float.NaN, 50.0f);
+    });
+    assertTrue(expected.getMessage().contains("invalid value"));
+    
+    expected = expectThrows(IllegalArgumentException.class, () -> {
+      addPointToDoc("foo", document, 50.0f, Float.NaN);
+    });
+    assertTrue(expected.getMessage().contains("invalid value"));
+  }
+  
+  /** Inf: illegal */
+  public void testIndexInfValues() {
+    Document document = new Document();
+    IllegalArgumentException expected;
+
+    expected = expectThrows(IllegalArgumentException.class, () -> {
+      addPointToDoc("foo", document, Float.POSITIVE_INFINITY, 0.0f);
+    });
+    assertTrue(expected.getMessage().contains("invalid value"));
+
+    expected = expectThrows(IllegalArgumentException.class, () -> {
+      addPointToDoc("foo", document, Float.NEGATIVE_INFINITY, 0.0f);
+    });
+    assertTrue(expected.getMessage().contains("invalid value"));
+
+    expected = expectThrows(IllegalArgumentException.class, () -> {
+      addPointToDoc("foo", document, 0.0f, Float.POSITIVE_INFINITY);
+    });
+    assertTrue(expected.getMessage().contains("invalid value"));
+
+    expected = expectThrows(IllegalArgumentException.class, () -> {
+      addPointToDoc("foo", document, 0.0f, Float.NEGATIVE_INFINITY);
+    });
+    assertTrue(expected.getMessage().contains("invalid value"));
+  }
+  
+  /** Add a single point and search for it in a box */
+  public void testBoxBasics() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // add a doc with a point
+    Document document = new Document();
+    addPointToDoc("field", document, 18.313694f, -65.227444f);
+    writer.addDocument(document);
+    
+    // search and verify we found our doc
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    assertEquals(1, searcher.count(newRectQuery("field", 18, 19, -66, -65)));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+
+  /** null field name not allowed */
+  public void testBoxNull() {
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+      newRectQuery(null, 18, 19, -66, -65);
+    });
+    assertTrue(expected.getMessage().contains("field must not be null"));
+  }
+
+  // box should not accept invalid x/y
+  public void testBoxInvalidCoordinates() throws Exception {
+    expectThrows(Exception.class, () -> {
+      newRectQuery("field", Float.NaN, Float.NaN,Float.NaN, Float.NaN);
+    });
+  }
+
+  /** test we can search for a point */
+  public void testDistanceBasics() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // add a doc with a location
+    Document document = new Document();
+    addPointToDoc("field", document, 18.313694f, -65.227444f);
+    writer.addDocument(document);
+    
+    // search within 50km and verify we found our doc
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    assertEquals(1, searcher.count(newDistanceQuery("field", 18, -65, 20)));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+  
+  /** null field name not allowed */
+  public void testDistanceNull() {
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+      newDistanceQuery(null, 18, -65, 50_000);
+    });
+    assertTrue(expected.getMessage().contains("field must not be null"));
+  }
+  
+  /** distance query should not accept invalid x/y as origin */
+  public void testDistanceIllegal() throws Exception {
+    expectThrows(Exception.class, () -> {
+      newDistanceQuery("field", Float.NaN, Float.NaN, 120000);
+    });
+  }
+
+  /** negative distance queries are not allowed */
+  public void testDistanceNegative() {
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+      newDistanceQuery("field", 18, 19, -1);
+    });
+    assertTrue(expected.getMessage().contains("radius"));
+  }
+  
+  /** NaN distance queries are not allowed */
+  public void testDistanceNaN() {
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+      newDistanceQuery("field", 18, 19, Float.NaN);
+    });
+    assertTrue(expected.getMessage().contains("radius"));
+    assertTrue(expected.getMessage().contains("NaN"));
+  }
+  
+  /** Inf distance queries are not allowed */
+  public void testDistanceInf() {
+    IllegalArgumentException expected;
+    
+    expected = expectThrows(IllegalArgumentException.class, () -> {
+      newDistanceQuery("field", 18, 19, Float.POSITIVE_INFINITY);
+    });
+    assertTrue(expected.getMessage(), expected.getMessage().contains("radius"));
+    assertTrue(expected.getMessage(), expected.getMessage().contains("finite"));
+    
+    expected = expectThrows(IllegalArgumentException.class, () -> {
+      newDistanceQuery("field", 18, 19, Float.NEGATIVE_INFINITY);
+    });
+    assertTrue(expected.getMessage(), expected.getMessage().contains("radius"));
+    assertTrue(expected.getMessage(), expected.getMessage().contains("bigger than 0"));
+  }
+  
+  /** test we can search for a polygon */
+  public void testPolygonBasics() throws Exception {
+    assumeTrue("Impl does not support polygons", supportsPolygons());
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // add a doc with a point
+    Document document = new Document();
+    addPointToDoc("field", document, 18.313694f, -65.227444f);
+    writer.addDocument(document);
+    
+    // search and verify we found our doc
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    assertEquals(1, searcher.count(newPolygonQuery("field", new XYPolygon(
+                                                   new float[] { 18, 18, 19, 19, 18 },
+                                                   new float[] { -66, -65, -65, -66, -66 }))));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+  
+  /** test we can search for a polygon with a hole (but still includes the doc) */
+  public void testPolygonHole() throws Exception {
+    assumeTrue("Impl does not support polygons", supportsPolygons());
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // add a doc with a point
+    Document document = new Document();
+    addPointToDoc("field", document, 18.313694f, -65.227444f);
+    writer.addDocument(document);
+    
+    // search and verify we found our doc
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    XYPolygon inner = new XYPolygon(new float[] { 18.5f, 18.5f, 18.7f, 18.7f, 18.5f },
+                                new float[] { -65.7f, -65.4f, -65.4f, -65.7f, -65.7f });
+    XYPolygon outer = new XYPolygon(new float[] { 18, 18, 19, 19, 18 },
+                                new float[] { -66, -65, -65, -66, -66 }, inner);
+    assertEquals(1, searcher.count(newPolygonQuery("field", outer)));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+  
+  /** test we can search for a polygon with a hole (that excludes the doc) */
+  public void testPolygonHoleExcludes() throws Exception {
+    assumeTrue("Impl does not support polygons", supportsPolygons());
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // add a doc with a point
+    Document document = new Document();
+    addPointToDoc("field", document, 18.313694f, -65.227444f);
+    writer.addDocument(document);
+    
+    // search and verify we found our doc
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    XYPolygon inner = new XYPolygon(new float[] { 18.2f, 18.2f, 18.4f, 18.4f, 18.2f },
+                                new float[] { -65.3f, -65.2f, -65.2f, -65.3f, -65.3f });
+    XYPolygon outer = new XYPolygon(new float[] { 18, 18, 19, 19, 18 },
+                                new float[] { -66, -65, -65, -66, -66 }, inner);
+    assertEquals(0, searcher.count(newPolygonQuery("field", outer)));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+  
+  /** test we can search for a multi-polygon */
+  public void testMultiPolygonBasics() throws Exception {
+    assumeTrue("Impl does not support polygons", supportsPolygons());
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // add a doc with a point
+    Document document = new Document();
+    addPointToDoc("field", document, 18.313694f, -65.227444f);
+    writer.addDocument(document);
+    
+    // search and verify we found our doc
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    XYPolygon a = new XYPolygon(new float[] { 28, 28, 29, 29, 28 },
+                           new float[] { -56, -55, -55, -56, -56 });
+    XYPolygon b = new XYPolygon(new float[] { 18, 18, 19, 19, 18 },
+                            new float[] { -66, -65, -65, -66, -66 });
+    assertEquals(1, searcher.count(newPolygonQuery("field", a, b)));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+  
+  /** null field name not allowed */
+  public void testPolygonNullField() {
+    assumeTrue("Impl does not support polygons", supportsPolygons());
+    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
+      newPolygonQuery(null, new XYPolygon(
+          new float[] { 18, 18, 19, 19, 18 },
+          new float[] { -66, -65, -65, -66, -66 }));
+    });
+    assertTrue(expected.getMessage().contains("field must not be null"));
+  }
+
+  // A particularly tricky adversary for BKD tree:
+  public void testSamePointManyTimes() throws Exception {
+    int numPoints = atLeast(1000);
+
+    // Every doc has 2 points:
+    float theX = nextX();
+    float theY = nextY();
+
+    float[] xs = new float[numPoints];
+    Arrays.fill(xs, theX);
+
+    float[] ys = new float[numPoints];
+    Arrays.fill(ys, theY);
+
+    verify(xs, ys);
+  }
+
+  // A particularly tricky adversary for BKD tree:
+  public void testLowCardinality() throws Exception {
+    int numPoints = atLeast(1000);
+    int cardinality = TestUtil.nextInt(random(), 2, 20);
+
+    float[] diffXs  = new float[cardinality];
+    float[] diffYs = new float[cardinality];
+    for (int i = 0; i< cardinality; i++) {
+      diffXs[i] = nextX();
+      diffYs[i] = nextY();
+    }
+
+    float[] xs = new float[numPoints];
+    float[] ys = new float[numPoints];
+    for (int i = 0; i < numPoints; i++) {
+      int index = random().nextInt(cardinality);
+      xs[i] = diffXs[index];
+      ys[i] = diffYs[index];
+    }
+    verify(xs, ys);
+  }
+
+  public void testAllYEqual() throws Exception {
+    int numPoints = atLeast(1000);
+    float y = nextY();
+    float[] xs = new float[numPoints];
+    float[] ys = new float[numPoints];
+
+    boolean haveRealDoc = false;
+
+    for(int docID=0;docID<numPoints;docID++) {
+      int x = random().nextInt(20);
+      if (x == 17) {
+        // Some docs don't have a point:
+        ys[docID] = Float.NaN;
+        if (VERBOSE) {
+          System.out.println("  doc=" + docID + " is missing");
+        }
+        continue;
+      }
+
+      if (docID > 0 && x == 14 && haveRealDoc) {
+        int oldDocID;
+        while (true) {
+          oldDocID = random().nextInt(docID);
+          if (Float.isNaN(ys[oldDocID]) == false) {
+            break;
+          }
+        }
+            
+        // Fully identical point:
+        ys[docID] = xs[oldDocID];
+        if (VERBOSE) {
+          System.out.println("  doc=" + docID + " y=" + y + " x=" + xs[docID] + " (same x/y as doc=" + oldDocID + ")");
+        }
+      } else {
+        xs[docID] = nextX();
+        haveRealDoc = true;
+        if (VERBOSE) {
+          System.out.println("  doc=" + docID + " y=" + y + " x=" + xs[docID]);
+        }
+      }
+      ys[docID] = y;
+    }
+
+    verify(xs, ys);
+  }
+
+  public void testAllXEqual() throws Exception {
+    int numPoints = atLeast(1000);
+    float theX = nextX();
+    float[] xs = new float[numPoints];
+    float[] ys = new float[numPoints];
+
+    boolean haveRealDoc = false;
+
+    for(int docID=0;docID<numPoints;docID++) {
+      int x = random().nextInt(20);
+      if (x == 17) {
+        // Some docs don't have a point:
+        ys[docID] = Float.NaN;
+        if (VERBOSE) {
+          System.out.println("  doc=" + docID + " is missing");
+        }
+        continue;
+      }
+
+      if (docID > 0 && x == 14 && haveRealDoc) {
+        int oldDocID;
+        while (true) {
+          oldDocID = random().nextInt(docID);
+          if (Float.isNaN(ys[oldDocID]) == false) {
+            break;
+          }
+        }
+            
+        // Fully identical point:
+        ys[docID] = ys[oldDocID];
+        if (VERBOSE) {
+          System.out.println("  doc=" + docID + " y=" + ys[docID] + " x=" + theX + " (same X/y as doc=" + oldDocID + ")");
+        }
+      } else {
+        ys[docID] = nextY();
+        haveRealDoc = true;
+        if (VERBOSE) {
+          System.out.println("  doc=" + docID + " y=" + ys[docID] + " x=" + theX);
+        }
+      }
+      xs[docID] = theX;
+    }
+
+    verify(xs, ys);
+  }
+
+  public void testMultiValued() throws Exception {
+    int numPoints = atLeast(1000);
+    // Every doc has 2 points:
+    float[] xs = new float[2*numPoints];
+    float[] ys = new float[2*numPoints];
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig();
+
+    // We rely on docID order:
+    iwc.setMergePolicy(newLogMergePolicy());
+    // and on seeds being able to reproduce:
+    iwc.setMergeScheduler(new SerialMergeScheduler());
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+
+    for (int id=0;id<numPoints;id++) {
+      Document doc = new Document();
+      xs[2*id] = nextX();
+      ys[2*id] = nextY();
+      doc.add(newStringField("id", ""+id, Field.Store.YES));
+      addPointToDoc(FIELD_NAME, doc, xs[2*id], ys[2*id]);
+      xs[2*id+1] = nextX();
+      ys[2*id+1] = nextY();
+      addPointToDoc(FIELD_NAME, doc, xs[2*id+1], ys[2*id+1]);
+
+      if (VERBOSE) {
+        System.out.println("id=" + id);
+        System.out.println("  x=" + xs[2*id] + " y=" + ys[2*id]);
+        System.out.println("  x=" + xs[2*id+1] + " y=" + ys[2*id+1]);
+      }
+      w.addDocument(doc);
+    }
+
+    // TODO: share w/ verify; just need parallel array of the expected ids
+    if (random().nextBoolean()) {
+      w.forceMerge(1);
+    }
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+
+    int iters = atLeast(25);
+    for (int iter=0;iter<iters;iter++) {
+      XYRectangle rect = nextBox();
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter + " rect=" + rect);
+      }
+
+      Query query = newRectQuery(FIELD_NAME, rect.minX, rect.maxX, rect.minY, rect.maxY);
+
+      final FixedBitSet hits = new FixedBitSet(r.maxDoc());
+      s.search(query, new SimpleCollector() {
+
+          private int docBase;
+
+          @Override
+          public ScoreMode scoreMode() {
+            return ScoreMode.COMPLETE_NO_SCORES;
+          }
+
+          @Override
+          protected void doSetNextReader(LeafReaderContext context) throws IOException {
+            docBase = context.docBase;
+          }
+
+          @Override
+          public void collect(int doc) {
+            hits.set(docBase+doc);
+          }
+        });
+
+      boolean fail = false;
+
+      for(int docID=0;docID<ys.length/2;docID++) {
+        float yDoc1 = ys[2*docID];
+        float xDoc1 = xs[2*docID];
+        float yDoc2 = ys[2*docID+1];
+        float xDoc2 = xs[2*docID+1];
+        
+        boolean result1 = rectContainsPoint(rect, xDoc1, yDoc1);
+        boolean result2 = rectContainsPoint(rect, xDoc2, yDoc2);
+
+        boolean expected = result1 || result2;
+
+        if (hits.get(docID) != expected) {
+          String id = s.doc(docID).get("id");
+          if (expected) {
+            System.out.println("TEST: id=" + id + " docID=" + docID + " should match but did not");
+          } else {
+            System.out.println("TEST: id=" + id + " docID=" + docID + " should not match but did");
+          }
+          System.out.println("  rect=" + rect);
+          System.out.println("  x=" + xDoc1 + " y=" + yDoc1 + "\n  x=" + xDoc2 + " y" + yDoc2);
+          System.out.println("  result1=" + result1 + " result2=" + result2);
+          fail = true;
+        }
+      }
+
+      if (fail) {
+        fail("some hits were wrong");
+      }
+    }
+    r.close();
+    dir.close();
+  }
+
+  public void testRandomTiny() throws Exception {
+    // Make sure single-leaf-node case is OK:
+    doTestRandom(10);
+  }
+
+  public void testRandomMedium() throws Exception {
+    doTestRandom(1000);
+  }
+
+  @Nightly
+  public void testRandomBig() throws Exception {
+    assumeFalse("Direct codec can OOME on this test", TestUtil.getDocValuesFormat(FIELD_NAME).equals("Direct"));
+    doTestRandom(200000);
+  }
+
+  private void doTestRandom(int count) throws Exception {
+
+    int numPoints = atLeast(count);
+
+    if (VERBOSE) {
+      System.out.println("TEST: numPoints=" + numPoints);
+    }
+
+    float[] xs = new float[numPoints];
+    float[] ys = new float[numPoints];
+
+    boolean haveRealDoc = false;
+
+    for (int id=0;id<numPoints;id++) {
+      int x = random().nextInt(20);
+      if (x == 17) {
+        // Some docs don't have a point:
+        ys[id] = Float.NaN;
+        if (VERBOSE) {
+          System.out.println("  id=" + id + " is missing");
+        }
+        continue;
+      }
+
+      if (id > 0 && x < 3 && haveRealDoc) {
+        int oldID;
+        while (true) {
+          oldID = random().nextInt(id);
+          if (Float.isNaN(ys[oldID]) == false) {
+            break;
+          }
+        }
+            
+        if (x == 0) {
+          // Identical x to old point
+          ys[id] = ys[oldID];
+          xs[id] = nextX();
+          if (VERBOSE) {
+            System.out.println("  id=" + id + " x=" + xs[id] + " y=" + ys[id] + " (same x as doc=" + oldID + ")");
+          }
+        } else if (x == 1) {
+          // Identical y to old point
+          ys[id] = nextY();
+          xs[id] = xs[oldID];
+          if (VERBOSE) {
+            System.out.println("  id=" + id + " x=" + xs[id] + " y=" + ys[id] + " (same y as doc=" + oldID + ")");
+          }
+        } else {
+          assert x == 2;
+          // Fully identical point:
+          xs[id] = xs[oldID];
+          ys[id] = ys[oldID];
+          if (VERBOSE) {
+            System.out.println("  id=" + id + " x=" + xs[id] + " y=" + ys[id] + " (same X/y as doc=" + oldID + ")");
+          }
+        }
+      } else {
+        xs[id] = nextX();
+        ys[id] = nextY();
+        haveRealDoc = true;
+        if (VERBOSE) {
+          System.out.println("  id=" + id + " x=" + xs[id] + " y=" + ys[id]);
+        }
+      }
+    }
+
+    verify(xs, ys);
+  }
+
+
+  protected abstract void addPointToDoc(String field, Document doc, float x, float y);
+
+  protected abstract Query newRectQuery(String field, float minX, float maxX, float minY, float maxY);
+
+  protected abstract Query newDistanceQuery(String field, float centerX, float centerY, float radius);
+
+  protected abstract Query newPolygonQuery(String field, XYPolygon... polygon);
+
+  static final boolean rectContainsPoint(XYRectangle rect, double x, double y) {
+    if (y < rect.minY || y > rect.maxY) {
+      return false;
+    }
+    return x >= rect.minX && x <= rect.maxX;
+  }
+
+  static double cartesianDistance(double x1, double y1, double x2, double y2) {
+    final double diffX = x1 - x2;
+    final double diffY = y1 - y2;
+    return Math.sqrt(diffX * diffX + diffY * diffY);
+  }
+
+  private void verify(float[] xs, float[] ys) throws Exception {
+    // NaN means missing for the doc!!!!!
+    verifyRandomRectangles(xs, ys);
+    verifyRandomDistances(xs, ys);
+    if (supportsPolygons()) {
+      verifyRandomPolygons(xs, ys);
+    }
+  }
+
+  protected void verifyRandomRectangles(float[] xs, float[] ys) throws Exception {
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    // Else seeds may not reproduce:
+    iwc.setMergeScheduler(new SerialMergeScheduler());
+    // Else we can get O(N^2) merging:
+    int mbd = iwc.getMaxBufferedDocs();
+    if (mbd != -1 && mbd < xs.length/100) {
+      iwc.setMaxBufferedDocs(xs.length/100);
+    }
+    Directory dir;
+    if (xs.length > 100000) {
+      dir = newFSDirectory(createTempDir(getClass().getSimpleName()));
+    } else {
+      dir = newDirectory();
+    }
+
+    Set<Integer> deleted = new HashSet<>();
+    // RandomIndexWriter is too slow here:
+    IndexWriter w = new IndexWriter(dir, iwc);
+    for(int id=0;id<xs.length;id++) {
+      Document doc = new Document();
+      doc.add(newStringField("id", ""+id, Field.Store.NO));
+      doc.add(new NumericDocValuesField("id", id));
+      if (Float.isNaN(xs[id]) == false && Float.isNaN(ys[id]) == false ) {
+        addPointToDoc(FIELD_NAME, doc, xs[id], ys[id]);
+      }
+      w.addDocument(doc);
+      if (id > 0 && random().nextInt(100) == 42) {
+        int idToDelete = random().nextInt(id);
+        w.deleteDocuments(new Term("id", ""+idToDelete));
+        deleted.add(idToDelete);
+        if (VERBOSE) {
+          System.out.println("  delete id=" + idToDelete);
+        }
+      }
+    }
+
+    if (random().nextBoolean()) {
+      w.forceMerge(1);
+    }
+    final IndexReader r = DirectoryReader.open(w);
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+
+    int iters = atLeast(25);
+
+    Bits liveDocs = MultiBits.getLiveDocs(s.getIndexReader());
+    int maxDoc = s.getIndexReader().maxDoc();
+
+    for (int iter=0;iter<iters;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter + " s=" + s);
+      }
+      
+      XYRectangle rect = nextBox();
+
+      Query query = newRectQuery(FIELD_NAME, rect.minX, rect.maxX, rect.minY, rect.maxY);
+
+      if (VERBOSE) {
+        System.out.println("  query=" + query);
+      }
+
+      final FixedBitSet hits = new FixedBitSet(maxDoc);
+      s.search(query, new SimpleCollector() {
+
+          private int docBase;
+
+          @Override
+          public ScoreMode scoreMode() {
+            return ScoreMode.COMPLETE_NO_SCORES;
+          }
+
+          @Override
+          protected void doSetNextReader(LeafReaderContext context) {
+            docBase = context.docBase;
+          }
+
+          @Override
+          public void collect(int doc) {
+            hits.set(docBase+doc);
+          }
+        });
+
+      boolean fail = false;
+      NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
+      for(int docID=0;docID<maxDoc;docID++) {
+        assertEquals(docID, docIDToID.nextDoc());
+        int id = (int) docIDToID.longValue();
+        boolean expected;
+        if (liveDocs != null && liveDocs.get(docID) == false) {
+          // document is deleted
+          expected = false;
+        } else if (Float.isNaN(xs[id]) || Float.isNaN(ys[id])) {
+          expected = false;
+        } else {
+          expected = rectContainsPoint(rect, xs[id], ys[id]);
+        }
+
+        if (hits.get(docID) != expected) {
+          StringBuilder b = new StringBuilder();
+          b.append("docID=(").append(docID).append(")\n");
+
+          if (expected) {
+            b.append("FAIL: id=").append(id).append(" should match but did not\n");
+          } else {
+            b.append("FAIL: id=").append(id).append(" should not match but did\n");
+          }
+          b.append("  box=").append(rect).append("\n");
+          b.append("  query=").append(query).append(" docID=").append(docID).append("\n");
+          b.append("  x=").append(xs[id]).append(" y=").append(ys[id]).append("\n");
+          b.append("  deleted?=").append(liveDocs != null && liveDocs.get(docID) == false);
+          if (true) {
+            fail("wrong hit (first of possibly more):\n\n" + b);
+          } else {
+            System.out.println(b.toString());
+            fail = true;
+          }
+        }
+      }
+      if (fail) {
+        fail("some hits were wrong");
+      }
+    }
+
+    IOUtils.close(r, dir);
+  }
+
+  protected void verifyRandomDistances(float[] xs, float[] ys) throws Exception {
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    // Else seeds may not reproduce:
+    iwc.setMergeScheduler(new SerialMergeScheduler());
+    // Else we can get O(N^2) merging:
+    int mbd = iwc.getMaxBufferedDocs();
+    if (mbd != -1 && mbd < xs.length/100) {
+      iwc.setMaxBufferedDocs(xs.length/100);
+    }
+    Directory dir;
+    if (xs.length > 100000) {
+      dir = newFSDirectory(createTempDir(getClass().getSimpleName()));
+    } else {
+      dir = newDirectory();
+    }
+
+    Set<Integer> deleted = new HashSet<>();
+    // RandomIndexWriter is too slow here:
+    IndexWriter w = new IndexWriter(dir, iwc);
+    for(int id=0;id<xs.length;id++) {
+      Document doc = new Document();
+      doc.add(newStringField("id", ""+id, Field.Store.NO));
+      doc.add(new NumericDocValuesField("id", id));
+      if (Float.isNaN(xs[id]) == false && Float.isNaN(ys[id]) == false) {
+        addPointToDoc(FIELD_NAME, doc, xs[id], ys[id]);
+      }
+      w.addDocument(doc);
+      if (id > 0 && random().nextInt(100) == 42) {
+        int idToDelete = random().nextInt(id);
+        w.deleteDocuments(new Term("id", ""+idToDelete));
+        deleted.add(idToDelete);
+        if (VERBOSE) {
+          System.out.println("  delete id=" + idToDelete);
+        }
+      }
+    }
+
+    if (random().nextBoolean()) {
+      w.forceMerge(1);
+    }
+    final IndexReader r = DirectoryReader.open(w);
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+
+    int iters = atLeast(25);
+
+    Bits liveDocs = MultiBits.getLiveDocs(s.getIndexReader());
+    int maxDoc = s.getIndexReader().maxDoc();
+
+    for (int iter=0;iter<iters;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter + " s=" + s);
+      }
+
+      // Distance
+      final float centerX = nextX();
+      final float centerY = nextY();
+
+      // So the query can cover at most 50% of the cartesian space:
+      final float radius = random().nextFloat() * Float.MAX_VALUE / 2;
+
+      if (VERBOSE) {
+        final DecimalFormat df = new DecimalFormat("#,###.00", DecimalFormatSymbols.getInstance(Locale.ENGLISH));
+        System.out.println("  radius = " + df.format(radius));
+      }
+
+      Query query = newDistanceQuery(FIELD_NAME, centerX, centerY, radius);
+
+      if (VERBOSE) {
+        System.out.println("  query=" + query);
+      }
+
+      final FixedBitSet hits = new FixedBitSet(maxDoc);
+      s.search(query, new SimpleCollector() {
+
+          private int docBase;
+
+          @Override
+          public ScoreMode scoreMode() {
+            return ScoreMode.COMPLETE_NO_SCORES;
+          }
+
+          @Override
+          protected void doSetNextReader(LeafReaderContext context) throws IOException {
+            docBase = context.docBase;
+          }
+
+          @Override
+          public void collect(int doc) {
+            hits.set(docBase+doc);
+          }
+        });
+
+      boolean fail = false;
+      NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
+      for(int docID=0;docID<maxDoc;docID++) {
+        assertEquals(docID, docIDToID.nextDoc());
+        int id = (int) docIDToID.longValue();
+        boolean expected;
+        if (liveDocs != null && liveDocs.get(docID) == false) {
+          // document is deleted
+          expected = false;
+        } else if (Float.isNaN(xs[id]) || Float.isNaN(ys[id])) {
+          expected = false;
+        } else {
+          expected = cartesianDistance(centerX, centerY, xs[id], ys[id]) <= radius;
+        }
+
+        if (hits.get(docID) != expected) {
+          StringBuilder b = new StringBuilder();
+
+          if (expected) {
+            b.append("FAIL: id=").append(id).append(" should match but did not\n");
+          } else {
+            b.append("FAIL: id=").append(id).append(" should not match but did\n");
+          }
+          b.append("  query=").append(query).append(" docID=").append(docID).append("\n");
+          b.append("  x=").append(xs[id]).append(" y=").append(ys[id]).append("\n");
+          b.append("  deleted?=").append(liveDocs != null && liveDocs.get(docID) == false);
+          if (Double.isNaN(xs[id]) == false) {
+            double distance = cartesianDistance(centerX, centerY, xs[id], ys[id]);
+            b.append("  centerX=").append(centerX).append(" centerY=").append(centerY).append(" distance=").append(distance).append(" vs radius=").append(radius);
+          }
+          if (true) {
+            fail("wrong hit (first of possibly more):\n\n" + b);
+          } else {
+            System.out.println(b.toString());
+            fail = true;
+          }
+        }
+      }
+      if (fail) {
+        fail("some hits were wrong");
+      }
+    }
+
+    IOUtils.close(r, dir);
+  }
+
+  protected void verifyRandomPolygons(float[] xs, float[] ys) throws Exception {
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    // Else seeds may not reproduce:
+    iwc.setMergeScheduler(new SerialMergeScheduler());
+    // Else we can get O(N^2) merging:
+    int mbd = iwc.getMaxBufferedDocs();
+    if (mbd != -1 && mbd < xs.length/100) {
+      iwc.setMaxBufferedDocs(xs.length/100);
+    }
+    Directory dir;
+    if (xs.length > 100000) {
+      dir = newFSDirectory(createTempDir(getClass().getSimpleName()));
+    } else {
+      dir = newDirectory();
+    }
+
+    Set<Integer> deleted = new HashSet<>();
+    // RandomIndexWriter is too slow here:
+    IndexWriter w = new IndexWriter(dir, iwc);
+    for(int id=0;id<xs.length;id++) {
+      Document doc = new Document();
+      doc.add(newStringField("id", ""+id, Field.Store.NO));
+      doc.add(new NumericDocValuesField("id", id));
+      if (Float.isNaN(xs[id]) == false && Float.isNaN(ys[id]) == false) {
+        addPointToDoc(FIELD_NAME, doc, xs[id], ys[id]);
+      }
+      w.addDocument(doc);
+      if (id > 0 && random().nextInt(100) == 42) {
+        int idToDelete = random().nextInt(id);
+        w.deleteDocuments(new Term("id", ""+idToDelete));
+        deleted.add(idToDelete);
+        if (VERBOSE) {
+          System.out.println("  delete id=" + idToDelete);
+        }
+      }
+    }
+
+    if (random().nextBoolean()) {
+      w.forceMerge(1);
+    }
+    final IndexReader r = DirectoryReader.open(w);
+    w.close();
+
+    // We can't wrap with "exotic" readers because points needs to work:
+    IndexSearcher s = newSearcher(r);
+
+    final int iters = atLeast(75);
+
+    Bits liveDocs = MultiBits.getLiveDocs(s.getIndexReader());
+    int maxDoc = s.getIndexReader().maxDoc();
+
+    for (int iter=0;iter<iters;iter++) {
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter + " s=" + s);
+      }
+
+      // Polygon
+      XYPolygon polygon = nextPolygon();
+      Query query = newPolygonQuery(FIELD_NAME, polygon);
+
+      if (VERBOSE) {
+        System.out.println("  query=" + query);
+      }
+
+      final FixedBitSet hits = new FixedBitSet(maxDoc);
+      s.search(query, new SimpleCollector() {
+
+          private int docBase;
+
+          @Override
+          public ScoreMode scoreMode() {
+            return ScoreMode.COMPLETE_NO_SCORES;
+          }
+
+          @Override
+          protected void doSetNextReader(LeafReaderContext context) throws IOException {
+            docBase = context.docBase;
+          }
+
+          @Override
+          public void collect(int doc) {
+            hits.set(docBase+doc);
+          }
+        });
+
+      boolean fail = false;
+      NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
+      for(int docID=0;docID<maxDoc;docID++) {
+        assertEquals(docID, docIDToID.nextDoc());
+        int id = (int) docIDToID.longValue();
+        boolean expected;
+        if (liveDocs != null && liveDocs.get(docID) == false) {
+          // document is deleted
+          expected = false;
+        } else if (Float.isNaN(xs[id]) || Float.isNaN(ys[id])) {
+          expected = false;
+        } else {
+          expected = ShapeTestUtil.containsSlowly(polygon, xs[id], ys[id]);
+        }
+
+        if (hits.get(docID) != expected) {
+          StringBuilder b = new StringBuilder();
+
+          if (expected) {
+            b.append("FAIL: id=").append(id).append(" should match but did not\n");
+          } else {
+            b.append("FAIL: id=").append(id).append(" should not match but did\n");
+          }
+          b.append("  query=").append(query).append(" docID=").append(docID).append("\n");
+          b.append("  x=").append(xs[id]).append(" y=").append(ys[id]).append("\n");
+          b.append("  deleted?=").append(liveDocs != null && liveDocs.get(docID) == false);
+          b.append("  polygon=").append(polygon);
+          if (true) {
+            fail("wrong hit (first of possibly more):\n\n" + b);
+          } else {
+            System.out.println(b.toString());
+            fail = true;
+          }
+        }
+      }
+      if (fail) {
+        fail("some hits were wrong");
+      }
+    }
+
+    IOUtils.close(r, dir);
+  }
+
+  public void testRectBoundariesAreInclusive() throws Exception {
+    XYRectangle rect = ShapeTestUtil.nextBox(random());
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    // Else seeds may not reproduce:
+    iwc.setMergeScheduler(new SerialMergeScheduler());
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+    for(int i = 0; i < 3; i++) {
+      float y;
+      if (i == 0) {
+        y = rect.minY;
+      } else if (i == 1) {
+        y = (float) (((double) rect.minY + rect.maxY) / 2.0);
+      } else {
+        y = rect.maxY;
+      }
+      for(int j = 0; j < 3; j++) {
+        float x;
+        if (j == 0) {
+          x = rect.minX;
+        } else if (j == 1) {
+          if (i == 1) {
+            continue;
+          }
+          x = (float) (((double) rect.minX + rect.maxX) / 2.0);
+        } else {
+          x = rect.maxX;
+        }
+
+        Document doc = new Document();
+        addPointToDoc(FIELD_NAME, doc, x, y);
+        w.addDocument(doc);
+      }
+    }
+    IndexReader r = w.getReader();
+    IndexSearcher s = newSearcher(r, false);
+    // exact edge cases
+    assertEquals(8, s.count(newRectQuery(FIELD_NAME, rect.minX, rect.maxX, rect.minY, rect.maxY)));
+    // expand 1 ulp in each direction if possible and test a slightly larger box!
+    if (rect.minX != -Float.MAX_VALUE) {
+      assertEquals(8, s.count(newRectQuery(FIELD_NAME, Math.nextDown(rect.minX), rect.maxX, rect.minY, rect.maxY)));
+    }
+    if (rect.maxX != Float.MAX_VALUE) {
+      assertEquals(8, s.count(newRectQuery(FIELD_NAME, rect.minX, Math.nextUp(rect.maxX), rect.minY, rect.maxY)));
+    }
+    if (rect.minY != -Float.MAX_VALUE) {
+      assertEquals(8, s.count(newRectQuery(FIELD_NAME, rect.minX, rect.maxX, Math.nextDown(rect.minY), rect.maxY)));
+    }
+    if (rect.maxY != Float.MAX_VALUE) {
+      assertEquals(8, s.count(newRectQuery(FIELD_NAME, rect.minX, rect.maxX, rect.minY, Math.nextUp(rect.maxY))));
+    }
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  /** Run a few iterations with just 10 docs, hopefully easy to debug */
+  public void testRandomDistance() throws Exception {
+    int numIters = atLeast(1);
+    for (int iters = 0; iters < numIters; iters++) {
+      doRandomDistanceTest(10, 100);
+    }
+  }
+
+  /** Runs with thousands of docs */
+  @Nightly
+  public void testRandomDistanceHuge() throws Exception {
+    for (int iters = 0; iters < 10; iters++) {
+      doRandomDistanceTest(2000, 100);
+    }
+  }
+
+  private void doRandomDistanceTest(int numDocs, int numQueries) throws IOException {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    // Else seeds may not reproduce:
+    iwc.setMergeScheduler(new SerialMergeScheduler());
+    int pointsInLeaf = 2 + random().nextInt(4);
+    iwc.setCodec(new FilterCodec("Lucene84", TestUtil.getDefaultCodec()) {
+      @Override
+      public PointsFormat pointsFormat() {
+        return new PointsFormat() {
+          @Override
+          public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
+            return new Lucene60PointsWriter(writeState, pointsInLeaf, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
+          }
+
+          @Override
+          public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
+            return new Lucene60PointsReader(readState);
+          }
+        };
+      }
+    });
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+
+    for (int i = 0; i < numDocs; i++) {
+      float x = nextX();
+      float y = nextY();
+      // pre-normalize up front, so we can just use quantized value for testing and do simple exact comparisons
+
+      Document doc = new Document();
+      addPointToDoc("field", doc, x, y);
+      doc.add(new StoredField("x", x));
+      doc.add(new StoredField("y", y));
+      writer.addDocument(doc);
+    }
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+
+    for (int i = 0; i < numQueries; i++) {
+      float x = nextX();
+      float y = nextY();
+      float radius = (Float.MAX_VALUE / 2) * random().nextFloat();
+
+      BitSet expected = new BitSet();
+      for (int doc = 0; doc < reader.maxDoc(); doc++) {
+        float docX = reader.document(doc).getField("x").numericValue().floatValue();
+        float docY = reader.document(doc).getField("y").numericValue().floatValue();
+        double distance = cartesianDistance(x, y, docX, docY);
+        if (distance <= radius) {
+          expected.set(doc);
+        }
+      }
+
+      TopDocs topDocs = searcher.search(newDistanceQuery("field", x, y, radius), reader.maxDoc(), Sort.INDEXORDER);
+      BitSet actual = new BitSet();
+      for (ScoreDoc doc : topDocs.scoreDocs) {
+        actual.set(doc.doc);
+      }
+
+      try {
+        assertEquals(expected, actual);
+      } catch (AssertionError e) {
+        System.out.println("center: (" + x + "," + y + "), radius=" + radius);
+        for (int doc = 0; doc < reader.maxDoc(); doc++) {
+          float docX = reader.document(doc).getField("x").numericValue().floatValue();
+          float docY = reader.document(doc).getField("y").numericValue().floatValue();
+          double distance = cartesianDistance(x, y, docX, docY);
+          System.out.println("" + doc + ": (" + x + "," + y + "), distance=" + distance);
+        }
+        throw e;
+      }
+    }
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+
+  public void testEquals()  {
+    Query q1, q2;
+
+    XYRectangle rect = nextBox();
+
+    q1 = newRectQuery("field", rect.minX, rect.maxX, rect.minY, rect.maxY);
+    q2 = newRectQuery("field", rect.minX, rect.maxX, rect.minY, rect.maxY);
+    assertEquals(q1, q2);
+
+
+    float x = nextX();
+    float y = nextY();
+    q1 = newDistanceQuery("field", x, y, 10000.0f);
+    q2 = newDistanceQuery("field", x, y, 10000.0f);
+    assertEquals(q1, q2);
+    assertFalse(q1.equals(newDistanceQuery("field2", x, y, 10000.0f)));
+
+    float[] xs = new float[5];
+    float[] ys = new float[5];
+    xs[0] = rect.minX;
+    ys[0] = rect.minY;
+    xs[1] = rect.maxX;
+    ys[1] = rect.minY;
+    xs[2] = rect.maxX;
+    ys[2] = rect.maxY;
+    xs[3] = rect.minX;
+    ys[3] = rect.maxY;
+    xs[4] = rect.minX;
+    ys[4] = rect.minY;
+    if (supportsPolygons()) {
+      q1 = newPolygonQuery("field", new XYPolygon(xs, ys));
+      q2 = newPolygonQuery("field", new XYPolygon(xs, ys));
+      assertEquals(q1, q2);
+      assertFalse(q1.equals(newPolygonQuery("field2", new XYPolygon(xs, ys))));
+    }
+  }
+
+  /** return topdocs over a small set of points in field "point" */
+  private TopDocs searchSmallSet(Query query, int size) throws Exception {
+    // this is a simple systematic test, indexing these points
+    double[][] pts = new double[][] {
+        { 32.763420,          -96.774             },
+        { 32.7559529921407,   -96.7759895324707   },
+        { 32.77866942010977,  -96.77701950073242  },
+        { 32.7756745755423,   -96.7706036567688   },
+        { 27.703618681345585, -139.73458170890808 },
+        { 32.94823588839368,  -96.4538113027811   },
+        { 33.06047141970814,  -96.65084838867188  },
+        { 32.778650,          -96.7772            },
+        { -88.56029371730983, -177.23537676036358 },
+        { 33.541429799076354, -26.779373834241003 },
+        { 26.774024500421728, -77.35379276106497  },
+        { -90.0,              -14.796283808944777 },
+        { 32.94823588839368,  -178.8538113027811  },
+        { 32.94823588839368,  178.8538113027811   },
+        { 40.720611,          -73.998776          },
+        { -44.5,              -179.5              }
+    };
+
+    Directory directory = newDirectory();
+
+    // TODO: must these simple tests really rely on docid order?
+    IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
+    iwc.setMaxBufferedDocs(TestUtil.nextInt(random(), 100, 1000));
+    iwc.setMergePolicy(newLogMergePolicy());
+    // Else seeds may not reproduce:
+    iwc.setMergeScheduler(new SerialMergeScheduler());
+    RandomIndexWriter writer = new RandomIndexWriter(random(), directory, iwc);
+
+    for (double p[] : pts) {
+        Document doc = new Document();
+        addPointToDoc("point", doc, (float) p[0], (float) p[1]);
+        writer.addDocument(doc);
+    }
+
+    // add explicit multi-valued docs
+    for (int i=0; i<pts.length; i+=2) {
+      Document doc = new Document();
+      addPointToDoc("point", doc, (float) pts[i][0], (float) pts[i][1]);
+      addPointToDoc("point", doc, (float) pts[i+1][0], (float) pts[i+1][1]);
+      writer.addDocument(doc);
+    }
+
+    // index random string documents
+    for (int i=0; i<random().nextInt(10); ++i) {
+      Document doc = new Document();
+      doc.add(new StringField("string", Integer.toString(i), Field.Store.NO));
+      writer.addDocument(doc);
+    }
+
+    IndexReader reader = writer.getReader();
+    writer.close();
+
+    IndexSearcher searcher = newSearcher(reader);
+    TopDocs topDocs = searcher.search(query, size);
+    reader.close();
+    directory.close();
+    return topDocs;
+  }
+
+  public void testSmallSetRect() throws Exception {
+    TopDocs td = searchSmallSet(newRectQuery("point", 32.778f, 32.779f, -96.778f, -96.777f), 5);
+    assertEquals(4, td.totalHits.value);
+  }
+
+  public void testSmallSetRect2() throws Exception {
+    TopDocs td = searchSmallSet(newRectQuery("point", -45.0f, -44.0f, -180.0f, 180.0f), 20);
+    assertEquals(2, td.totalHits.value);
+  }
+
+  public void testSmallSetMultiValued() throws Exception {
+    TopDocs td = searchSmallSet(newRectQuery("point", 32.755f, 32.776f, -180f, 180.770f), 20);
+    // 3 single valued docs + 2 multi-valued docs
+    assertEquals(5, td.totalHits.value);
+  }
+
+  public void testSmallSetWholeSpace() throws Exception {
+    TopDocs td = searchSmallSet(newRectQuery("point", -Float.MAX_VALUE, Float.MAX_VALUE, -Float.MAX_VALUE, Float.MAX_VALUE), 20);
+    assertEquals(24, td.totalHits.value);
+  }
+
+  public void testSmallSetPoly() throws Exception {
+    assumeTrue("Impl does not support polygons", supportsPolygons());
+    TopDocs td = searchSmallSet(newPolygonQuery("point",
+        new XYPolygon(
+        new float[]{33.073130f, 32.9942669f, 32.938386f, 33.0374494f,
+            33.1369762f, 33.1162747f, 33.073130f, 33.073130f},
+        new float[]{-96.7682647f, -96.8280029f, -96.6288757f, -96.4929199f,
+                     -96.6041564f, -96.7449188f, -96.76826477f, -96.7682647f})),
+        5);
+    assertEquals(2, td.totalHits.value);
+  }
+
+  public void testSmallSetPolyWholeSpace() throws Exception {
+    assumeTrue("Impl does not support polygons", supportsPolygons());
+    TopDocs td = searchSmallSet(newPolygonQuery("point",
+                      new XYPolygon(
+                      new float[] {-Float.MAX_VALUE, Float.MAX_VALUE, Float.MAX_VALUE, -Float.MAX_VALUE, -Float.MAX_VALUE},
+                      new float[] {-Float.MAX_VALUE, -Float.MAX_VALUE, Float.MAX_VALUE, Float.MAX_VALUE, -Float.MAX_VALUE})),
+                      20);
+    assertEquals("testWholeMap failed", 24, td.totalHits.value);
+  }
+
+  public void testSmallSetDistance() throws Exception {
+    TopDocs td = searchSmallSet(newDistanceQuery("point", 32.94823588839368f, -96.4538113027811f, 6.0f), 20);
+    assertEquals(11, td.totalHits.value);
+  }
+
+  public void testSmallSetTinyDistance() throws Exception {
+    TopDocs td = searchSmallSet(newDistanceQuery("point", 40.720611f, -73.998776f, 0.1f), 20);
+    assertEquals(2, td.totalHits.value);
+  }
+
+  /**
+   * Explicitly large
+   */
+  public void testSmallSetHugeDistance() throws Exception {
+    TopDocs td = searchSmallSet(newDistanceQuery("point", 32.94823588839368f, -96.4538113027811f, Float.MAX_VALUE), 20);
+    assertEquals(24, td.totalHits.value);
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/geo/ShapeTestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/geo/ShapeTestUtil.java
similarity index 66%
rename from lucene/core/src/test/org/apache/lucene/geo/ShapeTestUtil.java
rename to lucene/test-framework/src/java/org/apache/lucene/geo/ShapeTestUtil.java
index 51b7a70..47cd3bd 100644
--- a/lucene/core/src/test/org/apache/lucene/geo/ShapeTestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/geo/ShapeTestUtil.java
@@ -21,8 +21,6 @@ import java.util.Random;
 
 import com.carrotsearch.randomizedtesting.RandomizedContext;
 import com.carrotsearch.randomizedtesting.generators.BiasedNumbers;
-import org.apache.lucene.util.SloppyMath;
-
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -43,7 +41,7 @@ public class ShapeTestUtil {
         try {
           return createRegularPolygon(nextFloat(random), nextFloat(random), radius, gons);
         } catch (IllegalArgumentException iae) {
-          // we tried to cross dateline or pole ... try again
+          // something went wrong, try again
         }
       }
     }
@@ -137,9 +135,7 @@ public class ShapeTestUtil {
   }
 
   private static XYPolygon surpriseMePolygon(Random random) {
-    // repeat until we get a poly that doesn't cross dateline:
     while (true) {
-      //System.out.println("\nPOLY ITER");
       float centerX = nextFloat(random);
       float centerY = nextFloat(random);
       double radius = 0.1 + 20 * random.nextDouble();
@@ -150,7 +146,6 @@ public class ShapeTestUtil {
       double angle = 0.0;
       while (true) {
         angle += random.nextDouble()*40.0;
-        //System.out.println("  angle " + angle);
         if (angle > 360) {
           break;
         }
@@ -160,14 +155,11 @@ public class ShapeTestUtil {
 
         len = StrictMath.min(len, StrictMath.min(maxX, maxY));
 
-        //System.out.println("    len=" + len);
-        float x = (float)(centerX + len * Math.cos(SloppyMath.toRadians(angle)));
-        float y = (float)(centerY + len * Math.sin(SloppyMath.toRadians(angle)));
+        float x = (float)(centerX + len * Math.cos(Math.toRadians(angle)));
+        float y = (float)(centerY + len * Math.sin(Math.toRadians(angle)));
 
         xList.add(x);
         yList.add(y);
-
-        //System.out.println("    lat=" + lats.get(lats.size()-1) + " lon=" + lons.get(lons.size()-1));
       }
 
       // close it
@@ -223,4 +215,67 @@ public class ShapeTestUtil {
   private static Random random() {
     return RandomizedContext.current().getRandom();
   }
+
+  /**
+   * Simple slow point in polygon check (for testing)
+   */
+  // direct port of PNPOLY C code (https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html)
+  // this allows us to improve the code yet still ensure we have its properties
+  // it is under the BSD license (https://www.ecse.rpi.edu/~wrf/Research/Short_Notes/pnpoly.html#License%20to%20Use)
+  //
+  // Copyright (c) 1970-2003, Wm. Randolph Franklin
+  //
+  // Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
+  // documentation files (the "Software"), to deal in the Software without restriction, including without limitation
+  // the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and
+  // to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+  //
+  // 1. Redistributions of source code must retain the above copyright
+  //    notice, this list of conditions and the following disclaimers.
+  // 2. Redistributions in binary form must reproduce the above copyright
+  //    notice in the documentation and/or other materials provided with
+  //    the distribution.
+  // 3. The name of W. Randolph Franklin may not be used to endorse or
+  //    promote products derived from this Software without specific
+  //    prior written permission.
+  //
+  // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED
+  // TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL
+  // THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF
+  // CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS
+  // IN THE SOFTWARE.
+  public static boolean containsSlowly(XYPolygon polygon, double x, double y) {
+    if (polygon.getHoles().length > 0) {
+      throw new UnsupportedOperationException("this testing method does not support holes");
+    }
+    double polyXs[] = XYEncodingUtils.floatArrayToDoubleArray(polygon.getPolyX());
+    double polyYs[] =XYEncodingUtils.floatArrayToDoubleArray(polygon.getPolyY());
+    // bounding box check required due to rounding errors (we don't solve that problem)
+    if (x < polygon.minX || x > polygon.maxX || y < polygon.minY || y > polygon.maxY) {
+      return false;
+    }
+
+    boolean c = false;
+    int i, j;
+    int nvert = polyYs.length;
+    double verty[] = polyYs;
+    double vertx[] = polyXs;
+    double testy = y;
+    double testx = x;
+    for (i = 0, j = 1; j < nvert; ++i, ++j) {
+      if (testy == verty[j] && testy == verty[i] ||
+          ((testy <= verty[j] && testy >= verty[i]) != (testy >= verty[j] && testy <= verty[i]))) {
+        if ((testx == vertx[j] && testx == vertx[i]) ||
+            ((testx <= vertx[j] && testx >= vertx[i]) != (testx >= vertx[j] && testx <= vertx[i]) &&
+                GeoUtils.orient(vertx[i], verty[i], vertx[j], verty[j], testx, testy) == 0)) {
+          // return true if point is on boundary
+          return true;
+        } else if ( ((verty[i] > testy) != (verty[j] > testy)) &&
+            (testx < (vertx[j]-vertx[i]) * (testy-verty[i]) / (verty[j]-verty[i]) + vertx[i]) ) {
+          c = !c;
+        }
+      }
+    }
+    return c;
+  }
 }