You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by nk...@apache.org on 2016/02/05 18:11:34 UTC

[3/6] lucene-solr git commit: LUCENE-6997: refactor sandboxed GeoPointField and query classes to lucene-spatial module

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/665041c5/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInBBoxQueryImpl.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInBBoxQueryImpl.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInBBoxQueryImpl.java
new file mode 100644
index 0000000..bd44b1e
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInBBoxQueryImpl.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.MultiTermQuery;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.SloppyMath;
+import org.apache.lucene.spatial.document.GeoPointField;
+import org.apache.lucene.spatial.util.GeoRelationUtils;
+
+/** Package private implementation for the public facing GeoPointInBBoxQuery delegate class.
+ *
+ *    @lucene.experimental
+ */
+class GeoPointInBBoxQueryImpl extends GeoPointTermQuery {
+  /**
+   * Constructs a new GeoBBoxQuery that will match encoded GeoPoint terms that fall within or on the boundary
+   * of the bounding box defined by the input parameters
+   * @param field the field name
+   * @param minLon lower longitude (x) value of the bounding box
+   * @param minLat lower latitude (y) value of the bounding box
+   * @param maxLon upper longitude (x) value of the bounding box
+   * @param maxLat upper latitude (y) value of the bounding box
+   */
+  GeoPointInBBoxQueryImpl(final String field, final double minLon, final double minLat, final double maxLon, final double maxLat) {
+    super(field, minLon, minLat, maxLon, maxLat);
+  }
+
+  @Override @SuppressWarnings("unchecked")
+  protected TermsEnum getTermsEnum(final Terms terms, AttributeSource atts) throws IOException {
+    return new GeoPointInBBoxTermsEnum(terms.iterator(), minLon, minLat, maxLon, maxLat);
+  }
+
+  @Override
+  public void setRewriteMethod(MultiTermQuery.RewriteMethod method) {
+    throw new UnsupportedOperationException("cannot change rewrite method");
+  }
+
+  protected class GeoPointInBBoxTermsEnum extends GeoPointTermsEnum {
+    protected GeoPointInBBoxTermsEnum(final TermsEnum tenum, final double minLon, final double minLat,
+                            final double maxLon, final double maxLat) {
+      super(tenum, minLon, minLat, maxLon, maxLat);
+    }
+
+    @Override
+    protected short computeMaxShift() {
+      final short shiftFactor;
+
+      // compute diagonal radius
+      double midLon = (minLon + maxLon) * 0.5;
+      double midLat = (minLat + maxLat) * 0.5;
+
+      if (SloppyMath.haversin(minLat, minLon, midLat, midLon)*1000 > 1000000) {
+        shiftFactor = 5;
+      } else {
+        shiftFactor = 4;
+      }
+
+      return (short)(GeoPointField.PRECISION_STEP * shiftFactor);
+    }
+
+    /**
+     * Determine whether the quad-cell crosses the shape
+     */
+    @Override
+    protected boolean cellCrosses(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+      return GeoRelationUtils.rectCrosses(minLon, minLat, maxLon, maxLat, this.minLon, this.minLat, this.maxLon, this.maxLat);
+    }
+
+    /**
+     * Determine whether quad-cell is within the shape
+     */
+    @Override
+    protected boolean cellWithin(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+      return GeoRelationUtils.rectWithin(minLon, minLat, maxLon, maxLat, this.minLon, this.minLat, this.maxLon, this.maxLat);
+    }
+
+    @Override
+    protected boolean cellIntersectsShape(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+      return cellIntersectsMBR(minLon, minLat, maxLon, maxLat);
+    }
+
+    @Override
+    protected boolean postFilter(final double lon, final double lat) {
+      return GeoRelationUtils.pointInRectPrecise(lon, lat, minLon, minLat, maxLon, maxLat);
+    }
+  }
+
+  @Override
+  @SuppressWarnings({"unchecked","rawtypes"})
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+
+    GeoPointInBBoxQueryImpl that = (GeoPointInBBoxQueryImpl) o;
+
+    if (Double.compare(that.maxLat, maxLat) != 0) return false;
+    if (Double.compare(that.maxLon, maxLon) != 0) return false;
+    if (Double.compare(that.minLat, minLat) != 0) return false;
+    if (Double.compare(that.minLon, minLon) != 0) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    long temp;
+    temp = Double.doubleToLongBits(minLon);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    temp = Double.doubleToLongBits(minLat);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    temp = Double.doubleToLongBits(maxLon);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    temp = Double.doubleToLongBits(maxLat);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    return result;
+  }
+
+  @Override
+  public String toString(String field) {
+    final StringBuilder sb = new StringBuilder();
+    sb.append(getClass().getSimpleName());
+    sb.append(':');
+    if (!getField().equals(field)) {
+      sb.append(" field=");
+      sb.append(getField());
+      sb.append(':');
+    }
+    return sb.append(" Lower Left: [")
+        .append(minLon)
+        .append(',')
+        .append(minLat)
+        .append(']')
+        .append(" Upper Right: [")
+        .append(maxLon)
+        .append(',')
+        .append(maxLat)
+        .append("]")
+        .toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/665041c5/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInPolygonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInPolygonQuery.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInPolygonQuery.java
new file mode 100644
index 0000000..b1e864b
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInPolygonQuery.java
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.search;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.spatial.util.GeoRect;
+import org.apache.lucene.spatial.util.GeoRelationUtils;
+import org.apache.lucene.spatial.util.GeoUtils;
+
+/** Implements a simple point in polygon query on a GeoPoint field. This is based on
+ * {@code GeoPointInBBoxQueryImpl} and is implemented using a
+ * three phase approach. First, like {@code GeoPointInBBoxQueryImpl}
+ * candidate terms are queried using a numeric range based on the morton codes
+ * of the min and max lat/lon pairs. Terms passing this initial filter are passed
+ * to a secondary filter that verifies whether the decoded lat/lon point falls within
+ * (or on the boundary) of the bounding box query. Finally, the remaining candidate
+ * term is passed to the final point in polygon check. All value comparisons are subject
+ * to the same precision tolerance defined in {@value org.apache.lucene.spatial.util.GeoUtils#TOLERANCE}
+ *
+ * <p>NOTES:
+ *    1.  The polygon coordinates need to be in either clockwise or counter-clockwise order.
+ *    2.  The polygon must not be self-crossing, otherwise the query may result in unexpected behavior
+ *    3.  All latitude/longitude values must be in decimal degrees.
+ *    4.  Complex computational geometry (e.g., dateline wrapping, polygon with holes) is not supported
+ *    5.  For more advanced GeoSpatial indexing and query operations see spatial module
+ *
+ * @lucene.experimental
+ */
+public final class GeoPointInPolygonQuery extends GeoPointInBBoxQueryImpl {
+  // polygon position arrays - this avoids the use of any objects or
+  // or geo library dependencies
+  private final double[] x;
+  private final double[] y;
+
+  /**
+   * Constructs a new GeoPolygonQuery that will match encoded {@link org.apache.lucene.spatial.document.GeoPointField} terms
+   * that fall within or on the boundary of the polygon defined by the input parameters.
+   */
+  public GeoPointInPolygonQuery(final String field, final double[] polyLons, final double[] polyLats) {
+    this(field, GeoUtils.polyToBBox(polyLons, polyLats), polyLons, polyLats);
+  }
+
+  /** Common constructor, used only internally. */
+  private GeoPointInPolygonQuery(final String field, GeoRect bbox, final double[] polyLons, final double[] polyLats) {
+    super(field, bbox.minLon, bbox.minLat, bbox.maxLon, bbox.maxLat);
+    if (polyLats.length != polyLons.length) {
+      throw new IllegalArgumentException("polyLats and polyLons must be equal length");
+    }
+    if (polyLats.length < 4) {
+      throw new IllegalArgumentException("at least 4 polygon points required");
+    }
+    if (polyLats[0] != polyLats[polyLats.length-1]) {
+      throw new IllegalArgumentException("first and last points of the polygon must be the same (it must close itself): polyLats[0]=" + polyLats[0] + " polyLats[" + (polyLats.length-1) + "]=" + polyLats[polyLats.length-1]);
+    }
+    if (polyLons[0] != polyLons[polyLons.length-1]) {
+      throw new IllegalArgumentException("first and last points of the polygon must be the same (it must close itself): polyLons[0]=" + polyLons[0] + " polyLons[" + (polyLons.length-1) + "]=" + polyLons[polyLons.length-1]);
+    }
+
+    this.x = polyLons;
+    this.y = polyLats;
+  }
+
+  @Override @SuppressWarnings("unchecked")
+  protected TermsEnum getTermsEnum(final Terms terms, AttributeSource atts) throws IOException {
+    return new GeoPolygonTermsEnum(terms.iterator(), this.minLon, this.minLat, this.maxLon, this.maxLat);
+  }
+
+  /** throw exception if trying to change rewrite method */
+  @Override
+  public void setRewriteMethod(RewriteMethod method) {
+    throw new UnsupportedOperationException("cannot change rewrite method");
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+
+    GeoPointInPolygonQuery that = (GeoPointInPolygonQuery) o;
+
+    if (!Arrays.equals(x, that.x)) return false;
+    if (!Arrays.equals(y, that.y)) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + (x != null ? Arrays.hashCode(x) : 0);
+    result = 31 * result + (y != null ? Arrays.hashCode(y) : 0);
+    return result;
+  }
+
+  /** print out this polygon query */
+  @Override
+  public String toString(String field) {
+    assert x.length == y.length;
+
+    final StringBuilder sb = new StringBuilder();
+    sb.append(getClass().getSimpleName());
+    sb.append(':');
+    if (!getField().equals(field)) {
+      sb.append(" field=");
+      sb.append(getField());
+      sb.append(':');
+    }
+    sb.append(" Points: ");
+    for (int i=0; i<x.length; ++i) {
+      sb.append("[")
+        .append(x[i])
+        .append(", ")
+        .append(y[i])
+        .append("] ");
+    }
+
+    return sb.toString();
+  }
+
+  /**
+   * Custom {@link org.apache.lucene.index.TermsEnum} that computes morton hash ranges based on the defined edges of
+   * the provided polygon.
+   */
+  private final class GeoPolygonTermsEnum extends GeoPointTermsEnum {
+    GeoPolygonTermsEnum(final TermsEnum tenum, final double minLon, final double minLat,
+                        final double maxLon, final double maxLat) {
+      super(tenum, minLon, minLat, maxLon, maxLat);
+    }
+
+    @Override
+    protected boolean cellCrosses(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+      return GeoRelationUtils.rectCrossesPolyApprox(minLon, minLat, maxLon, maxLat, x, y, GeoPointInPolygonQuery.this.minLon,
+          GeoPointInPolygonQuery.this.minLat, GeoPointInPolygonQuery.this.maxLon, GeoPointInPolygonQuery.this.maxLat);
+    }
+
+    @Override
+    protected boolean cellWithin(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+      return GeoRelationUtils.rectWithinPolyApprox(minLon, minLat, maxLon, maxLat, x, y, GeoPointInPolygonQuery.this.minLon,
+          GeoPointInPolygonQuery.this.minLat, GeoPointInPolygonQuery.this.maxLon, GeoPointInPolygonQuery.this.maxLat);
+    }
+
+    @Override
+    protected boolean cellIntersectsShape(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+      return cellContains(minLon, minLat, maxLon, maxLat) || cellWithin(minLon, minLat, maxLon, maxLat)
+          || cellCrosses(minLon, minLat, maxLon, maxLat);
+    }
+
+    /**
+     * The two-phase query approach. The parent
+     * {@link GeoPointTermsEnum#accept} method is called to match
+     * encoded terms that fall within the bounding box of the polygon. Those documents that pass the initial
+     * bounding box filter are then compared to the provided polygon using the
+     * {@link org.apache.lucene.spatial.util.GeoRelationUtils#pointInPolygon} method.
+     */
+    @Override
+    protected boolean postFilter(final double lon, final double lat) {
+      return GeoRelationUtils.pointInPolygon(x, y, lat, lon);
+    }
+  }
+
+  /**
+   * API utility method for returning the array of longitudinal values for this GeoPolygon
+   * The returned array is not a copy so do not change it!
+   */
+  public double[] getLons() {
+    return this.x;
+  }
+
+  /**
+   * API utility method for returning the array of latitudinal values for this GeoPolygon
+   * The returned array is not a copy so do not change it!
+   */
+  public double[] getLats() {
+    return this.y;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/665041c5/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointTermQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointTermQuery.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointTermQuery.java
new file mode 100644
index 0000000..894a1e9
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointTermQuery.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.MultiTermQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.spatial.util.GeoUtils;
+import org.apache.lucene.util.AttributeSource;
+
+/**
+ * TermQuery for GeoPointField for overriding {@link org.apache.lucene.search.MultiTermQuery} methods specific to
+ * Geospatial operations
+ *
+ * @lucene.experimental
+ */
+abstract class GeoPointTermQuery extends MultiTermQuery {
+  // simple bounding box optimization - no objects used to avoid dependencies
+  /** minimum longitude value (in degrees) */
+  protected final double minLon;
+  /** minimum latitude value (in degrees) */
+  protected final double minLat;
+  /** maximum longitude value (in degrees) */
+  protected final double maxLon;
+  /** maximum latitude value (in degrees) */
+  protected final double maxLat;
+
+  /**
+   * Constructs a query matching terms that cannot be represented with a single
+   * Term.
+   */
+  public GeoPointTermQuery(String field, final double minLon, final double minLat, final double maxLon, final double maxLat) {
+    super(field);
+
+    if (GeoUtils.isValidLon(minLon) == false) {
+      throw new IllegalArgumentException("invalid minLon " + minLon);
+    }
+    if (GeoUtils.isValidLon(maxLon) == false) {
+      throw new IllegalArgumentException("invalid maxLon " + maxLon);
+    }
+    if (GeoUtils.isValidLat(minLat) == false) {
+      throw new IllegalArgumentException("invalid minLat " + minLat);
+    }
+    if (GeoUtils.isValidLat(maxLat) == false) {
+      throw new IllegalArgumentException("invalid maxLat " + maxLat);
+    }
+    this.minLon = minLon;
+    this.minLat = minLat;
+    this.maxLon = maxLon;
+    this.maxLat = maxLat;
+
+    this.rewriteMethod = GEO_CONSTANT_SCORE_REWRITE;
+  }
+
+  private static final RewriteMethod GEO_CONSTANT_SCORE_REWRITE = new RewriteMethod() {
+    @Override
+    public Query rewrite(IndexReader reader, MultiTermQuery query) {
+      return new GeoPointTermQueryConstantScoreWrapper<>((GeoPointTermQuery)query);
+    }
+  };
+
+  /** override package protected method */
+  @Override
+  protected abstract TermsEnum getTermsEnum(final Terms terms, AttributeSource atts) throws IOException;
+
+  /** check if this instance equals another instance */
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+
+    GeoPointTermQuery that = (GeoPointTermQuery) o;
+
+    if (Double.compare(that.minLon, minLon) != 0) return false;
+    if (Double.compare(that.minLat, minLat) != 0) return false;
+    if (Double.compare(that.maxLon, maxLon) != 0) return false;
+    return Double.compare(that.maxLat, maxLat) == 0;
+  }
+
+  /** compute hashcode */
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    long temp;
+    temp = Double.doubleToLongBits(minLon);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    temp = Double.doubleToLongBits(minLat);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    temp = Double.doubleToLongBits(maxLon);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    temp = Double.doubleToLongBits(maxLat);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/665041c5/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointTermQueryConstantScoreWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointTermQueryConstantScoreWrapper.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointTermQueryConstantScoreWrapper.java
new file mode 100644
index 0000000..8176aec
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointTermQueryConstantScoreWrapper.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.search.BulkScorer;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.spatial.util.GeoUtils;
+
+/**
+ * Custom ConstantScoreWrapper for {@code GeoPointTermQuery} that cuts over to DocValues
+ * for post filtering boundary ranges. Multi-valued GeoPoint documents are supported.
+ *
+ * @lucene.experimental
+ */
+final class GeoPointTermQueryConstantScoreWrapper <Q extends GeoPointTermQuery> extends Query {
+  protected final Q query;
+
+  protected GeoPointTermQueryConstantScoreWrapper(Q query) {
+    this.query = query;
+  }
+
+  @Override
+  public String toString(String field) {
+    return query.toString();
+  }
+
+  @Override
+  public final boolean equals(final Object o) {
+    if (super.equals(o) == false) {
+      return false;
+    }
+    final GeoPointTermQueryConstantScoreWrapper<?> that = (GeoPointTermQueryConstantScoreWrapper<?>) o;
+    return this.query.equals(that.query);
+  }
+
+  @Override
+  public final int hashCode() {
+    return 31 * super.hashCode() + query.hashCode();
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    return new ConstantScoreWeight(this) {
+
+      private DocIdSet getDocIDs(LeafReaderContext context) throws IOException {
+        final Terms terms = context.reader().terms(query.getField());
+        if (terms == null) {
+          return DocIdSet.EMPTY;
+        }
+
+        final GeoPointTermsEnum termsEnum = (GeoPointTermsEnum)(query.getTermsEnum(terms, null));
+        assert termsEnum != null;
+
+        LeafReader reader = context.reader();
+        DocIdSetBuilder builder = new DocIdSetBuilder(reader.maxDoc());
+        PostingsEnum docs = null;
+        SortedNumericDocValues sdv = reader.getSortedNumericDocValues(query.getField());
+
+        while (termsEnum.next() != null) {
+          docs = termsEnum.postings(docs, PostingsEnum.NONE);
+          // boundary terms need post filtering by
+          if (termsEnum.boundaryTerm()) {
+            int docId = docs.nextDoc();
+            long hash;
+            do {
+              sdv.setDocument(docId);
+              for (int i=0; i<sdv.count(); ++i) {
+                hash = sdv.valueAt(i);
+                if (termsEnum.postFilter(GeoUtils.mortonUnhashLon(hash), GeoUtils.mortonUnhashLat(hash))) {
+                  builder.add(docId);
+                  break;
+                }
+              }
+            } while ((docId = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS);
+          } else {
+            builder.add(docs);
+          }
+        }
+
+        return builder.build();
+      }
+
+      private Scorer scorer(DocIdSet set) throws IOException {
+        if (set == null) {
+          return null;
+        }
+        final DocIdSetIterator disi = set.iterator();
+        if (disi == null) {
+          return null;
+        }
+        return new ConstantScoreScorer(this, score(), disi);
+      }
+
+      @Override
+      public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
+        final Scorer scorer = scorer(getDocIDs(context));
+        if (scorer == null) {
+          return null;
+        }
+        return new DefaultBulkScorer(scorer);
+      }
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        return scorer(getDocIDs(context));
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/665041c5/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointTermsEnum.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointTermsEnum.java
new file mode 100644
index 0000000..71eb26e
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointTermsEnum.java
@@ -0,0 +1,249 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.search;
+
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.lucene.spatial.document.GeoPointField;
+import org.apache.lucene.index.FilteredTermsEnum;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.spatial.util.GeoRelationUtils;
+import org.apache.lucene.spatial.util.GeoUtils;
+import org.apache.lucene.util.LegacyNumericUtils;
+
+/**
+ * computes all ranges along a space-filling curve that represents
+ * the given bounding box and enumerates all terms contained within those ranges
+ *
+ *  @lucene.experimental
+ */
+abstract class GeoPointTermsEnum extends FilteredTermsEnum {
+  protected final double minLon;
+  protected final double minLat;
+  protected final double maxLon;
+  protected final double maxLat;
+
+  protected Range currentRange;
+  private final BytesRefBuilder currentCell = new BytesRefBuilder();
+  private final BytesRefBuilder nextSubRange = new BytesRefBuilder();
+
+  private final List<Range> rangeBounds = new LinkedList<>();
+
+  // detail level should be a factor of PRECISION_STEP limiting the depth of recursion (and number of ranges)
+  protected final short DETAIL_LEVEL;
+
+  GeoPointTermsEnum(final TermsEnum tenum, final double minLon, final double minLat,
+                    final double maxLon, final double maxLat) {
+    super(tenum);
+    final long rectMinHash = GeoUtils.mortonHash(minLon, minLat);
+    final long rectMaxHash = GeoUtils.mortonHash(maxLon, maxLat);
+    this.minLon = GeoUtils.mortonUnhashLon(rectMinHash);
+    this.minLat = GeoUtils.mortonUnhashLat(rectMinHash);
+    this.maxLon = GeoUtils.mortonUnhashLon(rectMaxHash);
+    this.maxLat = GeoUtils.mortonUnhashLat(rectMaxHash);
+    DETAIL_LEVEL = (short)(((GeoUtils.BITS<<1)-computeMaxShift())/2);
+
+    computeRange(0L, (short) ((GeoUtils.BITS << 1) - 1));
+    assert rangeBounds.isEmpty() == false;
+    Collections.sort(rangeBounds);
+  }
+
+  /**
+   * entry point for recursively computing ranges
+   */
+  private final void computeRange(long term, final short shift) {
+    final long split = term | (0x1L<<shift);
+    assert shift < 64;
+    final long upperMax;
+    if (shift < 63) {
+      upperMax = term | ((1L << (shift+1))-1);
+    } else {
+      upperMax = 0xffffffffffffffffL;
+    }
+    final long lowerMax = split-1;
+
+    relateAndRecurse(term, lowerMax, shift);
+    relateAndRecurse(split, upperMax, shift);
+  }
+
+  /**
+   * recurse to higher level precision cells to find ranges along the space-filling curve that fall within the
+   * query box
+   *
+   * @param start starting value on the space-filling curve for a cell at a given res
+   * @param end ending value on the space-filling curve for a cell at a given res
+   * @param res spatial res represented as a bit shift (MSB is lower res)
+   */
+  private void relateAndRecurse(final long start, final long end, final short res) {
+    final double minLon = GeoUtils.mortonUnhashLon(start);
+    final double minLat = GeoUtils.mortonUnhashLat(start);
+    final double maxLon = GeoUtils.mortonUnhashLon(end);
+    final double maxLat = GeoUtils.mortonUnhashLat(end);
+
+    final short level = (short)((GeoUtils.BITS<<1)-res>>>1);
+
+    // if cell is within and a factor of the precision step, or it crosses the edge of the shape add the range
+    final boolean within = res % GeoPointField.PRECISION_STEP == 0 && cellWithin(minLon, minLat, maxLon, maxLat);
+    if (within || (level == DETAIL_LEVEL && cellIntersectsShape(minLon, minLat, maxLon, maxLat))) {
+      final short nextRes = (short)(res-1);
+      if (nextRes % GeoPointField.PRECISION_STEP == 0) {
+        rangeBounds.add(new Range(start, nextRes, !within));
+        rangeBounds.add(new Range(start|(1L<<nextRes), nextRes, !within));
+      } else {
+        rangeBounds.add(new Range(start, res, !within));
+      }
+    } else if (level < DETAIL_LEVEL && cellIntersectsMBR(minLon, minLat, maxLon, maxLat)) {
+      computeRange(start, (short) (res - 1));
+    }
+  }
+
+  protected short computeMaxShift() {
+    // in this case a factor of 4 brings the detail level to ~0.002/0.001 degrees lon/lat respectively (or ~222m/111m)
+    return GeoPointField.PRECISION_STEP * 4;
+  }
+
+  /**
+   * Determine whether the quad-cell crosses the shape
+   */
+  protected abstract boolean cellCrosses(final double minLon, final double minLat, final double maxLon, final double maxLat);
+
+  /**
+   * Determine whether quad-cell is within the shape
+   */
+  protected abstract boolean cellWithin(final double minLon, final double minLat, final double maxLon, final double maxLat);
+
+  /**
+   * Default shape is a rectangle, so this returns the same as {@code cellIntersectsMBR}
+   */
+  protected abstract boolean cellIntersectsShape(final double minLon, final double minLat, final double maxLon, final double maxLat);
+
+  /**
+   * Primary driver for cells intersecting shape boundaries
+   */
+  protected boolean cellIntersectsMBR(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+    return GeoRelationUtils.rectIntersects(minLon, minLat, maxLon, maxLat, this.minLon, this.minLat, this.maxLon, this.maxLat);
+  }
+
+  /**
+   * Return whether quad-cell contains the bounding box of this shape
+   */
+  protected boolean cellContains(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+    return GeoRelationUtils.rectWithin(this.minLon, this.minLat, this.maxLon, this.maxLat, minLon, minLat, maxLon, maxLat);
+  }
+
+  public boolean boundaryTerm() {
+    if (currentRange == null) {
+      throw new IllegalStateException("GeoPointTermsEnum empty or not initialized");
+    }
+    return currentRange.boundary;
+  }
+
+  private void nextRange() {
+    currentRange = rangeBounds.remove(0);
+    currentRange.fillBytesRef(currentCell);
+  }
+
+  @Override
+  protected final BytesRef nextSeekTerm(BytesRef term) {
+    while (!rangeBounds.isEmpty()) {
+      if (currentRange == null) {
+        nextRange();
+      }
+
+      // if the new upper bound is before the term parameter, the sub-range is never a hit
+      if (term != null && term.compareTo(currentCell.get()) > 0) {
+        nextRange();
+        if (!rangeBounds.isEmpty()) {
+          continue;
+        }
+      }
+      // never seek backwards, so use current term if lower bound is smaller
+      return (term != null && term.compareTo(currentCell.get()) > 0) ?
+          term : currentCell.get();
+    }
+
+    // no more sub-range enums available
+    assert rangeBounds.isEmpty();
+    return null;
+  }
+
+  /**
+   * The two-phase query approach. {@link #nextSeekTerm} is called to obtain the next term that matches a numeric
+   * range of the bounding box. Those terms that pass the initial range filter are then compared against the
+   * decoded min/max latitude and longitude values of the bounding box only if the range is not a "boundary" range
+   * (e.g., a range that straddles the boundary of the bbox).
+   * @param term term for candidate document
+   * @return match status
+   */
+  @Override
+  protected AcceptStatus accept(BytesRef term) {
+    // validate value is in range
+    while (currentCell == null || term.compareTo(currentCell.get()) > 0) {
+      if (rangeBounds.isEmpty()) {
+        return AcceptStatus.END;
+      }
+      // peek next sub-range, only seek if the current term is smaller than next lower bound
+      rangeBounds.get(0).fillBytesRef(this.nextSubRange);
+      if (term.compareTo(this.nextSubRange.get()) < 0) {
+        return AcceptStatus.NO_AND_SEEK;
+      }
+      // step forward to next range without seeking, as next range is less or equal current term
+      nextRange();
+    }
+
+    return AcceptStatus.YES;
+  }
+
+  protected abstract boolean postFilter(final double lon, final double lat);
+
+  /**
+   * Internal class to represent a range along the space filling curve
+   */
+  protected final class Range implements Comparable<Range> {
+    final short shift;
+    final long start;
+    final boolean boundary;
+
+    Range(final long lower, final short shift, boolean boundary) {
+      this.boundary = boundary;
+      this.start = lower;
+      this.shift = shift;
+    }
+
+    /**
+     * Encode as a BytesRef using a reusable object. This allows us to lazily create the BytesRef (which is
+     * quite expensive), only when we need it.
+     */
+    private void fillBytesRef(BytesRefBuilder result) {
+      assert result != null;
+      LegacyNumericUtils.longToPrefixCoded(start, shift, result);
+    }
+
+    @Override
+    public int compareTo(Range other) {
+      final int result = Short.compare(this.shift, other.shift);
+      if (result == 0) {
+        return Long.compare(this.start, other.start);
+      }
+      return result;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/665041c5/lucene/spatial/src/java/org/apache/lucene/spatial/search/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/package-info.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/package-info.java
new file mode 100644
index 0000000..8e8265c
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/search/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Geospatial Query Implementations for Core Lucene
+ */
+package org.apache.lucene.spatial.search;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/665041c5/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoDistanceUtils.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoDistanceUtils.java b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoDistanceUtils.java
new file mode 100644
index 0000000..e845c9e
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoDistanceUtils.java
@@ -0,0 +1,223 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.util;
+
+import org.apache.lucene.util.SloppyMath;
+
+import static org.apache.lucene.util.SloppyMath.TO_RADIANS;
+
+/**
+ * Reusable geo-spatial distance utility methods.
+ *
+ * @lucene.experimental
+ */
+public class GeoDistanceUtils {
+  /** error threshold for point-distance queries (in percent) NOTE: Guideline from USGS is 0.005 **/
+  public static final double DISTANCE_PCT_ERR = 0.005;
+
+  // No instance:
+  private GeoDistanceUtils() {
+  }
+
+  /**
+   * Compute the great-circle distance using original haversine implementation published by Sinnot in:
+   * R.W. Sinnott, "Virtues of the Haversine", Sky and Telescope, vol. 68, no. 2, 1984, p. 159
+   *
+   * NOTE: this differs from {@link org.apache.lucene.util.SloppyMath#haversin} in that it uses the semi-major axis
+   * of the earth instead of an approximation based on the average latitude of the two points (which can introduce an
+   * additional error up to .337%, or ~67.6 km at the equator)
+   */
+  public static double haversin(double lat1, double lon1, double lat2, double lon2) {
+    double dLat = TO_RADIANS * (lat2 - lat1);
+    double dLon = TO_RADIANS * (lon2 - lon1);
+    lat1 = TO_RADIANS * (lat1);
+    lat2 = TO_RADIANS * (lat2);
+
+    final double sinDLatO2 = SloppyMath.sin(dLat / 2);
+    final double sinDLonO2 = SloppyMath.sin(dLon / 2);
+
+    double a = sinDLatO2*sinDLatO2 + sinDLonO2 * sinDLonO2 * SloppyMath.cos(lat1) * SloppyMath.cos(lat2);
+    double c = 2 * SloppyMath.asin(Math.sqrt(a));
+    return (GeoProjectionUtils.SEMIMAJOR_AXIS * c);
+  }
+
+  /**
+   * Compute the distance between two geo-points using vincenty distance formula
+   * Vincenty uses the oblate spheroid whereas haversine uses unit sphere, this will give roughly
+   * 22m better accuracy (in worst case) than haversine
+   *
+   * @param lonA longitudinal coordinate of point A (in degrees)
+   * @param latA latitudinal coordinate of point A (in degrees)
+   * @param lonB longitudinal coordinate of point B (in degrees)
+   * @param latB latitudinal coordinate of point B (in degrees)
+   * @return distance (in meters) between point A and point B
+   */
+  public static final double vincentyDistance(final double lonA, final double latA, final double lonB, final double latB) {
+    final double L = StrictMath.toRadians(lonB - lonA);
+    final double oF = 1 - GeoProjectionUtils.FLATTENING;
+    final double U1 = StrictMath.atan(oF * StrictMath.tan(StrictMath.toRadians(latA)));
+    final double U2 = StrictMath.atan(oF * StrictMath.tan(StrictMath.toRadians(latB)));
+    final double sU1 = StrictMath.sin(U1);
+    final double cU1 = StrictMath.cos(U1);
+    final double sU2 = StrictMath.sin(U2);
+    final double cU2 = StrictMath.cos(U2);
+
+    double sigma, sinSigma, cosSigma;
+    double sinAlpha, cos2Alpha, cos2SigmaM;
+    double lambda = L;
+    double lambdaP;
+    double iters = 100;
+    double sinLambda, cosLambda, c;
+
+    do {
+      sinLambda = StrictMath.sin(lambda);
+      cosLambda = Math.cos(lambda);
+      sinSigma = Math.sqrt((cU2 * sinLambda) * (cU2 * sinLambda) + (cU1 * sU2 - sU1 * cU2 * cosLambda)
+          * (cU1 * sU2 - sU1 * cU2 * cosLambda));
+      if (sinSigma == 0) {
+        return 0;
+      }
+
+      cosSigma = sU1 * sU2 + cU1 * cU2 * cosLambda;
+      sigma = Math.atan2(sinSigma, cosSigma);
+      sinAlpha = cU1 * cU2 * sinLambda / sinSigma;
+      cos2Alpha = 1 - sinAlpha * sinAlpha;
+      cos2SigmaM = cosSigma - 2 * sU1 * sU2 / cos2Alpha;
+
+      c = GeoProjectionUtils.FLATTENING/16 * cos2Alpha * (4 + GeoProjectionUtils.FLATTENING * (4 - 3 * cos2Alpha));
+      lambdaP = lambda;
+      lambda = L + (1 - c) * GeoProjectionUtils.FLATTENING * sinAlpha * (sigma + c * sinSigma * (cos2SigmaM + c * cosSigma *
+          (-1 + 2 * cos2SigmaM * cos2SigmaM)));
+    } while (StrictMath.abs(lambda - lambdaP) > 1E-12 && --iters > 0);
+
+    if (iters == 0) {
+      return 0;
+    }
+
+    final double uSq = cos2Alpha * (GeoProjectionUtils.SEMIMAJOR_AXIS2 - GeoProjectionUtils.SEMIMINOR_AXIS2) / (GeoProjectionUtils.SEMIMINOR_AXIS2);
+    final double A = 1 + uSq / 16384 * (4096 + uSq * (-768 + uSq * (320 - 175 * uSq)));
+    final double B = uSq / 1024 * (256 + uSq * (-128 + uSq * (74 - 47 * uSq)));
+    final double deltaSigma = B * sinSigma * (cos2SigmaM + B/4 * (cosSigma * (-1 + 2 * cos2SigmaM * cos2SigmaM) - B/6 * cos2SigmaM
+        * (-3 + 4 * sinSigma * sinSigma) * (-3 + 4 * cos2SigmaM * cos2SigmaM)));
+
+    return (GeoProjectionUtils.SEMIMINOR_AXIS * A * (sigma - deltaSigma));
+  }
+
+  /**
+   * Computes distance between two points in a cartesian (x, y, {z - optional}) coordinate system
+   */
+  public static double linearDistance(double[] pt1, double[] pt2) {
+    assert pt1 != null && pt2 != null && pt1.length == pt2.length && pt1.length > 1;
+    final double d0 = pt1[0] - pt2[0];
+    final double d1 = pt1[1] - pt2[1];
+    if (pt1.length == 3) {
+      final double d2 = pt1[2] - pt2[2];
+      return Math.sqrt(d0*d0 + d1*d1 + d2*d2);
+    }
+    return Math.sqrt(d0*d0 + d1*d1);
+  }
+
+  /**
+   * Compute the inverse haversine to determine distance in degrees longitude for provided distance in meters
+   * @param lat latitude to compute delta degrees lon
+   * @param distance distance in meters to convert to degrees lon
+   * @return Sloppy distance in degrees longitude for provided distance in meters
+   */
+  public static double distanceToDegreesLon(double lat, double distance) {
+    distance /= 1000.0;
+    // convert latitude to radians
+    lat = StrictMath.toRadians(lat);
+
+    // get the diameter at the latitude
+    final double diameter = SloppyMath.earthDiameter(StrictMath.toRadians(lat));
+
+    // compute inverse haversine
+    double a = StrictMath.sin(distance/diameter);
+    double h = StrictMath.min(1, a);
+    h *= h;
+    double cLat = StrictMath.cos(lat);
+
+    return StrictMath.toDegrees(StrictMath.acos(1-((2d*h)/(cLat*cLat))));
+  }
+
+  /**
+   *  Finds the closest point within a rectangle (defined by rMinX, rMinY, rMaxX, rMaxY) to the given (lon, lat) point
+   *  the result is provided in closestPt.  When the point is outside the rectangle, the closest point is on an edge
+   *  or corner of the rectangle; else, the closest point is the point itself.
+   */
+  public static void closestPointOnBBox(final double rMinX, final double rMinY, final double rMaxX, final double rMaxY,
+                                        final double lon, final double lat, double[] closestPt) {
+    assert closestPt != null && closestPt.length == 2;
+
+    closestPt[0] = 0;
+    closestPt[1] = 0;
+
+    boolean xSet = true;
+    boolean ySet = true;
+
+    if (lon > rMaxX) {
+      closestPt[0] = rMaxX;
+    } else if (lon < rMinX) {
+      closestPt[0] = rMinX;
+    } else {
+      xSet = false;
+    }
+
+    if (lat > rMaxY) {
+      closestPt[1] = rMaxY;
+    } else if (lat < rMinY) {
+      closestPt[1] = rMinY;
+    } else {
+      ySet = false;
+    }
+
+    if (closestPt[0] == 0 && xSet == false) {
+      closestPt[0] = lon;
+    }
+
+    if (closestPt[1] == 0 && ySet == false) {
+      closestPt[1] = lat;
+    }
+  }
+
+  /** Returns the maximum distance/radius (in meters) from the point 'center' before overlapping */
+  public static double maxRadialDistanceMeters(final double centerLon, final double centerLat) {
+    if (Math.abs(centerLat) == GeoUtils.MAX_LAT_INCL) {
+      return GeoDistanceUtils.haversin(centerLat, centerLon, 0, centerLon);
+    }
+    return GeoDistanceUtils.haversin(centerLat, centerLon, centerLat, (GeoUtils.MAX_LON_INCL + centerLon) % 360);
+  }
+
+  /**
+   * Compute the inverse haversine to determine distance in degrees longitude for provided distance in meters
+   * @param lat latitude to compute delta degrees lon
+   * @param distance distance in meters to convert to degrees lon
+   * @return Sloppy distance in degrees longitude for provided distance in meters
+   */
+  public static double distanceToDegreesLat(double lat, double distance) {
+    // get the diameter at the latitude
+    final double diameter = SloppyMath.earthDiameter(StrictMath.toRadians(lat));
+    distance /= 1000.0;
+
+    // compute inverse haversine
+    double a = StrictMath.sin(distance/diameter);
+    double h = StrictMath.min(1, a);
+    h *= h;
+
+    return StrictMath.toDegrees(StrictMath.acos(1-(2d*h)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/665041c5/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoHashUtils.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoHashUtils.java b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoHashUtils.java
new file mode 100644
index 0000000..9450c1e
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoHashUtils.java
@@ -0,0 +1,283 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.util;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.lucene.util.BitUtil;
+
+/**
+ * Utilities for converting to/from the GeoHash standard
+ *
+ * The geohash long format is represented as lon/lat (x/y) interleaved with the 4 least significant bits
+ * representing the level (1-12) [xyxy...xyxyllll]
+ *
+ * This differs from a morton encoded value which interleaves lat/lon (y/x).
+ *
+ * @lucene.experimental
+ */
+public class GeoHashUtils {
+  private static final char[] BASE_32 = {'0', '1', '2', '3', '4', '5', '6',
+      '7', '8', '9', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'j', 'k', 'm', 'n',
+      'p', 'q', 'r', 's', 't', 'u', 'v', 'w', 'x', 'y', 'z'};
+
+  private static final String BASE_32_STRING = new String(BASE_32);
+
+  /** maximum precision for geohash strings */
+  public static final int PRECISION = 12;
+  private static final short MORTON_OFFSET = (GeoUtils.BITS<<1) - (PRECISION*5);
+
+  // No instance:
+  private GeoHashUtils() {
+  }
+
+  /**
+   * Encode lon/lat to the geohash based long format (lon/lat interleaved, 4 least significant bits = level)
+   */
+  public static final long longEncode(final double lon, final double lat, final int level) {
+    // shift to appropriate level
+    final short msf = (short)(((12 - level) * 5) + MORTON_OFFSET);
+    return ((BitUtil.flipFlop(GeoUtils.mortonHash(lon, lat)) >>> msf) << 4) | level;
+  }
+
+  /**
+   * Encode from geohash string to the geohash based long format (lon/lat interleaved, 4 least significant bits = level)
+   */
+  public static final long longEncode(final String hash) {
+    int level = hash.length()-1;
+    long b;
+    long l = 0L;
+    for(char c : hash.toCharArray()) {
+      b = (long)(BASE_32_STRING.indexOf(c));
+      l |= (b<<(level--*5));
+    }
+    return (l<<4)|hash.length();
+  }
+
+  /**
+   * Encode an existing geohash long to the provided precision
+   */
+  public static long longEncode(long geohash, int level) {
+    final short precision = (short)(geohash & 15);
+    if (precision == level) {
+      return geohash;
+    } else if (precision > level) {
+      return ((geohash >>> (((precision - level) * 5) + 4)) << 4) | level;
+    }
+    return ((geohash >>> 4) << (((level - precision) * 5) + 4) | level);
+  }
+
+  /**
+   * Convert from a morton encoded long from a geohash encoded long
+   */
+  public static long fromMorton(long morton, int level) {
+    long mFlipped = BitUtil.flipFlop(morton);
+    mFlipped >>>= (((GeoHashUtils.PRECISION - level) * 5) + MORTON_OFFSET);
+    return (mFlipped << 4) | level;
+  }
+
+  /**
+   * Encode to a geohash string from the geohash based long format
+   */
+  public static final String stringEncode(long geoHashLong) {
+    int level = (int)geoHashLong&15;
+    geoHashLong >>>= 4;
+    char[] chars = new char[level];
+    do {
+      chars[--level] = BASE_32[(int) (geoHashLong&31L)];
+      geoHashLong>>>=5;
+    } while(level > 0);
+
+    return new String(chars);
+  }
+
+  /**
+   * Encode to a geohash string from full resolution longitude, latitude)
+   */
+  public static final String stringEncode(final double lon, final double lat) {
+    return stringEncode(lon, lat, 12);
+  }
+
+  /**
+   * Encode to a level specific geohash string from full resolution longitude, latitude
+   */
+  public static final String stringEncode(final double lon, final double lat, final int level) {
+    // convert to geohashlong
+    final long ghLong = fromMorton(GeoUtils.mortonHash(lon, lat), level);
+    return stringEncode(ghLong);
+
+  }
+
+  /**
+   * Encode to a full precision geohash string from a given morton encoded long value
+   */
+  public static final String stringEncodeFromMortonLong(final long hashedVal) throws Exception {
+    return stringEncode(hashedVal, PRECISION);
+  }
+
+  /**
+   * Encode to a geohash string at a given level from a morton long
+   */
+  public static final String stringEncodeFromMortonLong(long hashedVal, final int level) {
+    // bit twiddle to geohash (since geohash is a swapped (lon/lat) encoding)
+    hashedVal = BitUtil.flipFlop(hashedVal);
+
+    StringBuilder geoHash = new StringBuilder();
+    short precision = 0;
+    final short msf = (GeoUtils.BITS<<1)-5;
+    long mask = 31L<<msf;
+    do {
+      geoHash.append(BASE_32[(int)((mask & hashedVal)>>>(msf-(precision*5)))]);
+      // next 5 bits
+      mask >>>= 5;
+    } while (++precision < level);
+    return geoHash.toString();
+  }
+
+  /**
+   * Encode to a morton long value from a given geohash string
+   */
+  public static final long mortonEncode(final String hash) {
+    int level = 11;
+    long b;
+    long l = 0L;
+    for(char c : hash.toCharArray()) {
+      b = (long)(BASE_32_STRING.indexOf(c));
+      l |= (b<<((level--*5) + MORTON_OFFSET));
+    }
+    return BitUtil.flipFlop(l);
+  }
+
+  /**
+   * Encode to a morton long value from a given geohash long value
+   */
+  public static final long mortonEncode(final long geoHashLong) {
+    final int level = (int)(geoHashLong&15);
+    final short odd = (short)(level & 1);
+
+    return BitUtil.flipFlop(((geoHashLong >>> 4) << odd) << (((12 - level) * 5) + (MORTON_OFFSET - odd)));
+  }
+
+  private static final char encode(int x, int y) {
+    return BASE_32[((x & 1) + ((y & 1) * 2) + ((x & 2) * 2) + ((y & 2) * 4) + ((x & 4) * 4)) % 32];
+  }
+
+  /**
+   * Calculate all neighbors of a given geohash cell.
+   *
+   * @param geohash Geohash of the defined cell
+   * @return geohashes of all neighbor cells
+   */
+  public static Collection<? extends CharSequence> neighbors(String geohash) {
+    return addNeighbors(geohash, geohash.length(), new ArrayList<CharSequence>(8));
+  }
+
+  /**
+   * Calculate the geohash of a neighbor of a geohash
+   *
+   * @param geohash the geohash of a cell
+   * @param level   level of the geohash
+   * @param dx      delta of the first grid coordinate (must be -1, 0 or +1)
+   * @param dy      delta of the second grid coordinate (must be -1, 0 or +1)
+   * @return geohash of the defined cell
+   */
+  public final static String neighbor(String geohash, int level, int dx, int dy) {
+    int cell = BASE_32_STRING.indexOf(geohash.charAt(level -1));
+
+    // Decoding the Geohash bit pattern to determine grid coordinates
+    int x0 = cell & 1;  // first bit of x
+    int y0 = cell & 2;  // first bit of y
+    int x1 = cell & 4;  // second bit of x
+    int y1 = cell & 8;  // second bit of y
+    int x2 = cell & 16; // third bit of x
+
+    // combine the bitpattern to grid coordinates.
+    // note that the semantics of x and y are swapping
+    // on each level
+    int x = x0 + (x1 / 2) + (x2 / 4);
+    int y = (y0 / 2) + (y1 / 4);
+
+    if (level == 1) {
+      // Root cells at north (namely "bcfguvyz") or at
+      // south (namely "0145hjnp") do not have neighbors
+      // in north/south direction
+      if ((dy < 0 && y == 0) || (dy > 0 && y == 3)) {
+        return null;
+      } else {
+        return Character.toString(encode(x + dx, y + dy));
+      }
+    } else {
+      // define grid coordinates for next level
+      final int nx = ((level % 2) == 1) ? (x + dx) : (x + dy);
+      final int ny = ((level % 2) == 1) ? (y + dy) : (y + dx);
+
+      // if the defined neighbor has the same parent a the current cell
+      // encode the cell directly. Otherwise find the cell next to this
+      // cell recursively. Since encoding wraps around within a cell
+      // it can be encoded here.
+      // xLimit and YLimit must always be respectively 7 and 3
+      // since x and y semantics are swapping on each level.
+      if (nx >= 0 && nx <= 7 && ny >= 0 && ny <= 3) {
+        return geohash.substring(0, level - 1) + encode(nx, ny);
+      } else {
+        String neighbor = neighbor(geohash, level - 1, dx, dy);
+        return (neighbor != null) ? neighbor + encode(nx, ny) : neighbor;
+      }
+    }
+  }
+
+  /**
+   * Add all geohashes of the cells next to a given geohash to a list.
+   *
+   * @param geohash   Geohash of a specified cell
+   * @param neighbors list to add the neighbors to
+   * @return the given list
+   */
+  public static final <E extends Collection<? super String>> E addNeighbors(String geohash, E neighbors) {
+    return addNeighbors(geohash, geohash.length(), neighbors);
+  }
+
+  /**
+   * Add all geohashes of the cells next to a given geohash to a list.
+   *
+   * @param geohash   Geohash of a specified cell
+   * @param length    level of the given geohash
+   * @param neighbors list to add the neighbors to
+   * @return the given list
+   */
+  public static final <E extends Collection<? super String>> E addNeighbors(String geohash, int length, E neighbors) {
+    String south = neighbor(geohash, length, 0, -1);
+    String north = neighbor(geohash, length, 0, +1);
+    if (north != null) {
+      neighbors.add(neighbor(north, length, -1, 0));
+      neighbors.add(north);
+      neighbors.add(neighbor(north, length, +1, 0));
+    }
+
+    neighbors.add(neighbor(geohash, length, -1, 0));
+    neighbors.add(neighbor(geohash, length, +1, 0));
+
+    if (south != null) {
+      neighbors.add(neighbor(south, length, -1, 0));
+      neighbors.add(south);
+      neighbors.add(neighbor(south, length, +1, 0));
+    }
+
+    return neighbors;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/665041c5/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoProjectionUtils.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoProjectionUtils.java b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoProjectionUtils.java
new file mode 100644
index 0000000..5a81adc
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoProjectionUtils.java
@@ -0,0 +1,465 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.util;
+
+import static java.lang.StrictMath.sqrt;
+
+import static org.apache.lucene.util.SloppyMath.asin;
+import static org.apache.lucene.util.SloppyMath.cos;
+import static org.apache.lucene.util.SloppyMath.sin;
+import static org.apache.lucene.util.SloppyMath.tan;
+import static org.apache.lucene.util.SloppyMath.PIO2;
+import static org.apache.lucene.util.SloppyMath.TO_DEGREES;
+import static org.apache.lucene.util.SloppyMath.TO_RADIANS;
+
+import static org.apache.lucene.spatial.util.GeoUtils.MAX_LAT_INCL;
+import static org.apache.lucene.spatial.util.GeoUtils.MAX_LON_INCL;
+import static org.apache.lucene.spatial.util.GeoUtils.MIN_LAT_INCL;
+import static org.apache.lucene.spatial.util.GeoUtils.MIN_LON_INCL;
+import static org.apache.lucene.spatial.util.GeoUtils.normalizeLat;
+import static org.apache.lucene.spatial.util.GeoUtils.normalizeLon;
+
+/**
+ * Reusable geo-spatial projection utility methods.
+ *
+ * @lucene.experimental
+ */
+public class GeoProjectionUtils {
+  // WGS84 earth-ellipsoid parameters
+  /** major (a) axis in meters */
+  public static final double SEMIMAJOR_AXIS = 6_378_137; // [m]
+  /** earth flattening factor (f) */
+  public static final double FLATTENING = 1.0/298.257223563;
+  /** minor (b) axis in meters */
+  public static final double SEMIMINOR_AXIS = SEMIMAJOR_AXIS * (1.0 - FLATTENING); //6_356_752.31420; // [m]
+  /** first eccentricity (e) */
+  public static final double ECCENTRICITY = sqrt((2.0 - FLATTENING) * FLATTENING);
+  /** major axis squared (a2) */
+  public static final double SEMIMAJOR_AXIS2 = SEMIMAJOR_AXIS * SEMIMAJOR_AXIS;
+  /** minor axis squared (b2) */
+  public static final double SEMIMINOR_AXIS2 = SEMIMINOR_AXIS * SEMIMINOR_AXIS;
+  private static final double E2 = (SEMIMAJOR_AXIS2 - SEMIMINOR_AXIS2)/(SEMIMAJOR_AXIS2);
+  private static final double EP2 = (SEMIMAJOR_AXIS2 - SEMIMINOR_AXIS2)/(SEMIMINOR_AXIS2);
+
+  /** min longitude value in radians */
+  public static final double MIN_LON_RADIANS = TO_RADIANS * MIN_LON_INCL;
+  /** min latitude value in radians */
+  public static final double MIN_LAT_RADIANS = TO_RADIANS * MIN_LAT_INCL;
+  /** max longitude value in radians */
+  public static final double MAX_LON_RADIANS = TO_RADIANS * MAX_LON_INCL;
+  /** max latitude value in radians */
+  public static final double MAX_LAT_RADIANS = TO_RADIANS * MAX_LAT_INCL;
+
+  // No instance:
+  private GeoProjectionUtils() {
+  }
+
+  /**
+   * Converts from geocentric earth-centered earth-fixed to geodesic lat/lon/alt
+   * @param x Cartesian x coordinate
+   * @param y Cartesian y coordinate
+   * @param z Cartesian z coordinate
+   * @param lla 0: longitude 1: latitude: 2: altitude
+   * @return double array as 0: longitude 1: latitude 2: altitude
+   */
+  public static final double[] ecfToLLA(final double x, final double y, final double z, double[] lla) {
+    boolean atPole = false;
+    final double ad_c = 1.0026000D;
+    final double cos67P5 = 0.38268343236508977D;
+
+    if (lla == null) {
+      lla = new double[3];
+    }
+
+    if (x != 0.0) {
+      lla[0] = StrictMath.atan2(y,x);
+    } else {
+      if (y > 0) {
+        lla[0] = PIO2;
+      } else if (y < 0) {
+        lla[0] = -PIO2;
+      } else {
+        atPole = true;
+        lla[0] = 0.0D;
+        if (z > 0.0) {
+          lla[1] = PIO2;
+        } else if (z < 0.0) {
+          lla[1] = -PIO2;
+        } else {
+          lla[1] = PIO2;
+          lla[2] = -SEMIMINOR_AXIS;
+          return lla;
+        }
+      }
+    }
+
+    final double w2 = x*x + y*y;
+    final double w = StrictMath.sqrt(w2);
+    final double t0 = z * ad_c;
+    final double s0 = StrictMath.sqrt(t0 * t0 + w2);
+    final double sinB0 = t0 / s0;
+    final double cosB0 = w / s0;
+    final double sin3B0 = sinB0 * sinB0 * sinB0;
+    final double t1 = z + SEMIMINOR_AXIS * EP2 * sin3B0;
+    final double sum = w - SEMIMAJOR_AXIS * E2 * cosB0 * cosB0 * cosB0;
+    final double s1 = StrictMath.sqrt(t1 * t1 + sum * sum);
+    final double sinP1 = t1 / s1;
+    final double cosP1 = sum / s1;
+    final double rn = SEMIMAJOR_AXIS / StrictMath.sqrt(1.0D - E2 * sinP1 * sinP1);
+
+    if (cosP1 >= cos67P5) {
+      lla[2] = w / cosP1 - rn;
+    } else if (cosP1 <= -cos67P5) {
+      lla[2] = w / -cosP1 - rn;
+    } else {
+      lla[2] = z / sinP1 + rn * (E2 - 1.0);
+    }
+    if (!atPole) {
+      lla[1] = StrictMath.atan(sinP1/cosP1);
+    }
+    lla[0] = TO_DEGREES * lla[0];
+    lla[1] = TO_DEGREES * lla[1];
+
+    return lla;
+  }
+
+  /**
+   * Converts from geodesic lon lat alt to geocentric earth-centered earth-fixed
+   * @param lon geodesic longitude
+   * @param lat geodesic latitude
+   * @param alt geodesic altitude
+   * @param ecf reusable earth-centered earth-fixed result
+   * @return either a new ecef array or the reusable ecf parameter
+   */
+  public static final double[] llaToECF(double lon, double lat, double alt, double[] ecf) {
+    lon = TO_RADIANS * lon;
+    lat = TO_RADIANS * lat;
+
+    final double sl = sin(lat);
+    final double s2 = sl*sl;
+    final double cl = cos(lat);
+
+    if (ecf == null) {
+      ecf = new double[3];
+    }
+
+    if (lat < -PIO2 && lat > -1.001D * PIO2) {
+      lat = -PIO2;
+    } else if (lat > PIO2 && lat < 1.001D * PIO2) {
+      lat = PIO2;
+    }
+    assert (lat >= -PIO2) || (lat <= PIO2);
+
+    if (lon > StrictMath.PI) {
+      lon -= (2*StrictMath.PI);
+    }
+
+    final double rn = SEMIMAJOR_AXIS / StrictMath.sqrt(1.0D - E2 * s2);
+    ecf[0] = (rn+alt) * cl * cos(lon);
+    ecf[1] = (rn+alt) * cl * sin(lon);
+    ecf[2] = ((rn*(1.0-E2))+alt)*sl;
+
+    return ecf;
+  }
+
+  /**
+   * Converts from lat lon alt (in degrees) to East North Up right-hand coordinate system
+   * @param lon longitude in degrees
+   * @param lat latitude in degrees
+   * @param alt altitude in meters
+   * @param centerLon reference point longitude in degrees
+   * @param centerLat reference point latitude in degrees
+   * @param centerAlt reference point altitude in meters
+   * @param enu result east, north, up coordinate
+   * @return east, north, up coordinate
+   */
+  public static double[] llaToENU(final double lon, final double lat, final double alt, double centerLon,
+                                  double centerLat, final double centerAlt, double[] enu) {
+    if (enu == null) {
+      enu = new double[3];
+    }
+
+    // convert point to ecf coordinates
+    final double[] ecf = llaToECF(lon, lat, alt, null);
+
+    // convert from ecf to enu
+    return ecfToENU(ecf[0], ecf[1], ecf[2], centerLon, centerLat, centerAlt, enu);
+  }
+
+  /**
+   * Converts from East North Up right-hand rule to lat lon alt in degrees
+   * @param x easting (in meters)
+   * @param y northing (in meters)
+   * @param z up (in meters)
+   * @param centerLon reference point longitude (in degrees)
+   * @param centerLat reference point latitude (in degrees)
+   * @param centerAlt reference point altitude (in meters)
+   * @param lla resulting lat, lon, alt point (in degrees)
+   * @return lat, lon, alt point (in degrees)
+   */
+  public static double[] enuToLLA(final double x, final double y, final double z, final double centerLon,
+                                  final double centerLat, final double centerAlt, double[] lla) {
+    // convert enuToECF
+    if (lla == null) {
+      lla = new double[3];
+    }
+
+    // convert enuToECF, storing intermediate result in lla
+    lla = enuToECF(x, y, z, centerLon, centerLat, centerAlt, lla);
+
+    // convert ecf to LLA
+    return ecfToLLA(lla[0], lla[1], lla[2], lla);
+  }
+
+  /**
+   * Convert from Earth-Centered-Fixed to Easting, Northing, Up Right Hand System
+   * @param x ECF X coordinate (in meters)
+   * @param y ECF Y coordinate (in meters)
+   * @param z ECF Z coordinate (in meters)
+   * @param centerLon ENU origin longitude (in degrees)
+   * @param centerLat ENU origin latitude (in degrees)
+   * @param centerAlt ENU altitude (in meters)
+   * @param enu reusable enu result
+   * @return Easting, Northing, Up coordinate
+   */
+  public static double[] ecfToENU(double x, double y, double z, final double centerLon,
+                                  final double centerLat, final double centerAlt, double[] enu) {
+    if (enu == null) {
+      enu = new double[3];
+    }
+
+    // create rotation matrix and rotate to enu orientation
+    final double[][] phi = createPhiTransform(centerLon, centerLat, null);
+
+    // convert origin to ENU
+    final double[] originECF = llaToECF(centerLon, centerLat, centerAlt, null);
+    final double[] originENU = new double[3];
+    originENU[0] = ((phi[0][0] * originECF[0]) + (phi[0][1] * originECF[1]) + (phi[0][2] * originECF[2]));
+    originENU[1] = ((phi[1][0] * originECF[0]) + (phi[1][1] * originECF[1]) + (phi[1][2] * originECF[2]));
+    originENU[2] = ((phi[2][0] * originECF[0]) + (phi[2][1] * originECF[1]) + (phi[2][2] * originECF[2]));
+
+    // rotate then translate
+    enu[0] = ((phi[0][0] * x) + (phi[0][1] * y) + (phi[0][2] * z)) - originENU[0];
+    enu[1] = ((phi[1][0] * x) + (phi[1][1] * y) + (phi[1][2] * z)) - originENU[1];
+    enu[2] = ((phi[2][0] * x) + (phi[2][1] * y) + (phi[2][2] * z)) - originENU[2];
+
+    return enu;
+  }
+
+  /**
+   * Convert from Easting, Northing, Up Right-Handed system to Earth Centered Fixed system
+   * @param x ENU x coordinate (in meters)
+   * @param y ENU y coordinate (in meters)
+   * @param z ENU z coordinate (in meters)
+   * @param centerLon ENU origin longitude (in degrees)
+   * @param centerLat ENU origin latitude (in degrees)
+   * @param centerAlt ENU origin altitude (in meters)
+   * @param ecf reusable ecf result
+   * @return ecf result coordinate
+   */
+  public static double[] enuToECF(final double x, final double y, final double z, double centerLon,
+                                  double centerLat, final double centerAlt, double[] ecf) {
+    if (ecf == null) {
+      ecf = new double[3];
+    }
+
+    double[][] phi = createTransposedPhiTransform(centerLon, centerLat, null);
+    double[] ecfOrigin = llaToECF(centerLon, centerLat, centerAlt, null);
+
+    // rotate and translate
+    ecf[0] = (phi[0][0]*x + phi[0][1]*y + phi[0][2]*z) + ecfOrigin[0];
+    ecf[1] = (phi[1][0]*x + phi[1][1]*y + phi[1][2]*z) + ecfOrigin[1];
+    ecf[2] = (phi[2][0]*x + phi[2][1]*y + phi[2][2]*z) + ecfOrigin[2];
+
+    return ecf;
+  }
+
+  /**
+   * Create the rotation matrix for converting Earth Centered Fixed to Easting Northing Up
+   * @param originLon ENU origin longitude (in degrees)
+   * @param originLat ENU origin latitude (in degrees)
+   * @param phiMatrix reusable phi matrix result
+   * @return phi rotation matrix
+   */
+  private static double[][] createPhiTransform(double originLon, double originLat, double[][] phiMatrix) {
+
+    if (phiMatrix == null) {
+      phiMatrix = new double[3][3];
+    }
+
+    originLon = TO_RADIANS * originLon;
+    originLat = TO_RADIANS * originLat;
+
+    final double sLon = sin(originLon);
+    final double cLon = cos(originLon);
+    final double sLat = sin(originLat);
+    final double cLat = cos(originLat);
+
+    phiMatrix[0][0] = -sLon;
+    phiMatrix[0][1] = cLon;
+    phiMatrix[0][2] = 0.0D;
+    phiMatrix[1][0] = -sLat * cLon;
+    phiMatrix[1][1] = -sLat * sLon;
+    phiMatrix[1][2] = cLat;
+    phiMatrix[2][0] = cLat * cLon;
+    phiMatrix[2][1] = cLat * sLon;
+    phiMatrix[2][2] = sLat;
+
+    return phiMatrix;
+  }
+
+  /**
+   * Create the transposed rotation matrix for converting Easting Northing Up coordinates to Earth Centered Fixed
+   * @param originLon ENU origin longitude (in degrees)
+   * @param originLat ENU origin latitude (in degrees)
+   * @param phiMatrix reusable phi rotation matrix result
+   * @return transposed phi rotation matrix
+   */
+  private static double[][] createTransposedPhiTransform(double originLon, double originLat, double[][] phiMatrix) {
+
+    if (phiMatrix == null) {
+      phiMatrix = new double[3][3];
+    }
+
+    originLon = TO_RADIANS * originLon;
+    originLat = TO_RADIANS * originLat;
+
+    final double sLat = sin(originLat);
+    final double cLat = cos(originLat);
+    final double sLon = sin(originLon);
+    final double cLon = cos(originLon);
+
+    phiMatrix[0][0] = -sLon;
+    phiMatrix[1][0] = cLon;
+    phiMatrix[2][0] = 0.0D;
+    phiMatrix[0][1] = -sLat * cLon;
+    phiMatrix[1][1] = -sLat * sLon;
+    phiMatrix[2][1] = cLat;
+    phiMatrix[0][2] = cLat * cLon;
+    phiMatrix[1][2] = cLat * sLon;
+    phiMatrix[2][2] = sLat;
+
+    return phiMatrix;
+  }
+
+  /**
+   * Finds a point along a bearing from a given lon,lat geolocation using vincenty's distance formula
+   *
+   * @param lon origin longitude in degrees
+   * @param lat origin latitude in degrees
+   * @param bearing azimuthal bearing in degrees
+   * @param dist distance in meters
+   * @param pt resulting point
+   * @return the point along a bearing at a given distance in meters
+   */
+  public static final double[] pointFromLonLatBearingVincenty(double lon, double lat, double bearing, double dist, double[] pt) {
+
+    if (pt == null) {
+      pt = new double[2];
+    }
+
+    final double alpha1 = TO_RADIANS * bearing;
+    final double cosA1 = cos(alpha1);
+    final double sinA1 = sin(alpha1);
+    final double tanU1 = (1-FLATTENING) * tan(TO_RADIANS * lat);
+    final double cosU1 = 1 / StrictMath.sqrt((1+tanU1*tanU1));
+    final double sinU1 = tanU1*cosU1;
+    final double sig1 = StrictMath.atan2(tanU1, cosA1);
+    final double sinAlpha = cosU1 * sinA1;
+    final double cosSqAlpha = 1 - sinAlpha*sinAlpha;
+    final double uSq = cosSqAlpha * EP2;
+    final double A = 1 + uSq/16384D*(4096D + uSq * (-768D + uSq * (320D - 175D*uSq)));
+    final double B = uSq/1024D * (256D + uSq * (-128D + uSq * (74D - 47D * uSq)));
+
+    double sigma = dist / (SEMIMINOR_AXIS*A);
+    double sigmaP;
+    double sinSigma, cosSigma, cos2SigmaM, deltaSigma;
+
+    do {
+      cos2SigmaM = cos(2*sig1 + sigma);
+      sinSigma = sin(sigma);
+      cosSigma = cos(sigma);
+
+      deltaSigma = B * sinSigma * (cos2SigmaM + (B/4D) * (cosSigma*(-1+2*cos2SigmaM*cos2SigmaM)-
+          (B/6) * cos2SigmaM*(-3+4*sinSigma*sinSigma)*(-3+4*cos2SigmaM*cos2SigmaM)));
+      sigmaP = sigma;
+      sigma = dist / (SEMIMINOR_AXIS*A) + deltaSigma;
+    } while (StrictMath.abs(sigma-sigmaP) > 1E-12);
+
+    final double tmp = sinU1*sinSigma - cosU1*cosSigma*cosA1;
+    final double lat2 = StrictMath.atan2(sinU1*cosSigma + cosU1*sinSigma*cosA1,
+        (1-FLATTENING) * StrictMath.sqrt(sinAlpha*sinAlpha + tmp*tmp));
+    final double lambda = StrictMath.atan2(sinSigma*sinA1, cosU1*cosSigma - sinU1*sinSigma*cosA1);
+    final double c = FLATTENING/16 * cosSqAlpha * (4 + FLATTENING * (4 - 3 * cosSqAlpha));
+
+    final double lam = lambda - (1-c) * FLATTENING * sinAlpha *
+        (sigma + c * sinSigma * (cos2SigmaM + c * cosSigma * (-1 + 2* cos2SigmaM*cos2SigmaM)));
+    pt[0] = normalizeLon(lon + TO_DEGREES * lam);
+    pt[1] = normalizeLat(TO_DEGREES * lat2);
+
+    return pt;
+  }
+
+  /**
+   * Finds a point along a bearing from a given lon,lat geolocation using great circle arc
+   *
+   * @param lon origin longitude in degrees
+   * @param lat origin latitude in degrees
+   * @param bearing azimuthal bearing in degrees
+   * @param dist distance in meters
+   * @param pt resulting point
+   * @return the point along a bearing at a given distance in meters
+   */
+  public static final double[] pointFromLonLatBearingGreatCircle(double lon, double lat, double bearing, double dist, double[] pt) {
+
+    if (pt == null) {
+      pt = new double[2];
+    }
+
+    lon *= TO_RADIANS;
+    lat *= TO_RADIANS;
+    bearing *= TO_RADIANS;
+
+    final double cLat = cos(lat);
+    final double sLat = sin(lat);
+    final double sinDoR = sin(dist / GeoProjectionUtils.SEMIMAJOR_AXIS);
+    final double cosDoR = cos(dist / GeoProjectionUtils.SEMIMAJOR_AXIS);
+
+    pt[1] = asin(sLat*cosDoR + cLat * sinDoR * cos(bearing));
+    pt[0] = TO_DEGREES * (lon + Math.atan2(sin(bearing) * sinDoR * cLat, cosDoR - sLat * sin(pt[1])));
+    pt[1] *= TO_DEGREES;
+
+    return pt;
+  }
+
+  /**
+   * Finds the bearing (in degrees) between 2 geo points (lon, lat) using great circle arc
+   * @param lon1 first point longitude in degrees
+   * @param lat1 first point latitude in degrees
+   * @param lon2 second point longitude in degrees
+   * @param lat2 second point latitude in degrees
+   * @return the bearing (in degrees) between the two provided points
+   */
+  public static double bearingGreatCircle(double lon1, double lat1, double lon2, double lat2) {
+    double dLon = (lon2 - lon1) * TO_RADIANS;
+    lat2 *= TO_RADIANS;
+    lat1 *= TO_RADIANS;
+    double y = sin(dLon) * cos(lat2);
+    double x = cos(lat1) * sin(lat2) - sin(lat1) * cos(lat2) * cos(dLon);
+    return Math.atan2(y, x) * TO_DEGREES;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/665041c5/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoRect.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoRect.java b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoRect.java
new file mode 100644
index 0000000..fa93e61
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoRect.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.util;
+
+/** Represents a lat/lon rectangle. */
+public class GeoRect {
+  public final double minLon;
+  public final double maxLon;
+  public final double minLat;
+  public final double maxLat;
+
+  public GeoRect(double minLon, double maxLon, double minLat, double maxLat) {
+    if (GeoUtils.isValidLon(minLon) == false) {
+      throw new IllegalArgumentException("invalid minLon " + minLon);
+    }
+    if (GeoUtils.isValidLon(maxLon) == false) {
+      throw new IllegalArgumentException("invalid maxLon " + maxLon);
+    }
+    if (GeoUtils.isValidLat(minLat) == false) {
+      throw new IllegalArgumentException("invalid minLat " + minLat);
+    }
+    if (GeoUtils.isValidLat(maxLat) == false) {
+      throw new IllegalArgumentException("invalid maxLat " + maxLat);
+    }
+    this.minLon = minLon;
+    this.maxLon = maxLon;
+    this.minLat = minLat;
+    this.maxLat = maxLat;
+    assert maxLat >= minLat;
+
+    // NOTE: cannot assert maxLon >= minLon since this rect could cross the dateline
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder b = new StringBuilder();
+    b.append("GeoRect(lon=");
+    b.append(minLon);
+    b.append(" TO ");
+    b.append(maxLon);
+    if (maxLon < minLon) {
+      b.append(" (crosses dateline!)");
+    }
+    b.append(" lat=");
+    b.append(minLat);
+    b.append(" TO ");
+    b.append(maxLat);
+    b.append(")");
+
+    return b.toString();
+  }
+}