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/04 07:08:14 UTC

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java b/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
deleted file mode 100644
index ec75a6f..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
+++ /dev/null
@@ -1,279 +0,0 @@
-package org.apache.lucene.spatial.serialized;
-
-/*
- * 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.
- */
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.FilterOutputStream;
-import java.io.IOException;
-import java.util.Map;
-
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.io.BinaryCodec;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
-import org.apache.lucene.document.BinaryDocValuesField;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.RandomAccessWeight;
-import org.apache.lucene.search.TwoPhaseIterator;
-import org.apache.lucene.search.Weight;
-import org.apache.lucene.spatial.SpatialStrategy;
-import org.apache.lucene.spatial.query.SpatialArgs;
-import org.apache.lucene.spatial.util.DistanceToShapeValueSource;
-import org.apache.lucene.spatial.util.ShapePredicateValueSource;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-
-
-/**
- * A SpatialStrategy based on serializing a Shape stored into BinaryDocValues.
- * This is not at all fast; it's designed to be used in conjunction with another index based
- * SpatialStrategy that is approximated (like {@link org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy})
- * to add precision or eventually make more specific / advanced calculations on the per-document
- * geometry.
- * The serialization uses Spatial4j's {@link com.spatial4j.core.io.BinaryCodec}.
- *
- * @lucene.experimental
- */
-public class SerializedDVStrategy extends SpatialStrategy {
-
-  /**
-   * A cache heuristic for the buf size based on the last shape size.
-   */
-  //TODO do we make this non-volatile since it's merely a heuristic?
-  private volatile int indexLastBufSize = 8 * 1024;//8KB default on first run
-
-  /**
-   * Constructs the spatial strategy with its mandatory arguments.
-   */
-  public SerializedDVStrategy(SpatialContext ctx, String fieldName) {
-    super(ctx, fieldName);
-  }
-
-  @Override
-  public Field[] createIndexableFields(Shape shape) {
-    int bufSize = Math.max(128, (int) (this.indexLastBufSize * 1.5));//50% headroom over last
-    ByteArrayOutputStream byteStream = new ByteArrayOutputStream(bufSize);
-    final BytesRef bytesRef = new BytesRef();//receiver of byteStream's bytes
-    try {
-      ctx.getBinaryCodec().writeShape(new DataOutputStream(byteStream), shape);
-      //this is a hack to avoid redundant byte array copying by byteStream.toByteArray()
-      byteStream.writeTo(new FilterOutputStream(null/*not used*/) {
-        @Override
-        public void write(byte[] b, int off, int len) throws IOException {
-          bytesRef.bytes = b;
-          bytesRef.offset = off;
-          bytesRef.length = len;
-        }
-      });
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    this.indexLastBufSize = bytesRef.length;//cache heuristic
-    return new Field[]{new BinaryDocValuesField(getFieldName(), bytesRef)};
-  }
-
-  @Override
-  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
-    //TODO if makeShapeValueSource gets lifted to the top; this could become a generic impl.
-    return new DistanceToShapeValueSource(makeShapeValueSource(), queryPoint, multiplier, ctx);
-  }
-
-  /**
-   * Returns a Query that should be used in a random-access fashion.
-   * Use in another manner will be SLOW.
-   */
-  @Override
-  public Query makeQuery(SpatialArgs args) {
-    ValueSource shapeValueSource = makeShapeValueSource();
-    ShapePredicateValueSource predicateValueSource = new ShapePredicateValueSource(
-        shapeValueSource, args.getOperation(), args.getShape());
-    return new PredicateValueSourceQuery(predicateValueSource);
-  }
-
-  /**
-   * Provides access to each shape per document as a ValueSource in which
-   * {@link org.apache.lucene.queries.function.FunctionValues#objectVal(int)} returns a {@link
-   * Shape}.
-   */ //TODO raise to SpatialStrategy
-  public ValueSource makeShapeValueSource() {
-    return new ShapeDocValueSource(getFieldName(), ctx.getBinaryCodec());
-  }
-
-  /** Warning: don't iterate over the results of this query; it's designed for use in a random-access fashion
-   * by {@link TwoPhaseIterator}.
-   */
-  static class PredicateValueSourceQuery extends Query {
-    private final ValueSource predicateValueSource;//we call boolVal(doc)
-
-    public PredicateValueSourceQuery(ValueSource predicateValueSource) {
-      this.predicateValueSource = predicateValueSource;
-    }
-
-    @Override
-    public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
-      return new RandomAccessWeight(this) {
-        @Override
-        protected Bits getMatchingDocs(LeafReaderContext context) throws IOException {
-          final FunctionValues predFuncValues = predicateValueSource.getValues(null, context);
-          return new Bits() {
-            @Override
-            public boolean get(int index) {
-              return predFuncValues.boolVal(index);
-            }
-
-            @Override
-            public int length() {
-              return context.reader().maxDoc();
-            }
-          };
-        }
-      };
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) return true;
-      if (super.equals(o) == false) return false;
-
-      PredicateValueSourceQuery that = (PredicateValueSourceQuery) o;
-
-      if (!predicateValueSource.equals(that.predicateValueSource)) return false;
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      return super.hashCode() + 31 * predicateValueSource.hashCode();
-    }
-    
-    @Override
-    public String toString(String field) {
-      return "PredicateValueSourceQuery(" +
-               predicateValueSource.toString() +
-             ")";
-    }
-  }//PredicateValueSourceQuery
-
-  /**
-   * Implements a ValueSource by deserializing a Shape in from BinaryDocValues using BinaryCodec.
-   * @see #makeShapeValueSource()
-   */
-  static class ShapeDocValueSource extends ValueSource {
-
-    private final String fieldName;
-    private final BinaryCodec binaryCodec;//spatial4j
-
-    private ShapeDocValueSource(String fieldName, BinaryCodec binaryCodec) {
-      this.fieldName = fieldName;
-      this.binaryCodec = binaryCodec;
-    }
-
-    @Override
-    public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
-      final BinaryDocValues docValues = readerContext.reader().getBinaryDocValues(fieldName);
-
-      return new FunctionValues() {
-        int bytesRefDoc = -1;
-        BytesRefBuilder bytesRef = new BytesRefBuilder();
-
-        boolean fillBytes(int doc) {
-          if (bytesRefDoc != doc) {
-            bytesRef.copyBytes(docValues.get(doc));
-            bytesRefDoc = doc;
-          }
-          return bytesRef.length() != 0;
-        }
-
-        @Override
-        public boolean exists(int doc) {
-          return fillBytes(doc);
-        }
-
-        @Override
-        public boolean bytesVal(int doc, BytesRefBuilder target) {
-          target.clear();
-          if (fillBytes(doc)) {
-            target.copyBytes(bytesRef);
-            return true;
-          } else {
-            return false;
-          }
-        }
-
-        @Override
-        public Object objectVal(int docId) {
-          if (!fillBytes(docId))
-            return null;
-          DataInputStream dataInput = new DataInputStream(
-              new ByteArrayInputStream(bytesRef.bytes(), 0, bytesRef.length()));
-          try {
-            return binaryCodec.readShape(dataInput);
-          } catch (IOException e) {
-            throw new RuntimeException(e);
-          }
-        }
-
-        @Override
-        public Explanation explain(int doc) {
-          return Explanation.match(Float.NaN, toString(doc));
-        }
-
-        @Override
-        public String toString(int doc) {
-          return description() + "=" + objectVal(doc);//TODO truncate?
-        }
-
-      };
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (this == o) return true;
-      if (o == null || getClass() != o.getClass()) return false;
-
-      ShapeDocValueSource that = (ShapeDocValueSource) o;
-
-      if (!fieldName.equals(that.fieldName)) return false;
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      int result = fieldName.hashCode();
-      return result;
-    }
-
-    @Override
-    public String description() {
-      return "shapeDocVal(" + fieldName + ")";
-    }
-  }//ShapeDocValueSource
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/package-info.java b/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/package-info.java
deleted file mode 100644
index 7a316d9..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/package-info.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
- 
-/** 
- * Strategies that serialize the shape (non-indexed).
- */
-package org.apache.lucene.spatial.serialized;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java b/lucene/spatial/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java
deleted file mode 100644
index 19118a5..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java
+++ /dev/null
@@ -1,186 +0,0 @@
-package org.apache.lucene.spatial.spatial4j;
-
-/*
- * 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.
- */
-
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
-import com.spatial4j.core.shape.impl.RectangleImpl;
-import org.apache.lucene.geo3d.LatLonBounds;
-import org.apache.lucene.geo3d.GeoArea;
-import org.apache.lucene.geo3d.GeoAreaFactory;
-import org.apache.lucene.geo3d.GeoPoint;
-import org.apache.lucene.geo3d.GeoShape;
-import org.apache.lucene.geo3d.PlanetModel;
-
-/**
- * A Spatial4j Shape wrapping a {@link GeoShape} ("Geo3D") -- a 3D planar geometry based Spatial4j Shape implementation.
- * Geo3D implements shapes on the surface of a sphere or ellipsoid.
- *
- * @lucene.experimental
- */
-public class Geo3dShape implements Shape {
-  /** The required size of this adjustment depends on the actual planetary model chosen.
-   * This value is big enough to account for WGS84. */
-  protected static final double ROUNDOFF_ADJUSTMENT = 0.05;
-
-  public final SpatialContext ctx;
-  public final GeoShape shape;
-  public final PlanetModel planetModel;
-
-  private volatile Rectangle boundingBox = null; // lazy initialized
-
-  public Geo3dShape(final GeoShape shape, final SpatialContext ctx) {
-    this(PlanetModel.SPHERE, shape, ctx);
-  }
-  
-  public Geo3dShape(final PlanetModel planetModel, final GeoShape shape, final SpatialContext ctx) {
-    if (!ctx.isGeo()) {
-      throw new IllegalArgumentException("SpatialContext.isGeo() must be true");
-    }
-    this.ctx = ctx;
-    this.planetModel = planetModel;
-    this.shape = shape;
-  }
-
-  @Override
-  public SpatialContext getContext() {
-    return ctx;
-  }
-
-  @Override
-  public SpatialRelation relate(Shape other) {
-    if (other instanceof Rectangle)
-      return relate((Rectangle)other);
-    else if (other instanceof Point)
-      return relate((Point)other);
-    else
-      throw new RuntimeException("Unimplemented shape relationship determination: " + other.getClass());
-  }
-
-  protected SpatialRelation relate(Rectangle r) {
-    // Construct the right kind of GeoArea first
-    GeoArea geoArea = GeoAreaFactory.makeGeoArea(planetModel,
-        r.getMaxY() * DistanceUtils.DEGREES_TO_RADIANS,
-        r.getMinY() * DistanceUtils.DEGREES_TO_RADIANS,
-        r.getMinX() * DistanceUtils.DEGREES_TO_RADIANS,
-        r.getMaxX() * DistanceUtils.DEGREES_TO_RADIANS);
-    int relationship = geoArea.getRelationship(shape);
-    if (relationship == GeoArea.WITHIN)
-      return SpatialRelation.WITHIN;
-    else if (relationship == GeoArea.CONTAINS)
-      return SpatialRelation.CONTAINS;
-    else if (relationship == GeoArea.OVERLAPS)
-      return SpatialRelation.INTERSECTS;
-    else if (relationship == GeoArea.DISJOINT)
-      return SpatialRelation.DISJOINT;
-    else
-      throw new RuntimeException("Unknown relationship returned: "+relationship);
-  }
-
-  protected SpatialRelation relate(Point p) {
-    // Create a GeoPoint
-    GeoPoint point = new GeoPoint(planetModel, p.getY()* DistanceUtils.DEGREES_TO_RADIANS, p.getX()* DistanceUtils.DEGREES_TO_RADIANS);
-    if (shape.isWithin(point)) {
-      // Point within shape
-      return SpatialRelation.CONTAINS;
-    }
-    return SpatialRelation.DISJOINT;
-  }
-
-
-  
-  @Override
-  public Rectangle getBoundingBox() {
-    Rectangle bbox = this.boundingBox;//volatile read once
-    if (bbox == null) {
-      LatLonBounds bounds = new LatLonBounds();
-      shape.getBounds(bounds);
-      double leftLon;
-      double rightLon;
-      if (bounds.checkNoLongitudeBound()) {
-        leftLon = -180.0;
-        rightLon = 180.0;
-      } else {
-        leftLon = bounds.getLeftLongitude().doubleValue() * DistanceUtils.RADIANS_TO_DEGREES;
-        rightLon = bounds.getRightLongitude().doubleValue() * DistanceUtils.RADIANS_TO_DEGREES;
-      }
-      double minLat;
-      if (bounds.checkNoBottomLatitudeBound()) {
-        minLat = -90.0;
-      } else {
-        minLat = bounds.getMinLatitude().doubleValue() * DistanceUtils.RADIANS_TO_DEGREES;
-      }
-      double maxLat;
-      if (bounds.checkNoTopLatitudeBound()) {
-        maxLat = 90.0;
-      } else {
-        maxLat = bounds.getMaxLatitude().doubleValue() * DistanceUtils.RADIANS_TO_DEGREES;
-      }
-      bbox = new RectangleImpl(leftLon, rightLon, minLat, maxLat, ctx).getBuffered(ROUNDOFF_ADJUSTMENT, ctx);
-      this.boundingBox = bbox;
-    }
-    return bbox;
-  }
-
-  @Override
-  public boolean hasArea() {
-    return true;
-  }
-
-  @Override
-  public double getArea(SpatialContext ctx) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Point getCenter() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Shape getBuffered(double distance, SpatialContext ctx) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean isEmpty() {
-    return false;
-  }
-
-  @Override
-  public String toString() {
-    return "Geo3dShape{planetmodel=" + planetModel + ", shape=" + shape + '}';
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (!(other instanceof Geo3dShape))
-      return false;
-    Geo3dShape tr = (Geo3dShape)other;
-    return tr.ctx.equals(ctx) && tr.planetModel.equals(planetModel) && tr.shape.equals(shape);
-  }
-
-  @Override
-  public int hashCode() {
-    return planetModel.hashCode() + shape.hashCode();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial/src/java/org/apache/lucene/spatial/spatial4j/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/spatial4j/package-info.java b/lucene/spatial/src/java/org/apache/lucene/spatial/spatial4j/package-info.java
deleted file mode 100644
index 7815318..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/spatial4j/package-info.java
+++ /dev/null
@@ -1,19 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/** Spatial4j stuff that ideally belongs in Spatial4j (isn't related to Lucene). */
-package org.apache.lucene.spatial.spatial4j;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial/src/java/org/apache/lucene/spatial/util/CachingDoubleValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/util/CachingDoubleValueSource.java b/lucene/spatial/src/java/org/apache/lucene/spatial/util/CachingDoubleValueSource.java
deleted file mode 100644
index 9a91782..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/util/CachingDoubleValueSource.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.lucene.spatial.util;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Caches the doubleVal of another value source in a HashMap
- * so that it is computed only once.
- * @lucene.internal
- */
-public class CachingDoubleValueSource extends ValueSource {
-
-  final ValueSource source;
-  final Map<Integer, Double> cache;
-
-  public CachingDoubleValueSource( ValueSource source )
-  {
-    this.source = source;
-    cache = new HashMap<>();
-  }
-
-  @Override
-  public String description() {
-    return "Cached["+source.description()+"]";
-  }
-
-  @Override
-  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
-    final int base = readerContext.docBase;
-    final FunctionValues vals = source.getValues(context,readerContext);
-    return new FunctionValues() {
-
-      @Override
-      public double doubleVal(int doc) {
-        Integer key = Integer.valueOf( base+doc );
-        Double v = cache.get( key );
-        if( v == null ) {
-          v = Double.valueOf( vals.doubleVal(doc) );
-          cache.put( key, v );
-        }
-        return v.doubleValue();
-      }
-
-      @Override
-      public float floatVal(int doc) {
-        return (float)doubleVal(doc);
-      }
-
-      @Override
-      public String toString(int doc) {
-        return doubleVal(doc)+"";
-      }
-    };
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    CachingDoubleValueSource that = (CachingDoubleValueSource) o;
-
-    if (source != null ? !source.equals(that.source) : that.source != null) return false;
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    return source != null ? source.hashCode() : 0;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java b/lucene/spatial/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
deleted file mode 100644
index 703586b..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
+++ /dev/null
@@ -1,123 +0,0 @@
-package org.apache.lucene.spatial.util;
-
-/*
- * 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.
- */
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
-import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
-
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceCalculator;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
-
-/**
- * The distance from a provided Point to a Point retrieved from a ValueSource via
- * {@link org.apache.lucene.queries.function.FunctionValues#objectVal(int)}. The distance
- * is calculated via a {@link com.spatial4j.core.distance.DistanceCalculator}.
- *
- * @lucene.experimental
- */
-public class DistanceToShapeValueSource extends ValueSource {
-  private final ValueSource shapeValueSource;
-  private final Point queryPoint;
-  private final double multiplier;
-  private final DistanceCalculator distCalc;
-
-  //TODO if FunctionValues returns NaN; will things be ok?
-  private final double nullValue;//computed
-
-  public DistanceToShapeValueSource(ValueSource shapeValueSource, Point queryPoint,
-                                    double multiplier, SpatialContext ctx) {
-    this.shapeValueSource = shapeValueSource;
-    this.queryPoint = queryPoint;
-    this.multiplier = multiplier;
-    this.distCalc = ctx.getDistCalc();
-    this.nullValue =
-        (ctx.isGeo() ? 180 * multiplier : Double.MAX_VALUE);
-  }
-
-  @Override
-  public String description() {
-    return "distance(" + queryPoint + " to " + shapeValueSource.description() + ")*" + multiplier + ")";
-  }
-
-  @Override
-  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
-    shapeValueSource.createWeight(context, searcher);
-  }
-
-  @Override
-  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
-    final FunctionValues shapeValues = shapeValueSource.getValues(context, readerContext);
-
-    return new DoubleDocValues(this) {
-      @Override
-      public double doubleVal(int doc) {
-        Shape shape = (Shape) shapeValues.objectVal(doc);
-        if (shape == null || shape.isEmpty())
-          return nullValue;
-        Point pt = shape.getCenter();
-        return distCalc.distance(queryPoint, pt) * multiplier;
-      }
-
-      @Override
-      public Explanation explain(int doc) {
-        Explanation exp = super.explain(doc);
-        List<Explanation> details = new ArrayList<>(Arrays.asList(exp.getDetails()));
-        details.add(shapeValues.explain(doc));
-        return Explanation.match(exp.getValue(), exp.getDescription(), details);
-      }
-    };
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    DistanceToShapeValueSource that = (DistanceToShapeValueSource) o;
-
-    if (!queryPoint.equals(that.queryPoint)) return false;
-    if (Double.compare(that.multiplier, multiplier) != 0) return false;
-    if (!shapeValueSource.equals(that.shapeValueSource)) return false;
-    if (!distCalc.equals(that.distCalc)) return false;
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int result;
-    long temp;
-    result = shapeValueSource.hashCode();
-    result = 31 * result + queryPoint.hashCode();
-    temp = Double.doubleToLongBits(multiplier);
-    result = 31 * result + (int) (temp ^ (temp >>> 32));
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoBoundingBox.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoBoundingBox.java b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoBoundingBox.java
new file mode 100644
index 0000000..37018d1
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoBoundingBox.java
@@ -0,0 +1,74 @@
+package org.apache.lucene.spatial.util;
+
+/*
+ * 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.
+ */
+
+/** Represents a lat/lon rectangle. */
+public class GeoBoundingBox {
+  /** minimum longitude (in degrees) */
+  public final double minLon;
+  /** minimum latitude (in degrees) */
+  public final double maxLon;
+  /** maximum longitude (in degrees) */
+  public final double minLat;
+  /** maximum latitude (in degrees) */
+  public final double maxLat;
+
+  /**
+   * Constructs a Geospatial Bounding Box
+   */
+  public GeoBoundingBox(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 bbox could cross the dateline
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder b = new StringBuilder();
+    b.append("GeoBoundingBox(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();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b216c118/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..9c8639a
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoDistanceUtils.java
@@ -0,0 +1,222 @@
+package org.apache.lucene.spatial.util;
+
+/*
+ * 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.
+ */
+
+import org.apache.lucene.util.SloppyMath;
+
+/**
+ * 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 = SloppyMath.TO_RADIANS * (lat2 - lat1);
+    double dLon = SloppyMath.TO_RADIANS * (lon2 - lon1);
+    lat1 = SloppyMath.TO_RADIANS * (lat1);
+    lat2 = SloppyMath.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/b216c118/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..f087824
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoHashUtils.java
@@ -0,0 +1,284 @@
+package org.apache.lucene.spatial.util;
+
+/*
+ * 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.
+ */
+
+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/b216c118/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..ee5b3c7
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/util/GeoProjectionUtils.java
@@ -0,0 +1,466 @@
+package org.apache.lucene.spatial.util;
+
+/*
+ * 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.
+ */
+
+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 / SEMIMAJOR_AXIS);
+    final double cosDoR = cos(dist / 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;
+  }
+}