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/10 04:51:54 UTC

[1/3] lucene-solr git commit: LUCENE-6997: refactor GeoPointField and query classes from lucene.spatial to lucene.spatial.geopoint package

Repository: lucene-solr
Updated Branches:
  refs/heads/master b889109da -> 7d8f0127b


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointMultiTermQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointMultiTermQuery.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointMultiTermQuery.java
deleted file mode 100644
index f11c4a6..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointMultiTermQuery.java
+++ /dev/null
@@ -1,166 +0,0 @@
-package org.apache.lucene.spatial.search;
-
-/*
- * 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 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.util.AttributeSource;
-import org.apache.lucene.spatial.document.GeoPointField;
-import org.apache.lucene.spatial.document.GeoPointField.TermEncoding;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
-import org.apache.lucene.spatial.util.GeoRelationUtils;
-import org.apache.lucene.spatial.util.GeoUtils;
-import org.apache.lucene.util.SloppyMath;
-
-/**
- * TermQuery for GeoPointField for overriding {@link org.apache.lucene.search.MultiTermQuery} methods specific to
- * Geospatial operations
- *
- * @lucene.experimental
- */
-abstract class GeoPointMultiTermQuery extends MultiTermQuery {
-  // simple bounding box optimization - no objects used to avoid dependencies
-  protected final double minLon;
-  protected final double minLat;
-  protected final double maxLon;
-  protected final double maxLat;
-  protected final short maxShift;
-  protected final TermEncoding termEncoding;
-  protected final CellComparator cellComparator;
-
-  /**
-   * Constructs a query matching terms that cannot be represented with a single
-   * Term.
-   */
-  public GeoPointMultiTermQuery(String field, final TermEncoding termEncoding, 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);
-    }
-
-    final long minHash = GeoEncodingUtils.mortonHash(minLon, minLat);
-    final long maxHash = GeoEncodingUtils.mortonHash(maxLon, maxLat);
-    this.minLon = GeoEncodingUtils.mortonUnhashLon(minHash);
-    this.minLat = GeoEncodingUtils.mortonUnhashLat(minHash);
-    this.maxLon = GeoEncodingUtils.mortonUnhashLon(maxHash);
-    this.maxLat = GeoEncodingUtils.mortonUnhashLat(maxHash);
-
-    this.maxShift = computeMaxShift();
-    this.termEncoding = termEncoding;
-    this.cellComparator = newCellComparator();
-
-    this.rewriteMethod = GEO_CONSTANT_SCORE_REWRITE;
-  }
-
-  public static final RewriteMethod GEO_CONSTANT_SCORE_REWRITE = new RewriteMethod() {
-    @Override
-    public Query rewrite(IndexReader reader, MultiTermQuery query) {
-      return new GeoPointTermQueryConstantScoreWrapper<>((GeoPointMultiTermQuery)query);
-    }
-  };
-
-  @Override @SuppressWarnings("unchecked")
-  protected TermsEnum getTermsEnum(final Terms terms, AttributeSource atts) throws IOException {
-    return GeoPointTermsEnum.newInstance(terms.iterator(), this);
-  }
-
-  /**
-   * Computes the maximum shift based on the diagonal distance of the bounding box
-   */
-  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)
-    final short shiftFactor;
-
-    // compute diagonal distance
-    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);
-  }
-
-  /**
-   * Abstract method to construct the class that handles all geo point relations
-   * (e.g., GeoPointInPolygon)
-   */
-  abstract protected CellComparator newCellComparator();
-
-  /**
-   * Base class for all geo point relation comparators
-   */
-  static abstract class CellComparator {
-    protected final GeoPointMultiTermQuery geoPointQuery;
-
-    CellComparator(GeoPointMultiTermQuery query) {
-      this.geoPointQuery = query;
-    }
-
-    /**
-     * 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, geoPointQuery.minLon, geoPointQuery.minLat,
-          geoPointQuery.maxLon, geoPointQuery.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(geoPointQuery.minLon, geoPointQuery.minLat, geoPointQuery.maxLon,
-          geoPointQuery.maxLat, minLon, minLat, maxLon, maxLat);
-    }
-
-    /**
-     * Determine whether the quad-cell crosses the shape
-     */
-    abstract protected boolean cellCrosses(final double minLon, final double minLat, final double maxLon, final double maxLat);
-
-    /**
-     * Determine whether quad-cell is within the shape
-     */
-    abstract protected 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}
-     */
-    abstract protected boolean cellIntersectsShape(final double minLon, final double minLat, final double maxLon, final double maxLat);
-
-    abstract protected boolean postFilter(final double lon, final double lat);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointNumericTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointNumericTermsEnum.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointNumericTermsEnum.java
deleted file mode 100644
index adfb6a0..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointNumericTermsEnum.java
+++ /dev/null
@@ -1,161 +0,0 @@
-package org.apache.lucene.spatial.search;
-
-/*
- * 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.Collections;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.spatial.document.GeoPointField;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.LegacyNumericUtils;
-
-/**
- * Decomposes a given {@link GeoPointMultiTermQuery} into a set of terms that represent the query criteria using
- * {@link org.apache.lucene.spatial.document.GeoPointField.TermEncoding#NUMERIC} method defined by
- * {@link org.apache.lucene.analysis.LegacyNumericTokenStream}. The terms are then enumerated by the
- * {@link GeoPointTermQueryConstantScoreWrapper} and all docs whose GeoPoint fields match the prefix terms or
- * pass the {@link GeoPointMultiTermQuery.CellComparator#postFilter} criteria are returned in the resulting DocIdSet.
- *
- *  @lucene.experimental
- */
-@Deprecated
-final class GeoPointNumericTermsEnum extends GeoPointTermsEnum {
-  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)
-  private final short DETAIL_LEVEL;
-
-  GeoPointNumericTermsEnum(final TermsEnum tenum, final GeoPointMultiTermQuery query) {
-    super(tenum, query);
-    DETAIL_LEVEL = (short)(((GeoEncodingUtils.BITS<<1)-this.maxShift)/2);
-    computeRange(0L, (short) (((GeoEncodingUtils.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 = GeoEncodingUtils.mortonUnhashLon(start);
-    final double minLat = GeoEncodingUtils.mortonUnhashLat(start);
-    final double maxLon = GeoEncodingUtils.mortonUnhashLon(end);
-    final double maxLat = GeoEncodingUtils.mortonUnhashLat(end);
-
-    final short level = (short)((GeoEncodingUtils.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 && relationImpl.cellWithin(minLon, minLat, maxLon, maxLat);
-    if (within || (level == DETAIL_LEVEL && relationImpl.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 && relationImpl.cellIntersectsMBR(minLon, minLat, maxLon, maxLat)) {
-      computeRange(start, (short) (res - 1));
-    }
-  }
-
-  @Override
-  protected final BytesRef peek() {
-    rangeBounds.get(0).fillBytesRef(this.nextSubRangeBRB);
-    return nextSubRangeBRB.get();
-  }
-
-  @Override
-  protected void nextRange() {
-    currentRange = rangeBounds.remove(0);
-    super.nextRange();
-  }
-
-  @Override
-  protected final BytesRef nextSeekTerm(BytesRef term) {
-    while (hasNext()) {
-      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) > 0) {
-        nextRange();
-        if (!rangeBounds.isEmpty()) {
-          continue;
-        }
-      }
-      // never seek backwards, so use current term if lower bound is smaller
-      return (term != null && term.compareTo(currentCell) > 0) ? term : currentCell;
-    }
-
-    // no more sub-range enums available
-    assert rangeBounds.isEmpty();
-    return null;
-  }
-
-  @Override
-  protected final boolean hasNext() {
-    return rangeBounds.isEmpty() == false;
-  }
-
-  /**
-   * Internal class to represent a range along the space filling curve
-   */
-  protected final class Range extends BaseRange {
-    Range(final long lower, final short shift, boolean boundary) {
-      super(lower, shift, boundary);
-    }
-
-    /**
-     * 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.
-     */
-    @Override
-    protected void fillBytesRef(BytesRefBuilder result) {
-      assert result != null;
-      LegacyNumericUtils.longToPrefixCoded(start, shift, result);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointPrefixTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointPrefixTermsEnum.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointPrefixTermsEnum.java
deleted file mode 100644
index d08cd98..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointPrefixTermsEnum.java
+++ /dev/null
@@ -1,237 +0,0 @@
-package org.apache.lucene.spatial.search;
-
-/*
- * 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.index.TermsEnum;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.spatial.document.GeoPointField;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
-
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonHash;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLat;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLon;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.geoCodedToPrefixCoded;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.prefixCodedToGeoCoded;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.getPrefixCodedShift;
-
-/**
- * Decomposes a given {@link GeoPointMultiTermQuery} into a set of terms that represent the query criteria using
- * {@link org.apache.lucene.spatial.document.GeoPointField.TermEncoding#PREFIX} method defined by
- * {@link GeoPointField}. The terms are then enumerated by the
- * {@link GeoPointTermQueryConstantScoreWrapper} and all docs whose GeoPoint fields match the prefix terms or pass
- * the {@link GeoPointMultiTermQuery.CellComparator#postFilter} criteria are returned in the
- * resulting DocIdSet.
- *
- *  @lucene.experimental
- */
-final class GeoPointPrefixTermsEnum extends GeoPointTermsEnum {
-  private final long start;
-
-  private short shift;
-
-  // current range as long
-  private long currStart;
-  private long currEnd;
-
-  private final Range nextRange = new Range(-1, shift, true);
-
-  private boolean hasNext = false;
-
-  private boolean withinOnly = false;
-  private long lastWithin;
-
-  public GeoPointPrefixTermsEnum(final TermsEnum tenum, final GeoPointMultiTermQuery query) {
-    super(tenum, query);
-    this.start = mortonHash(query.minLon, query.minLat);
-    this.currentRange = new Range(0, shift, true);
-    // start shift at maxShift value (from computeMaxShift)
-    this.shift = maxShift;
-    final long mask = (1L << shift) - 1;
-    this.currStart = start & ~mask;
-    this.currEnd = currStart | mask;
-  }
-
-  private boolean within(final double minLon, final double minLat, final double maxLon, final double maxLat) {
-    return relationImpl.cellWithin(minLon, minLat, maxLon, maxLat);
-  }
-
-  private boolean boundary(final double minLon, final double minLat, final double maxLon, final double maxLat) {
-    return shift == maxShift && relationImpl.cellIntersectsShape(minLon, minLat, maxLon, maxLat);
-  }
-
-  private boolean nextWithin() {
-    if (withinOnly == false) {
-      return false;
-    }
-    currStart += (1L << shift);
-    setNextRange(false);
-    currentRange.set(nextRange);
-    hasNext = true;
-
-    withinOnly = lastWithin != currStart;
-    if (withinOnly == false) advanceVariables();
-    return true;
-  }
-
-  private void nextRelation() {
-    double minLon = mortonUnhashLon(currStart);
-    double minLat = mortonUnhashLat(currStart);
-    double maxLon;
-    double maxLat;
-    boolean isWithin;
-    do {
-      maxLon = mortonUnhashLon(currEnd);
-      maxLat = mortonUnhashLat(currEnd);
-
-      // within or a boundary
-      if ((isWithin = within(minLon, minLat, maxLon, maxLat) == true) || boundary(minLon, minLat, maxLon, maxLat) == true) {
-        final int m;
-        if (isWithin == false || (m = shift % GeoPointField.PRECISION_STEP) == 0) {
-          setNextRange(isWithin == false);
-          advanceVariables();
-          break;
-        } else if (shift < 54) {
-          withinOnly = true;
-          shift = (short)(shift - m);
-          lastWithin = currEnd & ~((1L << shift) - 1);
-          setNextRange(false);
-          break;
-        }
-      }
-
-      // within cell but not at a depth factor of PRECISION_STEP
-      if (isWithin == true || (relationImpl.cellIntersectsMBR(minLon, minLat, maxLon , maxLat) == true && shift != maxShift)) {
-        // descend: currStart need not change since shift handles end of range
-        currEnd = currStart | (1L<<--shift) - 1;
-      } else {
-        advanceVariables();
-        minLon = mortonUnhashLon(currStart);
-        minLat = mortonUnhashLat(currStart);
-      }
-    } while(shift < 63);
-  }
-
-  private void setNextRange(final boolean boundary) {
-    nextRange.start = currStart;
-    nextRange.shift = shift;
-    nextRange.boundary = boundary;
-  }
-
-  private void advanceVariables() {
-    /** set next variables */
-    long shiftMask = 1L << shift;
-    // pop-up if shift bit is set
-    while ( (currStart & shiftMask) == shiftMask) {
-      shiftMask = 1L << ++shift;
-    }
-    final long shiftMOne = shiftMask - 1;
-    currStart = currStart & ~shiftMOne | shiftMask;
-    currEnd = currStart | shiftMOne;
-  }
-
-  @Override
-  protected final BytesRef peek() {
-    nextRange.fillBytesRef(nextSubRangeBRB);
-    return super.peek();
-  }
-
-  protected void seek(long term, short res) {
-    if (term < currStart && res < maxShift) {
-      throw new IllegalArgumentException("trying to seek backwards");
-    } else if (term == currStart) {
-      return;
-    }
-    shift = res;
-    currStart = term;
-    currEnd = currStart | ((1L<<shift)-1);
-    withinOnly = false;
-  }
-
-  @Override
-  protected void nextRange() {
-    hasNext = false;
-    super.nextRange();
-  }
-
-  @Override
-  protected final boolean hasNext() {
-    if (hasNext == true || nextWithin()) {
-      return true;
-    }
-    nextRelation();
-    if (currentRange.compareTo(nextRange) != 0) {
-      currentRange.set(nextRange);
-      return (hasNext = true);
-    }
-    return false;
-  }
-
-  @Override
-  protected final BytesRef nextSeekTerm(BytesRef term) {
-    while (hasNext()) {
-      nextRange();
-      if (term == null) {
-        return currentCell;
-      }
-
-      final int comparison = term.compareTo(currentCell);
-      if (comparison > 0) {
-        seek(GeoEncodingUtils.prefixCodedToGeoCoded(term), (short)(64-GeoEncodingUtils.getPrefixCodedShift(term)));
-        continue;
-      }
-      return currentCell;
-    }
-
-    // no more sub-range enums available
-    return null;
-  }
-
-  @Override
-  protected AcceptStatus accept(BytesRef term) {
-    // range < term or range is null
-    while (currentCell == null || term.compareTo(currentCell) > 0) {
-      // no more ranges, be gone
-      if (hasNext() == false) {
-        return AcceptStatus.END;
-      }
-
-      // peek next range, if the range > term then seek
-      final int peekCompare = term.compareTo(peek());
-      if (peekCompare < 0) {
-        return AcceptStatus.NO_AND_SEEK;
-      } else if (peekCompare > 0) {
-        seek(prefixCodedToGeoCoded(term), (short)(64 - getPrefixCodedShift(term)));
-      }
-      nextRange();
-    }
-    return AcceptStatus.YES;
-  }
-
-  protected final class Range extends BaseRange {
-    public Range(final long start, final short res, final boolean boundary) {
-      super(start, res, boundary);
-    }
-
-    @Override
-    protected void fillBytesRef(BytesRefBuilder result) {
-      assert result != null;
-      geoCodedToPrefixCoded(start, shift, result);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/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
deleted file mode 100644
index 46a1783..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointTermQueryConstantScoreWrapper.java
+++ /dev/null
@@ -1,153 +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.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.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.TwoPhaseIterator;
-import org.apache.lucene.search.Weight;
-import org.apache.lucene.util.BitSet;
-import org.apache.lucene.util.DocIdSetBuilder;
-import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.SparseFixedBitSet;
-
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLat;
-import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLon;
-
-
-/**
- * Custom ConstantScoreWrapper for {@code GeoPointMultiTermQuery} that cuts over to DocValues
- * for post filtering boundary ranges. Multi-valued GeoPoint documents are supported.
- *
- * @lucene.experimental
- */
-final class GeoPointTermQueryConstantScoreWrapper <Q extends GeoPointMultiTermQuery> 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) {
-
-      @Override
-      public Scorer scorer(LeafReaderContext context) throws IOException {
-        final Terms terms = context.reader().terms(query.getField());
-        if (terms == null) {
-          return null;
-        }
-
-        final GeoPointTermsEnum termsEnum = (GeoPointTermsEnum)(query.getTermsEnum(terms, null));
-        assert termsEnum != null;
-
-        LeafReader reader = context.reader();
-        // approximation (postfiltering has not yet been applied)
-        DocIdSetBuilder builder = new DocIdSetBuilder(reader.maxDoc());
-        // subset of documents that need no postfiltering, this is purely an optimization
-        final BitSet preApproved;
-        // dumb heuristic: if the field is really sparse, use a sparse impl
-        if (terms.getDocCount() * 100L < reader.maxDoc()) {
-          preApproved = new SparseFixedBitSet(reader.maxDoc());
-        } else {
-          preApproved = new FixedBitSet(reader.maxDoc());
-        }
-        PostingsEnum docs = null;
-
-        while (termsEnum.next() != null) {
-          docs = termsEnum.postings(docs, PostingsEnum.NONE);
-          // boundary terms need post filtering
-          if (termsEnum.boundaryTerm()) {
-            builder.add(docs);
-          } else {
-            int docId;
-            while ((docId = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-              builder.add(docId);
-              preApproved.set(docId);
-            }
-          }
-        }
-
-        DocIdSet set = builder.build();
-        final DocIdSetIterator disi = set.iterator();
-        if (disi == null) {
-          return null;
-        }
-
-        // return two-phase iterator using docvalues to postfilter candidates
-        SortedNumericDocValues sdv = reader.getSortedNumericDocValues(query.getField());
-        TwoPhaseIterator iterator = new TwoPhaseIterator(disi) {
-          @Override
-          public boolean matches() throws IOException {
-            int docId = disi.docID();
-            if (preApproved.get(docId)) {
-              return true;
-            } else {
-              sdv.setDocument(docId);
-              int count = sdv.count();
-              for (int i = 0; i < count; i++) {
-                long hash = sdv.valueAt(i);
-                if (termsEnum.postFilter(mortonUnhashLon(hash), mortonUnhashLat(hash))) {
-                  return true;
-                }
-              }
-              return false;
-            }
-          }
-
-          @Override
-          public float matchCost() {
-            return 20; // TODO: make this fancier
-          }
-        };
-        return new ConstantScoreScorer(this, score(), iterator);
-      }
-    };
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/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
deleted file mode 100644
index 626003a..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointTermsEnum.java
+++ /dev/null
@@ -1,140 +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.search;
-
-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.document.GeoPointField.TermEncoding;
-import org.apache.lucene.spatial.search.GeoPointMultiTermQuery.CellComparator;
-
-/**
- * Base class for {@link GeoPointNumericTermsEnum} and {@link GeoPointPrefixTermsEnum} which compares
- * candidate GeoPointField encoded terms against terms matching the defined query criteria.
- *
- *  @lucene.experimental
- */
-abstract class GeoPointTermsEnum extends FilteredTermsEnum {
-  protected final short maxShift;
-
-  protected BaseRange currentRange;
-  protected BytesRef currentCell;
-  protected final BytesRefBuilder currentCellBRB = new BytesRefBuilder();
-  protected final BytesRefBuilder nextSubRangeBRB = new BytesRefBuilder();
-
-  protected final CellComparator relationImpl;
-
-  GeoPointTermsEnum(final TermsEnum tenum, final GeoPointMultiTermQuery query) {
-    super(tenum);
-    this.maxShift = query.maxShift;
-    this.relationImpl = query.cellComparator;
-  }
-
-  static GeoPointTermsEnum newInstance(final TermsEnum terms, final GeoPointMultiTermQuery query) {
-    if (query.termEncoding == TermEncoding.PREFIX) {
-      return new GeoPointPrefixTermsEnum(terms, query);
-    } else if (query.termEncoding == TermEncoding.NUMERIC) {
-      return new GeoPointNumericTermsEnum(terms, query);
-    }
-    throw new IllegalArgumentException("Invalid GeoPoint TermEncoding " + query.termEncoding);
-  }
-
-  public boolean boundaryTerm() {
-    if (currentCell == null) {
-      throw new IllegalStateException("GeoPointTermsEnum empty or not initialized");
-    }
-    return currentRange.boundary;
-  }
-
-  protected BytesRef peek() {
-    return nextSubRangeBRB.get();
-  }
-
-  abstract protected boolean hasNext();
-
-  protected void nextRange() {
-    currentRange.fillBytesRef(currentCellBRB);
-    currentCell = currentCellBRB.get();
-  }
-
-  /**
-   * 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) > 0) {
-      if (hasNext() == false) {
-        return AcceptStatus.END;
-      }
-      // peek next sub-range, only seek if the current term is smaller than next lower bound
-      if (term.compareTo(peek()) < 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 boolean postFilter(final double lon, final double lat) {
-    return relationImpl.postFilter(lon, lat);
-  }
-
-  /**
-   * Internal class to represent a range along the space filling curve
-   */
-  abstract class BaseRange implements Comparable<BaseRange> {
-    protected short shift;
-    protected long start;
-    protected boolean boundary;
-
-    BaseRange(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.
-     */
-    abstract protected void fillBytesRef(BytesRefBuilder result);
-
-    @Override
-    public int compareTo(BaseRange other) {
-      final int result = Short.compare(this.shift, other.shift);
-      if (result == 0) {
-        return Long.compare(this.start, other.start);
-      }
-      return result;
-    }
-
-    protected void set(BaseRange other) {
-      this.start = other.start;
-      this.shift = other.shift;
-      this.boundary = other.boundary;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/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
deleted file mode 100644
index 8e8265c..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/search/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.
- */
-
-/**
- * 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/7d8f0127/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestGeoPointQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestGeoPointQuery.java b/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestGeoPointQuery.java
new file mode 100644
index 0000000..bdc4d86
--- /dev/null
+++ b/lucene/spatial/src/test/org/apache/lucene/spatial/geopoint/search/TestGeoPointQuery.java
@@ -0,0 +1,404 @@
+/*
+ * 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.geopoint.search;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.spatial.util.GeoEncodingUtils;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding;
+import org.apache.lucene.spatial.util.BaseGeoPointTestCase;
+import org.apache.lucene.spatial.util.GeoRect;
+import org.apache.lucene.spatial.util.GeoRelationUtils;
+import org.apache.lucene.spatial.util.GeoUtils;
+import org.apache.lucene.util.SloppyMath;
+import org.apache.lucene.util.TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import static org.apache.lucene.spatial.util.GeoDistanceUtils.DISTANCE_PCT_ERR;
+
+/**
+ * Unit testing for basic GeoPoint query logic
+ *
+ * @lucene.experimental
+ */
+
+public class TestGeoPointQuery extends BaseGeoPointTestCase {
+
+  private static Directory directory = null;
+  private static IndexReader reader = null;
+  private static IndexSearcher searcher = null;
+  private static TermEncoding termEncoding = null;
+  private static FieldType fieldType = null;
+
+  @Override
+  protected boolean forceSmall() {
+    return false;
+  }
+
+  @Override
+  protected void addPointToDoc(String field, Document doc, double lat, double lon) {
+    doc.add(new GeoPointField(field, lon, lat, fieldType));
+  }
+
+  @Override
+  protected Query newRectQuery(String field, GeoRect rect) {
+    return new GeoPointInBBoxQuery(field, termEncoding, rect.minLon, rect.minLat, rect.maxLon, rect.maxLat);
+  }
+
+  @Override
+  protected Query newDistanceQuery(String field, double centerLat, double centerLon, double radiusMeters) {
+    return new GeoPointDistanceQuery(field, termEncoding, centerLon, centerLat, radiusMeters);
+  }
+
+  @Override
+  protected Query newDistanceRangeQuery(String field, double centerLat, double centerLon, double minRadiusMeters, double radiusMeters) {
+    return new GeoPointDistanceRangeQuery(field, termEncoding, centerLon, centerLat, minRadiusMeters, radiusMeters);
+  }
+
+  @Override
+  protected Query newPolygonQuery(String field, double[] lats, double[] lons) {
+    return new GeoPointInPolygonQuery(field, termEncoding, lons, lats);
+  }
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    directory = newDirectory();
+    termEncoding = randomTermEncoding();
+    fieldType = randomFieldType();
+
+    RandomIndexWriter writer = new RandomIndexWriter(random(), directory,
+            newIndexWriterConfig(new MockAnalyzer(random()))
+                    .setMaxBufferedDocs(TestUtil.nextInt(random(), 100, 1000))
+                    .setMergePolicy(newLogMergePolicy()));
+
+    // this is a simple systematic test
+    GeoPointField[] pts = new GeoPointField[] {
+        new GeoPointField(FIELD_NAME, -96.774, 32.763420, fieldType),
+        new GeoPointField(FIELD_NAME, -96.7759895324707, 32.7559529921407, fieldType),
+        new GeoPointField(FIELD_NAME, -96.77701950073242, 32.77866942010977, fieldType),
+        new GeoPointField(FIELD_NAME, -96.7706036567688, 32.7756745755423, fieldType),
+        new GeoPointField(FIELD_NAME, -139.73458170890808, 27.703618681345585, fieldType),
+        new GeoPointField(FIELD_NAME, -96.4538113027811, 32.94823588839368, fieldType),
+        new GeoPointField(FIELD_NAME, -96.65084838867188, 33.06047141970814, fieldType),
+        new GeoPointField(FIELD_NAME, -96.7772, 32.778650, fieldType),
+        new GeoPointField(FIELD_NAME, -177.23537676036358, -88.56029371730983, fieldType),
+        new GeoPointField(FIELD_NAME, -26.779373834241003, 33.541429799076354, fieldType),
+        new GeoPointField(FIELD_NAME, -77.35379276106497, 26.774024500421728, fieldType),
+        new GeoPointField(FIELD_NAME, -14.796283808944777, -90.0, fieldType),
+        new GeoPointField(FIELD_NAME, -178.8538113027811, 32.94823588839368, fieldType),
+        new GeoPointField(FIELD_NAME, 178.8538113027811, 32.94823588839368, fieldType),
+        new GeoPointField(FIELD_NAME, -73.998776, 40.720611, fieldType),
+        new GeoPointField(FIELD_NAME, -179.5, -44.5, fieldType)};
+
+    for (GeoPointField p : pts) {
+        Document doc = new Document();
+        doc.add(p);
+        writer.addDocument(doc);
+    }
+
+    // add explicit multi-valued docs
+    for (int i=0; i<pts.length; i+=2) {
+      Document doc = new Document();
+      doc.add(pts[i]);
+      doc.add(pts[i+1]);
+      writer.addDocument(doc);
+    }
+
+    // index random string documents
+    for (int i=0; i<random().nextInt(10); ++i) {
+      Document doc = new Document();
+      doc.add(new StringField("string", Integer.toString(i), Field.Store.NO));
+      writer.addDocument(doc);
+    }
+
+    reader = writer.getReader();
+    searcher = newSearcher(reader);
+    writer.close();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    searcher = null;
+    reader.close();
+    reader = null;
+    directory.close();
+    directory = null;
+  }
+
+  private static TermEncoding randomTermEncoding() {
+    return random().nextBoolean() ? TermEncoding.NUMERIC : TermEncoding.PREFIX;
+  }
+
+  private static FieldType randomFieldType() {
+    if (termEncoding == TermEncoding.PREFIX) {
+      return GeoPointField.PREFIX_TYPE_NOT_STORED;
+    }
+    return GeoPointField.NUMERIC_TYPE_NOT_STORED;
+  }
+
+  private TopDocs bboxQuery(double minLon, double minLat, double maxLon, double maxLat, int limit) throws Exception {
+    GeoPointInBBoxQuery q = new GeoPointInBBoxQuery(FIELD_NAME, termEncoding, minLon, minLat, maxLon, maxLat);
+    return searcher.search(q, limit);
+  }
+
+  private TopDocs polygonQuery(double[] lon, double[] lat, int limit) throws Exception {
+    GeoPointInPolygonQuery q = new GeoPointInPolygonQuery(FIELD_NAME, termEncoding, lon, lat);
+    return searcher.search(q, limit);
+  }
+
+  private TopDocs geoDistanceQuery(double lon, double lat, double radius, int limit) throws Exception {
+    GeoPointDistanceQuery q = new GeoPointDistanceQuery(FIELD_NAME, termEncoding, lon, lat, radius);
+    return searcher.search(q, limit);
+  }
+
+  @Override
+  protected Boolean rectContainsPoint(GeoRect rect, double pointLat, double pointLon) {
+    if (GeoEncodingUtils.compare(pointLon, rect.minLon) == 0.0 ||
+        GeoEncodingUtils.compare(pointLon, rect.maxLon) == 0.0 ||
+        GeoEncodingUtils.compare(pointLat, rect.minLat) == 0.0 ||
+        GeoEncodingUtils.compare(pointLat, rect.maxLat) == 0.0) {
+      // Point is very close to rect boundary
+      return null;
+    }
+
+    if (rect.minLon < rect.maxLon) {
+      return GeoRelationUtils.pointInRectPrecise(pointLon, pointLat, rect.minLon, rect.minLat, rect.maxLon, rect.maxLat);
+    } else {
+      // Rect crosses dateline:
+      return GeoRelationUtils.pointInRectPrecise(pointLon, pointLat, -180.0, rect.minLat, rect.maxLon, rect.maxLat)
+          || GeoRelationUtils.pointInRectPrecise(pointLon, pointLat, rect.minLon, rect.minLat, 180.0, rect.maxLat);
+    }
+  }
+
+  @Override
+  protected Boolean polyRectContainsPoint(GeoRect rect, double pointLat, double pointLon) {
+    return rectContainsPoint(rect, pointLat, pointLon);
+  }
+
+  @Override
+  protected Boolean circleContainsPoint(double centerLat, double centerLon, double radiusMeters, double pointLat, double pointLon) {
+    if (radiusQueryCanBeWrong(centerLat, centerLon, pointLon, pointLat, radiusMeters)) {
+      return null;
+    } else {
+      return SloppyMath.haversin(centerLat, centerLon, pointLat, pointLon)*1000.0 <= radiusMeters;
+    }
+  }
+
+  @Override
+  protected Boolean distanceRangeContainsPoint(double centerLat, double centerLon, double minRadiusMeters, double radiusMeters, double pointLat, double pointLon) {
+    if (radiusQueryCanBeWrong(centerLat, centerLon, pointLon, pointLat, minRadiusMeters)
+        || radiusQueryCanBeWrong(centerLat, centerLon, pointLon, pointLat, radiusMeters)) {
+      return null;
+    } else {
+      final double d = SloppyMath.haversin(centerLat, centerLon, pointLat, pointLon)*1000.0;
+      return d >= minRadiusMeters && d <= radiusMeters;
+    }
+  }
+
+  private static boolean radiusQueryCanBeWrong(double centerLat, double centerLon, double ptLon, double ptLat,
+                                               final double radius) {
+    final long hashedCntr = GeoEncodingUtils.mortonHash(centerLon, centerLat);
+    centerLon = GeoEncodingUtils.mortonUnhashLon(hashedCntr);
+    centerLat = GeoEncodingUtils.mortonUnhashLat(hashedCntr);
+    final long hashedPt = GeoEncodingUtils.mortonHash(ptLon, ptLat);
+    ptLon = GeoEncodingUtils.mortonUnhashLon(hashedPt);
+    ptLat = GeoEncodingUtils.mortonUnhashLat(hashedPt);
+
+    double ptDistance = SloppyMath.haversin(centerLat, centerLon, ptLat, ptLon)*1000.0;
+    double delta = StrictMath.abs(ptDistance - radius);
+
+    // if its within the distance error then it can be wrong
+    return delta < (ptDistance*DISTANCE_PCT_ERR);
+  }
+
+  public void testRectCrossesCircle() throws Exception {
+    assertTrue(GeoRelationUtils.rectCrossesCircle(-180, -90, 180, 0.0, 0.667, 0.0, 88000.0));
+  }
+
+  private TopDocs geoDistanceRangeQuery(double lon, double lat, double minRadius, double maxRadius, int limit)
+      throws Exception {
+    GeoPointDistanceRangeQuery q = new GeoPointDistanceRangeQuery(FIELD_NAME, termEncoding, lon, lat, minRadius, maxRadius);
+    return searcher.search(q, limit);
+  }
+
+  public void testBBoxQuery() throws Exception {
+    TopDocs td = bboxQuery(-96.7772, 32.778650, -96.77690000, 32.778950, 5);
+    assertEquals("GeoBoundingBoxQuery failed", 4, td.totalHits);
+  }
+
+  public void testPolyQuery() throws Exception {
+    TopDocs td = polygonQuery(new double[]{-96.7682647, -96.8280029, -96.6288757, -96.4929199,
+            -96.6041564, -96.7449188, -96.76826477, -96.7682647},
+        new double[]{33.073130, 32.9942669, 32.938386, 33.0374494,
+            33.1369762, 33.1162747, 33.073130, 33.073130}, 5);
+    assertEquals("GeoPolygonQuery failed", 2, td.totalHits);
+  }
+
+  public void testPacManPolyQuery() throws Exception {
+    // pacman
+    double[] px = {0, 10, 10, 0, -8, -10, -8, 0, 10, 10, 0};
+    double[] py = {0, 5, 9, 10, 9, 0, -9, -10, -9, -5, 0};
+
+    // shape bbox
+    double xMinA = -10;
+    double xMaxA = 10;
+    double yMinA = -10;
+    double yMaxA = 10;
+
+    // candidate crosses cell
+    double xMin = 2;//-5;
+    double xMax = 11;//0.000001;
+    double yMin = -1;//0;
+    double yMax = 1;//5;
+
+    // test cell crossing poly
+    assertTrue(GeoRelationUtils.rectCrossesPolyApprox(xMin, yMin, xMax, yMax, px, py, xMinA, yMinA, xMaxA, yMaxA));
+    assertFalse(GeoRelationUtils.rectCrossesPolyApprox(-5, 0,  0.000001, 5, px, py, xMin, yMin, xMax, yMax));
+    assertTrue(GeoRelationUtils.rectWithinPolyApprox(-5, 0, -2, 5, px, py, xMin, yMin, xMax, yMax));
+  }
+
+  public void testBBoxCrossDateline() throws Exception {
+    TopDocs td = bboxQuery(179.0, -45.0, -179.0, -44.0, 20);
+    assertEquals("BBoxCrossDateline query failed", 2, td.totalHits);
+  }
+
+  public void testWholeMap() throws Exception {
+    TopDocs td = bboxQuery(GeoUtils.MIN_LON_INCL, GeoUtils.MIN_LAT_INCL, GeoUtils.MAX_LON_INCL, GeoUtils.MAX_LAT_INCL, 20);
+    assertEquals("testWholeMap failed", 24, td.totalHits);
+    td = polygonQuery(new double[] {GeoUtils.MIN_LON_INCL, GeoUtils.MIN_LON_INCL, GeoUtils.MAX_LON_INCL, GeoUtils.MAX_LON_INCL, GeoUtils.MIN_LON_INCL},
+        new double[] {GeoUtils.MIN_LAT_INCL, GeoUtils.MAX_LAT_INCL, GeoUtils.MAX_LAT_INCL, GeoUtils.MIN_LAT_INCL, GeoUtils.MIN_LAT_INCL}, 20);
+    assertEquals("testWholeMap failed", 24, td.totalHits);
+  }
+
+  public void smallTest() throws Exception {
+    TopDocs td = geoDistanceQuery(-73.998776, 40.720611, 1, 20);
+    assertEquals("smallTest failed", 2, td.totalHits);
+  }
+
+  public void testInvalidBBox() throws Exception {
+    try {
+      bboxQuery(179.0, -92.0, 181.0, -91.0, 20);
+    } catch(Exception e) {
+      return;
+    }
+    throw new Exception("GeoBoundingBox should not accept invalid lat/lon");
+  }
+
+  public void testGeoDistanceQuery() throws Exception {
+    TopDocs td = geoDistanceQuery(-96.4538113027811, 32.94823588839368, 6000, 20);
+    assertEquals("GeoDistanceQuery failed", 2, td.totalHits);
+  }
+
+  /** see https://issues.apache.org/jira/browse/LUCENE-6905 */
+  public void testNonEmptyTermsEnum() throws Exception {
+    TopDocs td = geoDistanceQuery(-177.23537676036358, -88.56029371730983, 7757.999232959935, 20);
+    assertEquals("GeoDistanceQuery failed", 2, td.totalHits);
+  }
+
+  public void testMultiValuedQuery() throws Exception {
+    TopDocs td = bboxQuery(-96.4538113027811, 32.7559529921407, -96.7706036567688, 32.7756745755423, 20);
+    // 3 single valued docs + 2 multi-valued docs
+    assertEquals("testMultiValuedQuery failed", 5, td.totalHits);
+  }
+
+  public void testTooBigRadius() throws Exception {
+    try {
+      geoDistanceQuery(0.0, 85.0, 4000000, 20);
+    } catch (IllegalArgumentException e) {
+      e.getMessage().contains("exceeds maxRadius");
+    }
+  }
+
+  /**
+   * Explicitly large
+   */
+  public void testGeoDistanceQueryHuge() throws Exception {
+    TopDocs td = geoDistanceQuery(-96.4538113027811, 32.94823588839368, 6000000, 20);
+    assertEquals("GeoDistanceQuery failed", 16, td.totalHits);
+  }
+
+  public void testGeoDistanceQueryCrossDateline() throws Exception {
+    TopDocs td = geoDistanceQuery(-179.9538113027811, 32.94823588839368, 120000, 20);
+    assertEquals("GeoDistanceQuery failed", 3, td.totalHits);
+  }
+
+  public void testInvalidGeoDistanceQuery() throws Exception {
+    try {
+      geoDistanceQuery(181.0, 92.0, 120000, 20);
+    } catch (Exception e) {
+      return;
+    }
+    throw new Exception("GeoDistanceQuery should not accept invalid lat/lon as origin");
+  }
+
+  public void testMaxDistanceRangeQuery() throws Exception {
+    TopDocs td = geoDistanceRangeQuery(0.0, 0.0, 10, 20000000, 20);
+    assertEquals("GeoDistanceRangeQuery failed", 24, td.totalHits);
+  }
+
+  public void testMortonEncoding() throws Exception {
+    long hash = GeoEncodingUtils.mortonHash(180, 90);
+    assertEquals(180.0, GeoEncodingUtils.mortonUnhashLon(hash), 0);
+    assertEquals(90.0, GeoEncodingUtils.mortonUnhashLat(hash), 0);
+  }
+
+  public void testEncodeDecode() throws Exception {
+    int iters = atLeast(10000);
+    boolean small = random().nextBoolean();
+    for(int iter=0;iter<iters;iter++) {
+      double lat = randomLat(small);
+      double lon = randomLon(small);
+
+      long enc = GeoEncodingUtils.mortonHash(lon, lat);
+      double latEnc = GeoEncodingUtils.mortonUnhashLat(enc);
+      double lonEnc = GeoEncodingUtils.mortonUnhashLon(enc);
+
+      assertEquals("lat=" + lat + " latEnc=" + latEnc + " diff=" + (lat - latEnc), lat, latEnc, GeoEncodingUtils.TOLERANCE);
+      assertEquals("lon=" + lon + " lonEnc=" + lonEnc + " diff=" + (lon - lonEnc), lon, lonEnc, GeoEncodingUtils.TOLERANCE);
+    }
+  }
+
+  public void testScaleUnscaleIsStable() throws Exception {
+    int iters = atLeast(1000);
+    boolean small = random().nextBoolean();
+    for(int iter=0;iter<iters;iter++) {
+      double lat = randomLat(small);
+      double lon = randomLon(small);
+
+      long enc = GeoEncodingUtils.mortonHash(lon, lat);
+      double latEnc = GeoEncodingUtils.mortonUnhashLat(enc);
+      double lonEnc = GeoEncodingUtils.mortonUnhashLon(enc);
+
+      long enc2 = GeoEncodingUtils.mortonHash(lon, lat);
+      double latEnc2 = GeoEncodingUtils.mortonUnhashLat(enc2);
+      double lonEnc2 = GeoEncodingUtils.mortonUnhashLon(enc2);
+      assertEquals(latEnc, latEnc2, 0.0);
+      assertEquals(lonEnc, lonEnc2, 0.0);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/test/org/apache/lucene/spatial/search/TestGeoPointQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/search/TestGeoPointQuery.java b/lucene/spatial/src/test/org/apache/lucene/spatial/search/TestGeoPointQuery.java
deleted file mode 100644
index 0a353bb..0000000
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/search/TestGeoPointQuery.java
+++ /dev/null
@@ -1,404 +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.search;
-
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
-import org.apache.lucene.spatial.document.GeoPointField;
-import org.apache.lucene.spatial.document.GeoPointField.TermEncoding;
-import org.apache.lucene.spatial.util.BaseGeoPointTestCase;
-import org.apache.lucene.spatial.util.GeoRect;
-import org.apache.lucene.spatial.util.GeoRelationUtils;
-import org.apache.lucene.spatial.util.GeoUtils;
-import org.apache.lucene.util.SloppyMath;
-import org.apache.lucene.util.TestUtil;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-
-import static org.apache.lucene.spatial.util.GeoDistanceUtils.DISTANCE_PCT_ERR;
-
-/**
- * Unit testing for basic GeoPoint query logic
- *
- * @lucene.experimental
- */
-
-public class TestGeoPointQuery extends BaseGeoPointTestCase {
-
-  private static Directory directory = null;
-  private static IndexReader reader = null;
-  private static IndexSearcher searcher = null;
-  private static TermEncoding termEncoding = null;
-  private static FieldType fieldType = null;
-
-  @Override
-  protected boolean forceSmall() {
-    return false;
-  }
-
-  @Override
-  protected void addPointToDoc(String field, Document doc, double lat, double lon) {
-    doc.add(new GeoPointField(field, lon, lat, fieldType));
-  }
-
-  @Override
-  protected Query newRectQuery(String field, GeoRect rect) {
-    return new GeoPointInBBoxQuery(field, termEncoding, rect.minLon, rect.minLat, rect.maxLon, rect.maxLat);
-  }
-
-  @Override
-  protected Query newDistanceQuery(String field, double centerLat, double centerLon, double radiusMeters) {
-    return new GeoPointDistanceQuery(field, termEncoding, centerLon, centerLat, radiusMeters);
-  }
-
-  @Override
-  protected Query newDistanceRangeQuery(String field, double centerLat, double centerLon, double minRadiusMeters, double radiusMeters) {
-    return new GeoPointDistanceRangeQuery(field, termEncoding, centerLon, centerLat, minRadiusMeters, radiusMeters);
-  }
-
-  @Override
-  protected Query newPolygonQuery(String field, double[] lats, double[] lons) {
-    return new GeoPointInPolygonQuery(field, termEncoding, lons, lats);
-  }
-
-  @BeforeClass
-  public static void beforeClass() throws Exception {
-    directory = newDirectory();
-    termEncoding = randomTermEncoding();
-    fieldType = randomFieldType();
-
-    RandomIndexWriter writer = new RandomIndexWriter(random(), directory,
-            newIndexWriterConfig(new MockAnalyzer(random()))
-                    .setMaxBufferedDocs(TestUtil.nextInt(random(), 100, 1000))
-                    .setMergePolicy(newLogMergePolicy()));
-
-    // this is a simple systematic test
-    GeoPointField[] pts = new GeoPointField[] {
-        new GeoPointField(FIELD_NAME, -96.774, 32.763420, fieldType),
-        new GeoPointField(FIELD_NAME, -96.7759895324707, 32.7559529921407, fieldType),
-        new GeoPointField(FIELD_NAME, -96.77701950073242, 32.77866942010977, fieldType),
-        new GeoPointField(FIELD_NAME, -96.7706036567688, 32.7756745755423, fieldType),
-        new GeoPointField(FIELD_NAME, -139.73458170890808, 27.703618681345585, fieldType),
-        new GeoPointField(FIELD_NAME, -96.4538113027811, 32.94823588839368, fieldType),
-        new GeoPointField(FIELD_NAME, -96.65084838867188, 33.06047141970814, fieldType),
-        new GeoPointField(FIELD_NAME, -96.7772, 32.778650, fieldType),
-        new GeoPointField(FIELD_NAME, -177.23537676036358, -88.56029371730983, fieldType),
-        new GeoPointField(FIELD_NAME, -26.779373834241003, 33.541429799076354, fieldType),
-        new GeoPointField(FIELD_NAME, -77.35379276106497, 26.774024500421728, fieldType),
-        new GeoPointField(FIELD_NAME, -14.796283808944777, -90.0, fieldType),
-        new GeoPointField(FIELD_NAME, -178.8538113027811, 32.94823588839368, fieldType),
-        new GeoPointField(FIELD_NAME, 178.8538113027811, 32.94823588839368, fieldType),
-        new GeoPointField(FIELD_NAME, -73.998776, 40.720611, fieldType),
-        new GeoPointField(FIELD_NAME, -179.5, -44.5, fieldType)};
-
-    for (GeoPointField p : pts) {
-        Document doc = new Document();
-        doc.add(p);
-        writer.addDocument(doc);
-    }
-
-    // add explicit multi-valued docs
-    for (int i=0; i<pts.length; i+=2) {
-      Document doc = new Document();
-      doc.add(pts[i]);
-      doc.add(pts[i+1]);
-      writer.addDocument(doc);
-    }
-
-    // index random string documents
-    for (int i=0; i<random().nextInt(10); ++i) {
-      Document doc = new Document();
-      doc.add(new StringField("string", Integer.toString(i), Field.Store.NO));
-      writer.addDocument(doc);
-    }
-
-    reader = writer.getReader();
-    searcher = newSearcher(reader);
-    writer.close();
-  }
-
-  @AfterClass
-  public static void afterClass() throws Exception {
-    searcher = null;
-    reader.close();
-    reader = null;
-    directory.close();
-    directory = null;
-  }
-
-  private static TermEncoding randomTermEncoding() {
-    return random().nextBoolean() ? TermEncoding.NUMERIC : TermEncoding.PREFIX;
-  }
-
-  private static FieldType randomFieldType() {
-    if (termEncoding == TermEncoding.PREFIX) {
-      return GeoPointField.PREFIX_TYPE_NOT_STORED;
-    }
-    return GeoPointField.NUMERIC_TYPE_NOT_STORED;
-  }
-
-  private TopDocs bboxQuery(double minLon, double minLat, double maxLon, double maxLat, int limit) throws Exception {
-    GeoPointInBBoxQuery q = new GeoPointInBBoxQuery(FIELD_NAME, termEncoding, minLon, minLat, maxLon, maxLat);
-    return searcher.search(q, limit);
-  }
-
-  private TopDocs polygonQuery(double[] lon, double[] lat, int limit) throws Exception {
-    GeoPointInPolygonQuery q = new GeoPointInPolygonQuery(FIELD_NAME, termEncoding, lon, lat);
-    return searcher.search(q, limit);
-  }
-
-  private TopDocs geoDistanceQuery(double lon, double lat, double radius, int limit) throws Exception {
-    GeoPointDistanceQuery q = new GeoPointDistanceQuery(FIELD_NAME, termEncoding, lon, lat, radius);
-    return searcher.search(q, limit);
-  }
-
-  @Override
-  protected Boolean rectContainsPoint(GeoRect rect, double pointLat, double pointLon) {
-    if (GeoEncodingUtils.compare(pointLon, rect.minLon) == 0.0 ||
-        GeoEncodingUtils.compare(pointLon, rect.maxLon) == 0.0 ||
-        GeoEncodingUtils.compare(pointLat, rect.minLat) == 0.0 ||
-        GeoEncodingUtils.compare(pointLat, rect.maxLat) == 0.0) {
-      // Point is very close to rect boundary
-      return null;
-    }
-
-    if (rect.minLon < rect.maxLon) {
-      return GeoRelationUtils.pointInRectPrecise(pointLon, pointLat, rect.minLon, rect.minLat, rect.maxLon, rect.maxLat);
-    } else {
-      // Rect crosses dateline:
-      return GeoRelationUtils.pointInRectPrecise(pointLon, pointLat, -180.0, rect.minLat, rect.maxLon, rect.maxLat)
-          || GeoRelationUtils.pointInRectPrecise(pointLon, pointLat, rect.minLon, rect.minLat, 180.0, rect.maxLat);
-    }
-  }
-
-  @Override
-  protected Boolean polyRectContainsPoint(GeoRect rect, double pointLat, double pointLon) {
-    return rectContainsPoint(rect, pointLat, pointLon);
-  }
-
-  @Override
-  protected Boolean circleContainsPoint(double centerLat, double centerLon, double radiusMeters, double pointLat, double pointLon) {
-    if (radiusQueryCanBeWrong(centerLat, centerLon, pointLon, pointLat, radiusMeters)) {
-      return null;
-    } else {
-      return SloppyMath.haversin(centerLat, centerLon, pointLat, pointLon)*1000.0 <= radiusMeters;
-    }
-  }
-
-  @Override
-  protected Boolean distanceRangeContainsPoint(double centerLat, double centerLon, double minRadiusMeters, double radiusMeters, double pointLat, double pointLon) {
-    if (radiusQueryCanBeWrong(centerLat, centerLon, pointLon, pointLat, minRadiusMeters)
-        || radiusQueryCanBeWrong(centerLat, centerLon, pointLon, pointLat, radiusMeters)) {
-      return null;
-    } else {
-      final double d = SloppyMath.haversin(centerLat, centerLon, pointLat, pointLon)*1000.0;
-      return d >= minRadiusMeters && d <= radiusMeters;
-    }
-  }
-
-  private static boolean radiusQueryCanBeWrong(double centerLat, double centerLon, double ptLon, double ptLat,
-                                               final double radius) {
-    final long hashedCntr = GeoEncodingUtils.mortonHash(centerLon, centerLat);
-    centerLon = GeoEncodingUtils.mortonUnhashLon(hashedCntr);
-    centerLat = GeoEncodingUtils.mortonUnhashLat(hashedCntr);
-    final long hashedPt = GeoEncodingUtils.mortonHash(ptLon, ptLat);
-    ptLon = GeoEncodingUtils.mortonUnhashLon(hashedPt);
-    ptLat = GeoEncodingUtils.mortonUnhashLat(hashedPt);
-
-    double ptDistance = SloppyMath.haversin(centerLat, centerLon, ptLat, ptLon)*1000.0;
-    double delta = StrictMath.abs(ptDistance - radius);
-
-    // if its within the distance error then it can be wrong
-    return delta < (ptDistance*DISTANCE_PCT_ERR);
-  }
-
-  public void testRectCrossesCircle() throws Exception {
-    assertTrue(GeoRelationUtils.rectCrossesCircle(-180, -90, 180, 0.0, 0.667, 0.0, 88000.0));
-  }
-
-  private TopDocs geoDistanceRangeQuery(double lon, double lat, double minRadius, double maxRadius, int limit)
-      throws Exception {
-    GeoPointDistanceRangeQuery q = new GeoPointDistanceRangeQuery(FIELD_NAME, termEncoding, lon, lat, minRadius, maxRadius);
-    return searcher.search(q, limit);
-  }
-
-  public void testBBoxQuery() throws Exception {
-    TopDocs td = bboxQuery(-96.7772, 32.778650, -96.77690000, 32.778950, 5);
-    assertEquals("GeoBoundingBoxQuery failed", 4, td.totalHits);
-  }
-
-  public void testPolyQuery() throws Exception {
-    TopDocs td = polygonQuery(new double[]{-96.7682647, -96.8280029, -96.6288757, -96.4929199,
-            -96.6041564, -96.7449188, -96.76826477, -96.7682647},
-        new double[]{33.073130, 32.9942669, 32.938386, 33.0374494,
-            33.1369762, 33.1162747, 33.073130, 33.073130}, 5);
-    assertEquals("GeoPolygonQuery failed", 2, td.totalHits);
-  }
-
-  public void testPacManPolyQuery() throws Exception {
-    // pacman
-    double[] px = {0, 10, 10, 0, -8, -10, -8, 0, 10, 10, 0};
-    double[] py = {0, 5, 9, 10, 9, 0, -9, -10, -9, -5, 0};
-
-    // shape bbox
-    double xMinA = -10;
-    double xMaxA = 10;
-    double yMinA = -10;
-    double yMaxA = 10;
-
-    // candidate crosses cell
-    double xMin = 2;//-5;
-    double xMax = 11;//0.000001;
-    double yMin = -1;//0;
-    double yMax = 1;//5;
-
-    // test cell crossing poly
-    assertTrue(GeoRelationUtils.rectCrossesPolyApprox(xMin, yMin, xMax, yMax, px, py, xMinA, yMinA, xMaxA, yMaxA));
-    assertFalse(GeoRelationUtils.rectCrossesPolyApprox(-5, 0,  0.000001, 5, px, py, xMin, yMin, xMax, yMax));
-    assertTrue(GeoRelationUtils.rectWithinPolyApprox(-5, 0, -2, 5, px, py, xMin, yMin, xMax, yMax));
-  }
-
-  public void testBBoxCrossDateline() throws Exception {
-    TopDocs td = bboxQuery(179.0, -45.0, -179.0, -44.0, 20);
-    assertEquals("BBoxCrossDateline query failed", 2, td.totalHits);
-  }
-
-  public void testWholeMap() throws Exception {
-    TopDocs td = bboxQuery(GeoUtils.MIN_LON_INCL, GeoUtils.MIN_LAT_INCL, GeoUtils.MAX_LON_INCL, GeoUtils.MAX_LAT_INCL, 20);
-    assertEquals("testWholeMap failed", 24, td.totalHits);
-    td = polygonQuery(new double[] {GeoUtils.MIN_LON_INCL, GeoUtils.MIN_LON_INCL, GeoUtils.MAX_LON_INCL, GeoUtils.MAX_LON_INCL, GeoUtils.MIN_LON_INCL},
-        new double[] {GeoUtils.MIN_LAT_INCL, GeoUtils.MAX_LAT_INCL, GeoUtils.MAX_LAT_INCL, GeoUtils.MIN_LAT_INCL, GeoUtils.MIN_LAT_INCL}, 20);
-    assertEquals("testWholeMap failed", 24, td.totalHits);
-  }
-
-  public void smallTest() throws Exception {
-    TopDocs td = geoDistanceQuery(-73.998776, 40.720611, 1, 20);
-    assertEquals("smallTest failed", 2, td.totalHits);
-  }
-
-  public void testInvalidBBox() throws Exception {
-    try {
-      bboxQuery(179.0, -92.0, 181.0, -91.0, 20);
-    } catch(Exception e) {
-      return;
-    }
-    throw new Exception("GeoBoundingBox should not accept invalid lat/lon");
-  }
-
-  public void testGeoDistanceQuery() throws Exception {
-    TopDocs td = geoDistanceQuery(-96.4538113027811, 32.94823588839368, 6000, 20);
-    assertEquals("GeoDistanceQuery failed", 2, td.totalHits);
-  }
-
-  /** see https://issues.apache.org/jira/browse/LUCENE-6905 */
-  public void testNonEmptyTermsEnum() throws Exception {
-    TopDocs td = geoDistanceQuery(-177.23537676036358, -88.56029371730983, 7757.999232959935, 20);
-    assertEquals("GeoDistanceQuery failed", 2, td.totalHits);
-  }
-
-  public void testMultiValuedQuery() throws Exception {
-    TopDocs td = bboxQuery(-96.4538113027811, 32.7559529921407, -96.7706036567688, 32.7756745755423, 20);
-    // 3 single valued docs + 2 multi-valued docs
-    assertEquals("testMultiValuedQuery failed", 5, td.totalHits);
-  }
-
-  public void testTooBigRadius() throws Exception {
-    try {
-      geoDistanceQuery(0.0, 85.0, 4000000, 20);
-    } catch (IllegalArgumentException e) {
-      e.getMessage().contains("exceeds maxRadius");
-    }
-  }
-
-  /**
-   * Explicitly large
-   */
-  public void testGeoDistanceQueryHuge() throws Exception {
-    TopDocs td = geoDistanceQuery(-96.4538113027811, 32.94823588839368, 6000000, 20);
-    assertEquals("GeoDistanceQuery failed", 16, td.totalHits);
-  }
-
-  public void testGeoDistanceQueryCrossDateline() throws Exception {
-    TopDocs td = geoDistanceQuery(-179.9538113027811, 32.94823588839368, 120000, 20);
-    assertEquals("GeoDistanceQuery failed", 3, td.totalHits);
-  }
-
-  public void testInvalidGeoDistanceQuery() throws Exception {
-    try {
-      geoDistanceQuery(181.0, 92.0, 120000, 20);
-    } catch (Exception e) {
-      return;
-    }
-    throw new Exception("GeoDistanceQuery should not accept invalid lat/lon as origin");
-  }
-
-  public void testMaxDistanceRangeQuery() throws Exception {
-    TopDocs td = geoDistanceRangeQuery(0.0, 0.0, 10, 20000000, 20);
-    assertEquals("GeoDistanceRangeQuery failed", 24, td.totalHits);
-  }
-
-  public void testMortonEncoding() throws Exception {
-    long hash = GeoEncodingUtils.mortonHash(180, 90);
-    assertEquals(180.0, GeoEncodingUtils.mortonUnhashLon(hash), 0);
-    assertEquals(90.0, GeoEncodingUtils.mortonUnhashLat(hash), 0);
-  }
-
-  public void testEncodeDecode() throws Exception {
-    int iters = atLeast(10000);
-    boolean small = random().nextBoolean();
-    for(int iter=0;iter<iters;iter++) {
-      double lat = randomLat(small);
-      double lon = randomLon(small);
-
-      long enc = GeoEncodingUtils.mortonHash(lon, lat);
-      double latEnc = GeoEncodingUtils.mortonUnhashLat(enc);
-      double lonEnc = GeoEncodingUtils.mortonUnhashLon(enc);
-
-      assertEquals("lat=" + lat + " latEnc=" + latEnc + " diff=" + (lat - latEnc), lat, latEnc, GeoEncodingUtils.TOLERANCE);
-      assertEquals("lon=" + lon + " lonEnc=" + lonEnc + " diff=" + (lon - lonEnc), lon, lonEnc, GeoEncodingUtils.TOLERANCE);
-    }
-  }
-
-  public void testScaleUnscaleIsStable() throws Exception {
-    int iters = atLeast(1000);
-    boolean small = random().nextBoolean();
-    for(int iter=0;iter<iters;iter++) {
-      double lat = randomLat(small);
-      double lon = randomLon(small);
-
-      long enc = GeoEncodingUtils.mortonHash(lon, lat);
-      double latEnc = GeoEncodingUtils.mortonUnhashLat(enc);
-      double lonEnc = GeoEncodingUtils.mortonUnhashLon(enc);
-
-      long enc2 = GeoEncodingUtils.mortonHash(lon, lat);
-      double latEnc2 = GeoEncodingUtils.mortonUnhashLat(enc2);
-      double lonEnc2 = GeoEncodingUtils.mortonUnhashLon(enc2);
-      assertEquals(latEnc, latEnc2, 0.0);
-      assertEquals(lonEnc, lonEnc2, 0.0);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java b/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
index aa1da81..f8f00e7 100644
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
+++ b/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
@@ -44,7 +44,6 @@ import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SimpleCollector;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;


[3/3] lucene-solr git commit: LUCENE-6997: refactor GeoPointField and query classes from lucene.spatial to lucene.spatial.geopoint package

Posted by nk...@apache.org.
LUCENE-6997: refactor GeoPointField and query classes from lucene.spatial to lucene.spatial.geopoint package


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

Branch: refs/heads/master
Commit: 7d8f0127b7e8514b19c2898dbaac2d1419964b1b
Parents: b889109
Author: nknize <nk...@apache.org>
Authored: Tue Feb 9 21:51:30 2016 -0600
Committer: nknize <nk...@apache.org>
Committed: Tue Feb 9 21:51:48 2016 -0600

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   2 +-
 .../lucene/spatial/document/GeoPointField.java  | 250 ------------
 .../spatial/document/GeoPointTokenStream.java   | 233 -----------
 .../lucene/spatial/document/package-info.java   |  21 -
 .../geopoint/document/GeoPointField.java        | 250 ++++++++++++
 .../geopoint/document/GeoPointTokenStream.java  | 233 +++++++++++
 .../spatial/geopoint/document/package-info.java |  21 +
 .../spatial/geopoint/search/GeoBoundingBox.java |  53 +++
 .../geopoint/search/GeoPointDistanceQuery.java  | 189 +++++++++
 .../search/GeoPointDistanceQueryImpl.java       | 107 +++++
 .../search/GeoPointDistanceRangeQuery.java      | 116 ++++++
 .../geopoint/search/GeoPointInBBoxQuery.java    | 173 ++++++++
 .../search/GeoPointInBBoxQueryImpl.java         | 156 +++++++
 .../geopoint/search/GeoPointInPolygonQuery.java | 153 +++++++
 .../search/GeoPointInPolygonQueryImpl.java      | 103 +++++
 .../geopoint/search/GeoPointMultiTermQuery.java | 166 ++++++++
 .../search/GeoPointNumericTermsEnum.java        | 161 ++++++++
 .../search/GeoPointPrefixTermsEnum.java         | 237 +++++++++++
 .../GeoPointTermQueryConstantScoreWrapper.java  | 153 +++++++
 .../geopoint/search/GeoPointTermsEnum.java      | 140 +++++++
 .../spatial/geopoint/search/package-info.java   |  21 +
 .../lucene/spatial/search/GeoBoundingBox.java   |  53 ---
 .../spatial/search/GeoPointDistanceQuery.java   | 189 ---------
 .../search/GeoPointDistanceQueryImpl.java       | 107 -----
 .../search/GeoPointDistanceRangeQuery.java      | 116 ------
 .../spatial/search/GeoPointInBBoxQuery.java     | 173 --------
 .../spatial/search/GeoPointInBBoxQueryImpl.java | 156 -------
 .../spatial/search/GeoPointInPolygonQuery.java  | 153 -------
 .../search/GeoPointInPolygonQueryImpl.java      | 103 -----
 .../spatial/search/GeoPointMultiTermQuery.java  | 166 --------
 .../search/GeoPointNumericTermsEnum.java        | 161 --------
 .../spatial/search/GeoPointPrefixTermsEnum.java | 237 -----------
 .../GeoPointTermQueryConstantScoreWrapper.java  | 153 -------
 .../spatial/search/GeoPointTermsEnum.java       | 140 -------
 .../lucene/spatial/search/package-info.java     |  21 -
 .../geopoint/search/TestGeoPointQuery.java      | 404 +++++++++++++++++++
 .../spatial/search/TestGeoPointQuery.java       | 404 -------------------
 .../spatial/util/BaseGeoPointTestCase.java      |   1 -
 38 files changed, 2837 insertions(+), 2838 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index e86b0ae..54124df 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -149,7 +149,7 @@ New Features
 API Changes
 
 * LUCENE-6997: refactor sandboxed GeoPointField and query classes to lucene-spatial 
-  module (Nick Knize)
+  module under new lucene.spatial.geopoint package (Nick Knize)
 
 * LUCENE-6908: GeoUtils static relational methods have been refactored to new 
   GeoRelationUtils and now correctly handle large irregular rectangles, and

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/document/GeoPointField.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/document/GeoPointField.java b/lucene/spatial/src/java/org/apache/lucene/spatial/document/GeoPointField.java
deleted file mode 100644
index cafa67c..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/document/GeoPointField.java
+++ /dev/null
@@ -1,250 +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.document;
-
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.index.DocValuesType;
-import org.apache.lucene.index.IndexOptions;
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
-
-/**
- * <p>
- * Field that indexes <code>latitude</code> <code>longitude</code> decimal-degree values
- * for efficient encoding, sorting, and querying. This Geo capability is intended
- * to provide a basic and efficient out of the box field type for indexing and
- * querying 2 dimensional points in WGS-84 decimal degrees. An example usage is as follows:
- *
- * <pre class="prettyprint">
- *  document.add(new GeoPointField(name, -96.33, 32.66, Field.Store.NO));
- * </pre>
- *
- * <p>To perform simple geospatial queries against a <code>GeoPointField</code>,
- * see {@link org.apache.lucene.spatial.search.GeoPointInBBoxQuery}, {@link org.apache.lucene.spatial.search.GeoPointInPolygonQuery},
- * or {@link org.apache.lucene.spatial.search.GeoPointDistanceQuery}
- *
- * NOTE: This indexes only high precision encoded terms which may result in visiting a high number
- * of terms for large queries. See LUCENE-6481 for a future improvement.
- *
- * @lucene.experimental
- */
-public final class GeoPointField extends Field {
-  /** encoding step value for GeoPoint prefix terms */
-  public static final int PRECISION_STEP = 9;
-
-  /**
-   * <b>Expert:</b> Optional flag to select term encoding for GeoPointField types
-   */
-  public enum TermEncoding {
-    /**
-     * encodes prefix terms only resulting in a small index and faster queries - use with
-     * {@code GeoPointTokenStream}
-     */
-    PREFIX,
-    /**
-     * @deprecated encodes prefix and full resolution terms - use with
-     * {@link org.apache.lucene.analysis.LegacyNumericTokenStream}
-     */
-    @Deprecated
-    NUMERIC
-  }
-
-  /**
-   * @deprecated Type for a GeoPointField that is not stored:
-   * normalization factors, frequencies, and positions are omitted.
-   */
-  @Deprecated
-  public static final FieldType NUMERIC_TYPE_NOT_STORED = new FieldType();
-  static {
-    NUMERIC_TYPE_NOT_STORED.setTokenized(false);
-    NUMERIC_TYPE_NOT_STORED.setOmitNorms(true);
-    NUMERIC_TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS);
-    NUMERIC_TYPE_NOT_STORED.setDocValuesType(DocValuesType.SORTED_NUMERIC);
-    NUMERIC_TYPE_NOT_STORED.setNumericType(FieldType.LegacyNumericType.LONG);
-    NUMERIC_TYPE_NOT_STORED.setNumericPrecisionStep(PRECISION_STEP);
-    NUMERIC_TYPE_NOT_STORED.freeze();
-  }
-
-  /**
-   * @deprecated Type for a stored GeoPointField:
-   * normalization factors, frequencies, and positions are omitted.
-   */
-  @Deprecated
-  public static final FieldType NUMERIC_TYPE_STORED = new FieldType();
-  static {
-    NUMERIC_TYPE_STORED.setTokenized(false);
-    NUMERIC_TYPE_STORED.setOmitNorms(true);
-    NUMERIC_TYPE_STORED.setIndexOptions(IndexOptions.DOCS);
-    NUMERIC_TYPE_STORED.setDocValuesType(DocValuesType.SORTED_NUMERIC);
-    NUMERIC_TYPE_STORED.setNumericType(FieldType.LegacyNumericType.LONG);
-    NUMERIC_TYPE_STORED.setNumericPrecisionStep(PRECISION_STEP);
-    NUMERIC_TYPE_STORED.setStored(true);
-    NUMERIC_TYPE_STORED.freeze();
-  }
-
-  /**
-   * Type for a GeoPointField that is not stored:
-   * normalization factors, frequencies, and positions are omitted.
-   */
-  public static final FieldType PREFIX_TYPE_NOT_STORED = new FieldType();
-  static {
-    PREFIX_TYPE_NOT_STORED.setTokenized(false);
-    PREFIX_TYPE_NOT_STORED.setOmitNorms(true);
-    PREFIX_TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS);
-    PREFIX_TYPE_NOT_STORED.setDocValuesType(DocValuesType.SORTED_NUMERIC);
-    PREFIX_TYPE_NOT_STORED.freeze();
-  }
-
-  /**
-   * Type for a stored GeoPointField:
-   * normalization factors, frequencies, and positions are omitted.
-   */
-  public static final FieldType PREFIX_TYPE_STORED = new FieldType();
-  static {
-    PREFIX_TYPE_STORED.setTokenized(false);
-    PREFIX_TYPE_STORED.setOmitNorms(true);
-    PREFIX_TYPE_STORED.setIndexOptions(IndexOptions.DOCS);
-    PREFIX_TYPE_STORED.setDocValuesType(DocValuesType.SORTED_NUMERIC);
-    PREFIX_TYPE_STORED.setStored(true);
-    PREFIX_TYPE_STORED.freeze();
-  }
-
-  /** Creates a stored or un-stored GeoPointField
-   *  @param name field name
-   *  @param lon longitude double value [-180.0 : 180.0]
-   *  @param lat latitude double value [-90.0 : 90.0]
-   *  @param stored Store.YES if the content should also be stored
-   *  @throws IllegalArgumentException if the field name is null.
-   */
-  public GeoPointField(String name, double lon, double lat, Store stored) {
-    this(name, lon, lat, getFieldType(stored));
-  }
-
-  /** Creates a stored or un-stored GeoPointField using the specified {@link TermEncoding} method
-   *  @param name field name
-   *  @param lon longitude double value [-180.0 : 180.0]
-   *  @param lat latitude double value [-90.0 : 90.0]
-   *  @param termEncoding encoding type to use ({@link TermEncoding#NUMERIC} Terms, or {@link TermEncoding#PREFIX} only Terms)
-   *  @param stored Store.YES if the content should also be stored
-   *  @throws IllegalArgumentException if the field name is null.
-   */
-  @Deprecated
-  public GeoPointField(String name, double lon, double lat, TermEncoding termEncoding, Store stored) {
-    this(name, lon, lat, getFieldType(termEncoding, stored));
-  }
-
-  /** Expert: allows you to customize the {@link
-   *  FieldType}.
-   *  @param name field name
-   *  @param lon longitude double value [-180.0 : 180.0]
-   *  @param lat latitude double value [-90.0 : 90.0]
-   *  @param type customized field type: must have {@link FieldType#numericType()}
-   *         of {@link org.apache.lucene.document.FieldType.LegacyNumericType#LONG}.
-   *  @throws IllegalArgumentException if the field name or type is null, or
-   *          if the field type does not have a LONG numericType()
-   */
-  public GeoPointField(String name, double lon, double lat, FieldType type) {
-    super(name, type);
-
-    // field must be indexed
-    // todo does it make sense here to provide the ability to store a GeoPointField but not index?
-    if (type.indexOptions() == IndexOptions.NONE && type.stored() == false) {
-      throw new IllegalArgumentException("type.indexOptions() is set to NONE but type.stored() is false");
-    } else if (type.indexOptions() == IndexOptions.DOCS) {
-      if (type.docValuesType() != DocValuesType.SORTED_NUMERIC) {
-        throw new IllegalArgumentException("type.docValuesType() must be SORTED_NUMERIC but got " + type.docValuesType());
-      }
-      if (type.numericType() != null) {
-        // make sure numericType is a LONG
-        if (type.numericType() != FieldType.LegacyNumericType.LONG) {
-          throw new IllegalArgumentException("type.numericType() must be LONG but got " + type.numericType());
-        }
-      }
-    } else {
-      throw new IllegalArgumentException("type.indexOptions() must be one of NONE or DOCS but got " + type.indexOptions());
-    }
-
-    // set field data
-    fieldsData = GeoEncodingUtils.mortonHash(lon, lat);
-  }
-
-  private static FieldType getFieldType(Store stored) {
-    return getFieldType(TermEncoding.PREFIX, stored);
-  }
-
-  /**
-   * @deprecated
-   * Static helper method for returning a valid FieldType based on termEncoding and stored options
-   */
-  @Deprecated
-  private static FieldType getFieldType(TermEncoding termEncoding, Store stored) {
-    if (stored == Store.YES) {
-      return termEncoding == TermEncoding.PREFIX ? PREFIX_TYPE_STORED : NUMERIC_TYPE_STORED;
-    } else if (stored == Store.NO) {
-      return termEncoding == TermEncoding.PREFIX ? PREFIX_TYPE_NOT_STORED : NUMERIC_TYPE_NOT_STORED;
-    } else {
-      throw new IllegalArgumentException("stored option must be NO or YES but got " + stored);
-    }
-  }
-
-  @Override
-  public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
-    if (fieldType().indexOptions() == IndexOptions.NONE) {
-      // not indexed
-      return null;
-    }
-
-    // if numericType is set
-    if (type.numericType() != null) {
-      // return numeric encoding
-      return super.tokenStream(analyzer, reuse);
-    }
-
-    if (reuse instanceof GeoPointTokenStream == false) {
-      reuse = new GeoPointTokenStream();
-    }
-
-    final GeoPointTokenStream gpts = (GeoPointTokenStream)reuse;
-    gpts.setGeoCode(((Number) fieldsData).longValue());
-
-    return reuse;
-  }
-
-  /** access longitude value */
-  public double getLon() {
-    return GeoEncodingUtils.mortonUnhashLon((long) fieldsData);
-  }
-
-  /** access latitude value */
-  public double getLat() {
-    return GeoEncodingUtils.mortonUnhashLat((long) fieldsData);
-  }
-
-  @Override
-  public String toString() {
-    if (fieldsData == null) {
-      return null;
-    }
-    StringBuilder sb = new StringBuilder();
-    sb.append(GeoEncodingUtils.mortonUnhashLon((long) fieldsData));
-    sb.append(',');
-    sb.append(GeoEncodingUtils.mortonUnhashLat((long) fieldsData));
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/document/GeoPointTokenStream.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/document/GeoPointTokenStream.java b/lucene/spatial/src/java/org/apache/lucene/spatial/document/GeoPointTokenStream.java
deleted file mode 100644
index ff97fee..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/document/GeoPointTokenStream.java
+++ /dev/null
@@ -1,233 +0,0 @@
-package org.apache.lucene.spatial.document;
-
-/*
- * 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.Objects;
-
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
-import org.apache.lucene.util.Attribute;
-import org.apache.lucene.util.AttributeFactory;
-import org.apache.lucene.util.AttributeImpl;
-import org.apache.lucene.util.AttributeReflector;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
-
-import static org.apache.lucene.spatial.document.GeoPointField.PRECISION_STEP;
-
-/**
- * <b>Expert:</b> This class provides a {@link TokenStream} used by {@link GeoPointField}
- * for encoding {@link GeoPointField.TermEncoding#PREFIX} only GeoPointTerms.
- *
- * <p><i>NOTE: This is used as the default encoding unless
- * {@code GeoPointField.setNumericType(FieldType.LegacyNumericType.LONG)} is set</i></p>
- *
- * This class is similar to {@link org.apache.lucene.analysis.LegacyNumericTokenStream} but encodes terms up to a
- * a maximum of {@link #MAX_SHIFT} using a fixed precision step defined by
- * {@link GeoPointField#PRECISION_STEP}. This yields a total of 4 terms per GeoPoint
- * each consisting of 5 bytes (4 prefix bytes + 1 precision byte).
- *
- * <p>For best performance use the provided {@link GeoPointField#PREFIX_TYPE_NOT_STORED} or
- * {@link GeoPointField#PREFIX_TYPE_STORED}</p>
- *
- * <p>If prefix terms are used then the default GeoPoint query constructors may be used, but if
- * {@link org.apache.lucene.analysis.LegacyNumericTokenStream} is used, then be sure to pass
- * {@link GeoPointField.TermEncoding#NUMERIC} to all GeoPointQuery constructors</p>
- *
- * Here's an example usage:
- *
- * <pre class="prettyprint">
- *   // using prefix terms
- *   GeoPointField geoPointField = new GeoPointField(fieldName1, lon, lat, GeoPointField.PREFIX_TYPE_NOT_STORED);
- *   document.add(geoPointField);
- *
- *   // query by bounding box (default uses TermEncoding.PREFIX)
- *   Query q = new GeoPointInBBoxQuery(fieldName1, minLon, minLat, maxLon, maxLat);
- *
- *   // using numeric terms
- *   geoPointField = new GeoPointField(fieldName2, lon, lat, GeoPointField.NUMERIC_TYPE_NOT_STORED);
- *   document.add(geoPointField);
- *
- *   // query by distance (requires TermEncoding.NUMERIC)
- *   q = new GeoPointDistanceQuery(fieldName2, TermEncoding.NUMERIC, centerLon, centerLat, radiusMeters);
- * </pre>
- *
- * @lucene.experimental
- */
-final class GeoPointTokenStream extends TokenStream {
-  private static final int MAX_SHIFT = PRECISION_STEP * 4;
-
-  private final GeoPointTermAttribute geoPointTermAtt = addAttribute(GeoPointTermAttribute.class);
-  private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
-
-  private boolean isInit = false;
-
-  /**
-   * Expert: Creates a token stream for geo point fields with the specified
-   * <code>precisionStep</code> using the given
-   * {@link org.apache.lucene.util.AttributeFactory}.
-   * The stream is not yet initialized,
-   * before using set a value using the various setGeoCode method.
-   */
-  public GeoPointTokenStream() {
-    super(new GeoPointAttributeFactory(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY));
-    assert PRECISION_STEP > 0;
-  }
-
-  public GeoPointTokenStream setGeoCode(final long geoCode) {
-    geoPointTermAtt.init(geoCode, MAX_SHIFT-PRECISION_STEP);
-    isInit = true;
-    return this;
-  }
-
-  @Override
-  public void reset() {
-    if (isInit == false) {
-      throw new IllegalStateException("call setGeoCode() before usage");
-    }
-  }
-
-  @Override
-  public boolean incrementToken() {
-    if (isInit == false) {
-      throw new IllegalStateException("call setGeoCode() before usage");
-    }
-
-    // this will only clear all other attributes in this TokenStream
-    clearAttributes();
-
-    final int shift = geoPointTermAtt.incShift();
-    posIncrAtt.setPositionIncrement((shift == MAX_SHIFT) ? 1 : 0);
-    return (shift < 63);
-  }
-
-  /**
-   * Tracks shift values during encoding
-   */
-  public interface GeoPointTermAttribute extends Attribute {
-    /** Returns current shift value, undefined before first token */
-    int getShift();
-
-    /** <em>Don't call this method!</em>
-     * @lucene.internal */
-    void init(long value, int shift);
-
-    /** <em>Don't call this method!</em>
-     * @lucene.internal */
-    int incShift();
-  }
-
-  // just a wrapper to prevent adding CTA
-  private static final class GeoPointAttributeFactory extends AttributeFactory {
-    private final AttributeFactory delegate;
-
-    GeoPointAttributeFactory(AttributeFactory delegate) {
-      this.delegate = delegate;
-    }
-
-    @Override
-    public AttributeImpl createAttributeInstance(Class<? extends Attribute> attClass) {
-      if (CharTermAttribute.class.isAssignableFrom(attClass)) {
-        throw new IllegalArgumentException("GeoPointTokenStream does not support CharTermAttribute.");
-      }
-      return delegate.createAttributeInstance(attClass);
-    }
-  }
-
-  public static final class GeoPointTermAttributeImpl extends AttributeImpl implements GeoPointTermAttribute,TermToBytesRefAttribute {
-    private long value = 0L;
-    private int shift = 0;
-    private BytesRefBuilder bytes = new BytesRefBuilder();
-
-    public GeoPointTermAttributeImpl() {
-      this.shift = MAX_SHIFT-PRECISION_STEP;
-    }
-
-    @Override
-    public BytesRef getBytesRef() {
-      GeoEncodingUtils.geoCodedToPrefixCoded(value, shift, bytes);
-      return bytes.get();
-    }
-
-    @Override
-    public void init(long value, int shift) {
-      this.value = value;
-      this.shift = shift;
-    }
-
-    @Override
-    public int getShift() { return shift; }
-
-    @Override
-    public int incShift() {
-      return (shift += PRECISION_STEP);
-    }
-
-    @Override
-    public void clear() {
-      // this attribute has no contents to clear!
-      // we keep it untouched as it's fully controlled by outer class.
-    }
-
-    @Override
-    public void reflectWith(AttributeReflector reflector) {
-      reflector.reflect(TermToBytesRefAttribute.class, "bytes", getBytesRef());
-      reflector.reflect(GeoPointTermAttribute.class, "shift", shift);
-    }
-
-    @Override
-    public void copyTo(AttributeImpl target) {
-      final GeoPointTermAttribute a = (GeoPointTermAttribute) target;
-      a.init(value, shift);
-    }
-
-    @Override
-    public GeoPointTermAttributeImpl clone() {
-      GeoPointTermAttributeImpl t = (GeoPointTermAttributeImpl)super.clone();
-      // Do a deep clone
-      t.bytes = new BytesRefBuilder();
-      t.bytes.copyBytes(getBytesRef());
-      return t;
-    }
-
-    @Override
-    public int hashCode() {
-      return Objects.hash(shift, value);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (this == obj) return true;
-      if (obj == null) return false;
-      if (getClass() != obj.getClass()) return false;
-      GeoPointTermAttributeImpl other = (GeoPointTermAttributeImpl) obj;
-      if (shift != other.shift) return false;
-      if (value != other.value) return false;
-      return true;
-    }
-  }
-
-  /** override toString because it can throw cryptic "illegal shift value": */
-  @Override
-  public String toString() {
-    return getClass().getSimpleName() + "(precisionStep=" + PRECISION_STEP + " shift=" + geoPointTermAtt.getShift() + ")";
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/document/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/document/package-info.java b/lucene/spatial/src/java/org/apache/lucene/spatial/document/package-info.java
deleted file mode 100644
index 2550fa1..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/document/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.
- */
-
-/**
- * Geospatial Field Implementations for Core Lucene
- */
-package org.apache.lucene.spatial.document;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointField.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointField.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointField.java
new file mode 100644
index 0000000..39f12df
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointField.java
@@ -0,0 +1,250 @@
+/*
+ * 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.geopoint.document;
+
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.spatial.util.GeoEncodingUtils;
+
+/**
+ * <p>
+ * Field that indexes <code>latitude</code> <code>longitude</code> decimal-degree values
+ * for efficient encoding, sorting, and querying. This Geo capability is intended
+ * to provide a basic and efficient out of the box field type for indexing and
+ * querying 2 dimensional points in WGS-84 decimal degrees. An example usage is as follows:
+ *
+ * <pre class="prettyprint">
+ *  document.add(new GeoPointField(name, -96.33, 32.66, Field.Store.NO));
+ * </pre>
+ *
+ * <p>To perform simple geospatial queries against a <code>GeoPointField</code>,
+ * see {@link org.apache.lucene.spatial.geopoint.search.GeoPointInBBoxQuery}, {@link org.apache.lucene.spatial.geopoint.search.GeoPointInPolygonQuery},
+ * or {@link org.apache.lucene.spatial.geopoint.search.GeoPointDistanceQuery}
+ *
+ * NOTE: This indexes only high precision encoded terms which may result in visiting a high number
+ * of terms for large queries. See LUCENE-6481 for a future improvement.
+ *
+ * @lucene.experimental
+ */
+public final class GeoPointField extends Field {
+  /** encoding step value for GeoPoint prefix terms */
+  public static final int PRECISION_STEP = 9;
+
+  /**
+   * <b>Expert:</b> Optional flag to select term encoding for GeoPointField types
+   */
+  public enum TermEncoding {
+    /**
+     * encodes prefix terms only resulting in a small index and faster queries - use with
+     * {@code GeoPointTokenStream}
+     */
+    PREFIX,
+    /**
+     * @deprecated encodes prefix and full resolution terms - use with
+     * {@link org.apache.lucene.analysis.LegacyNumericTokenStream}
+     */
+    @Deprecated
+    NUMERIC
+  }
+
+  /**
+   * @deprecated Type for a GeoPointField that is not stored:
+   * normalization factors, frequencies, and positions are omitted.
+   */
+  @Deprecated
+  public static final FieldType NUMERIC_TYPE_NOT_STORED = new FieldType();
+  static {
+    NUMERIC_TYPE_NOT_STORED.setTokenized(false);
+    NUMERIC_TYPE_NOT_STORED.setOmitNorms(true);
+    NUMERIC_TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS);
+    NUMERIC_TYPE_NOT_STORED.setDocValuesType(DocValuesType.SORTED_NUMERIC);
+    NUMERIC_TYPE_NOT_STORED.setNumericType(FieldType.LegacyNumericType.LONG);
+    NUMERIC_TYPE_NOT_STORED.setNumericPrecisionStep(PRECISION_STEP);
+    NUMERIC_TYPE_NOT_STORED.freeze();
+  }
+
+  /**
+   * @deprecated Type for a stored GeoPointField:
+   * normalization factors, frequencies, and positions are omitted.
+   */
+  @Deprecated
+  public static final FieldType NUMERIC_TYPE_STORED = new FieldType();
+  static {
+    NUMERIC_TYPE_STORED.setTokenized(false);
+    NUMERIC_TYPE_STORED.setOmitNorms(true);
+    NUMERIC_TYPE_STORED.setIndexOptions(IndexOptions.DOCS);
+    NUMERIC_TYPE_STORED.setDocValuesType(DocValuesType.SORTED_NUMERIC);
+    NUMERIC_TYPE_STORED.setNumericType(FieldType.LegacyNumericType.LONG);
+    NUMERIC_TYPE_STORED.setNumericPrecisionStep(PRECISION_STEP);
+    NUMERIC_TYPE_STORED.setStored(true);
+    NUMERIC_TYPE_STORED.freeze();
+  }
+
+  /**
+   * Type for a GeoPointField that is not stored:
+   * normalization factors, frequencies, and positions are omitted.
+   */
+  public static final FieldType PREFIX_TYPE_NOT_STORED = new FieldType();
+  static {
+    PREFIX_TYPE_NOT_STORED.setTokenized(false);
+    PREFIX_TYPE_NOT_STORED.setOmitNorms(true);
+    PREFIX_TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS);
+    PREFIX_TYPE_NOT_STORED.setDocValuesType(DocValuesType.SORTED_NUMERIC);
+    PREFIX_TYPE_NOT_STORED.freeze();
+  }
+
+  /**
+   * Type for a stored GeoPointField:
+   * normalization factors, frequencies, and positions are omitted.
+   */
+  public static final FieldType PREFIX_TYPE_STORED = new FieldType();
+  static {
+    PREFIX_TYPE_STORED.setTokenized(false);
+    PREFIX_TYPE_STORED.setOmitNorms(true);
+    PREFIX_TYPE_STORED.setIndexOptions(IndexOptions.DOCS);
+    PREFIX_TYPE_STORED.setDocValuesType(DocValuesType.SORTED_NUMERIC);
+    PREFIX_TYPE_STORED.setStored(true);
+    PREFIX_TYPE_STORED.freeze();
+  }
+
+  /** Creates a stored or un-stored GeoPointField
+   *  @param name field name
+   *  @param lon longitude double value [-180.0 : 180.0]
+   *  @param lat latitude double value [-90.0 : 90.0]
+   *  @param stored Store.YES if the content should also be stored
+   *  @throws IllegalArgumentException if the field name is null.
+   */
+  public GeoPointField(String name, double lon, double lat, Store stored) {
+    this(name, lon, lat, getFieldType(stored));
+  }
+
+  /** Creates a stored or un-stored GeoPointField using the specified {@link TermEncoding} method
+   *  @param name field name
+   *  @param lon longitude double value [-180.0 : 180.0]
+   *  @param lat latitude double value [-90.0 : 90.0]
+   *  @param termEncoding encoding type to use ({@link TermEncoding#NUMERIC} Terms, or {@link TermEncoding#PREFIX} only Terms)
+   *  @param stored Store.YES if the content should also be stored
+   *  @throws IllegalArgumentException if the field name is null.
+   */
+  @Deprecated
+  public GeoPointField(String name, double lon, double lat, TermEncoding termEncoding, Store stored) {
+    this(name, lon, lat, getFieldType(termEncoding, stored));
+  }
+
+  /** Expert: allows you to customize the {@link
+   *  FieldType}.
+   *  @param name field name
+   *  @param lon longitude double value [-180.0 : 180.0]
+   *  @param lat latitude double value [-90.0 : 90.0]
+   *  @param type customized field type: must have {@link FieldType#numericType()}
+   *         of {@link org.apache.lucene.document.FieldType.LegacyNumericType#LONG}.
+   *  @throws IllegalArgumentException if the field name or type is null, or
+   *          if the field type does not have a LONG numericType()
+   */
+  public GeoPointField(String name, double lon, double lat, FieldType type) {
+    super(name, type);
+
+    // field must be indexed
+    // todo does it make sense here to provide the ability to store a GeoPointField but not index?
+    if (type.indexOptions() == IndexOptions.NONE && type.stored() == false) {
+      throw new IllegalArgumentException("type.indexOptions() is set to NONE but type.stored() is false");
+    } else if (type.indexOptions() == IndexOptions.DOCS) {
+      if (type.docValuesType() != DocValuesType.SORTED_NUMERIC) {
+        throw new IllegalArgumentException("type.docValuesType() must be SORTED_NUMERIC but got " + type.docValuesType());
+      }
+      if (type.numericType() != null) {
+        // make sure numericType is a LONG
+        if (type.numericType() != FieldType.LegacyNumericType.LONG) {
+          throw new IllegalArgumentException("type.numericType() must be LONG but got " + type.numericType());
+        }
+      }
+    } else {
+      throw new IllegalArgumentException("type.indexOptions() must be one of NONE or DOCS but got " + type.indexOptions());
+    }
+
+    // set field data
+    fieldsData = GeoEncodingUtils.mortonHash(lon, lat);
+  }
+
+  private static FieldType getFieldType(Store stored) {
+    return getFieldType(TermEncoding.PREFIX, stored);
+  }
+
+  /**
+   * @deprecated
+   * Static helper method for returning a valid FieldType based on termEncoding and stored options
+   */
+  @Deprecated
+  private static FieldType getFieldType(TermEncoding termEncoding, Store stored) {
+    if (stored == Store.YES) {
+      return termEncoding == TermEncoding.PREFIX ? PREFIX_TYPE_STORED : NUMERIC_TYPE_STORED;
+    } else if (stored == Store.NO) {
+      return termEncoding == TermEncoding.PREFIX ? PREFIX_TYPE_NOT_STORED : NUMERIC_TYPE_NOT_STORED;
+    } else {
+      throw new IllegalArgumentException("stored option must be NO or YES but got " + stored);
+    }
+  }
+
+  @Override
+  public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
+    if (fieldType().indexOptions() == IndexOptions.NONE) {
+      // not indexed
+      return null;
+    }
+
+    // if numericType is set
+    if (type.numericType() != null) {
+      // return numeric encoding
+      return super.tokenStream(analyzer, reuse);
+    }
+
+    if (reuse instanceof GeoPointTokenStream == false) {
+      reuse = new GeoPointTokenStream();
+    }
+
+    final GeoPointTokenStream gpts = (GeoPointTokenStream)reuse;
+    gpts.setGeoCode(((Number) fieldsData).longValue());
+
+    return reuse;
+  }
+
+  /** access longitude value */
+  public double getLon() {
+    return GeoEncodingUtils.mortonUnhashLon((long) fieldsData);
+  }
+
+  /** access latitude value */
+  public double getLat() {
+    return GeoEncodingUtils.mortonUnhashLat((long) fieldsData);
+  }
+
+  @Override
+  public String toString() {
+    if (fieldsData == null) {
+      return null;
+    }
+    StringBuilder sb = new StringBuilder();
+    sb.append(GeoEncodingUtils.mortonUnhashLon((long) fieldsData));
+    sb.append(',');
+    sb.append(GeoEncodingUtils.mortonUnhashLat((long) fieldsData));
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointTokenStream.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointTokenStream.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointTokenStream.java
new file mode 100644
index 0000000..4a70ea3
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/GeoPointTokenStream.java
@@ -0,0 +1,233 @@
+package org.apache.lucene.spatial.geopoint.document;
+
+/*
+ * 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.Objects;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.util.Attribute;
+import org.apache.lucene.util.AttributeFactory;
+import org.apache.lucene.util.AttributeImpl;
+import org.apache.lucene.util.AttributeReflector;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.spatial.util.GeoEncodingUtils;
+
+import static org.apache.lucene.spatial.geopoint.document.GeoPointField.PRECISION_STEP;
+
+/**
+ * <b>Expert:</b> This class provides a {@link TokenStream} used by {@link GeoPointField}
+ * for encoding {@link GeoPointField.TermEncoding#PREFIX} only GeoPointTerms.
+ *
+ * <p><i>NOTE: This is used as the default encoding unless
+ * {@code GeoPointField.setNumericType(FieldType.LegacyNumericType.LONG)} is set</i></p>
+ *
+ * This class is similar to {@link org.apache.lucene.analysis.LegacyNumericTokenStream} but encodes terms up to a
+ * a maximum of {@link #MAX_SHIFT} using a fixed precision step defined by
+ * {@link GeoPointField#PRECISION_STEP}. This yields a total of 4 terms per GeoPoint
+ * each consisting of 5 bytes (4 prefix bytes + 1 precision byte).
+ *
+ * <p>For best performance use the provided {@link GeoPointField#PREFIX_TYPE_NOT_STORED} or
+ * {@link GeoPointField#PREFIX_TYPE_STORED}</p>
+ *
+ * <p>If prefix terms are used then the default GeoPoint query constructors may be used, but if
+ * {@link org.apache.lucene.analysis.LegacyNumericTokenStream} is used, then be sure to pass
+ * {@link GeoPointField.TermEncoding#NUMERIC} to all GeoPointQuery constructors</p>
+ *
+ * Here's an example usage:
+ *
+ * <pre class="prettyprint">
+ *   // using prefix terms
+ *   GeoPointField geoPointField = new GeoPointField(fieldName1, lon, lat, GeoPointField.PREFIX_TYPE_NOT_STORED);
+ *   document.add(geoPointField);
+ *
+ *   // query by bounding box (default uses TermEncoding.PREFIX)
+ *   Query q = new GeoPointInBBoxQuery(fieldName1, minLon, minLat, maxLon, maxLat);
+ *
+ *   // using numeric terms
+ *   geoPointField = new GeoPointField(fieldName2, lon, lat, GeoPointField.NUMERIC_TYPE_NOT_STORED);
+ *   document.add(geoPointField);
+ *
+ *   // query by distance (requires TermEncoding.NUMERIC)
+ *   q = new GeoPointDistanceQuery(fieldName2, TermEncoding.NUMERIC, centerLon, centerLat, radiusMeters);
+ * </pre>
+ *
+ * @lucene.experimental
+ */
+final class GeoPointTokenStream extends TokenStream {
+  private static final int MAX_SHIFT = PRECISION_STEP * 4;
+
+  private final GeoPointTermAttribute geoPointTermAtt = addAttribute(GeoPointTermAttribute.class);
+  private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
+
+  private boolean isInit = false;
+
+  /**
+   * Expert: Creates a token stream for geo point fields with the specified
+   * <code>precisionStep</code> using the given
+   * {@link org.apache.lucene.util.AttributeFactory}.
+   * The stream is not yet initialized,
+   * before using set a value using the various setGeoCode method.
+   */
+  public GeoPointTokenStream() {
+    super(new GeoPointAttributeFactory(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY));
+    assert PRECISION_STEP > 0;
+  }
+
+  public GeoPointTokenStream setGeoCode(final long geoCode) {
+    geoPointTermAtt.init(geoCode, MAX_SHIFT-PRECISION_STEP);
+    isInit = true;
+    return this;
+  }
+
+  @Override
+  public void reset() {
+    if (isInit == false) {
+      throw new IllegalStateException("call setGeoCode() before usage");
+    }
+  }
+
+  @Override
+  public boolean incrementToken() {
+    if (isInit == false) {
+      throw new IllegalStateException("call setGeoCode() before usage");
+    }
+
+    // this will only clear all other attributes in this TokenStream
+    clearAttributes();
+
+    final int shift = geoPointTermAtt.incShift();
+    posIncrAtt.setPositionIncrement((shift == MAX_SHIFT) ? 1 : 0);
+    return (shift < 63);
+  }
+
+  /**
+   * Tracks shift values during encoding
+   */
+  public interface GeoPointTermAttribute extends Attribute {
+    /** Returns current shift value, undefined before first token */
+    int getShift();
+
+    /** <em>Don't call this method!</em>
+     * @lucene.internal */
+    void init(long value, int shift);
+
+    /** <em>Don't call this method!</em>
+     * @lucene.internal */
+    int incShift();
+  }
+
+  // just a wrapper to prevent adding CTA
+  private static final class GeoPointAttributeFactory extends AttributeFactory {
+    private final AttributeFactory delegate;
+
+    GeoPointAttributeFactory(AttributeFactory delegate) {
+      this.delegate = delegate;
+    }
+
+    @Override
+    public AttributeImpl createAttributeInstance(Class<? extends Attribute> attClass) {
+      if (CharTermAttribute.class.isAssignableFrom(attClass)) {
+        throw new IllegalArgumentException("GeoPointTokenStream does not support CharTermAttribute.");
+      }
+      return delegate.createAttributeInstance(attClass);
+    }
+  }
+
+  public static final class GeoPointTermAttributeImpl extends AttributeImpl implements GeoPointTermAttribute,TermToBytesRefAttribute {
+    private long value = 0L;
+    private int shift = 0;
+    private BytesRefBuilder bytes = new BytesRefBuilder();
+
+    public GeoPointTermAttributeImpl() {
+      this.shift = MAX_SHIFT-PRECISION_STEP;
+    }
+
+    @Override
+    public BytesRef getBytesRef() {
+      GeoEncodingUtils.geoCodedToPrefixCoded(value, shift, bytes);
+      return bytes.get();
+    }
+
+    @Override
+    public void init(long value, int shift) {
+      this.value = value;
+      this.shift = shift;
+    }
+
+    @Override
+    public int getShift() { return shift; }
+
+    @Override
+    public int incShift() {
+      return (shift += PRECISION_STEP);
+    }
+
+    @Override
+    public void clear() {
+      // this attribute has no contents to clear!
+      // we keep it untouched as it's fully controlled by outer class.
+    }
+
+    @Override
+    public void reflectWith(AttributeReflector reflector) {
+      reflector.reflect(TermToBytesRefAttribute.class, "bytes", getBytesRef());
+      reflector.reflect(GeoPointTermAttribute.class, "shift", shift);
+    }
+
+    @Override
+    public void copyTo(AttributeImpl target) {
+      final GeoPointTermAttribute a = (GeoPointTermAttribute) target;
+      a.init(value, shift);
+    }
+
+    @Override
+    public GeoPointTermAttributeImpl clone() {
+      GeoPointTermAttributeImpl t = (GeoPointTermAttributeImpl)super.clone();
+      // Do a deep clone
+      t.bytes = new BytesRefBuilder();
+      t.bytes.copyBytes(getBytesRef());
+      return t;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(shift, value);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) return true;
+      if (obj == null) return false;
+      if (getClass() != obj.getClass()) return false;
+      GeoPointTermAttributeImpl other = (GeoPointTermAttributeImpl) obj;
+      if (shift != other.shift) return false;
+      if (value != other.value) return false;
+      return true;
+    }
+  }
+
+  /** override toString because it can throw cryptic "illegal shift value": */
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(precisionStep=" + PRECISION_STEP + " shift=" + geoPointTermAtt.getShift() + ")";
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/package-info.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/package-info.java
new file mode 100644
index 0000000..2d23448
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/document/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 Field Implementations for Core Lucene
+ */
+package org.apache.lucene.spatial.geopoint.document;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoBoundingBox.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoBoundingBox.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoBoundingBox.java
new file mode 100644
index 0000000..8f30f60
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoBoundingBox.java
@@ -0,0 +1,53 @@
+/*
+ * 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.geopoint.search;
+
+import org.apache.lucene.spatial.util.GeoUtils;
+
+/** NOTE: package private; just used so {@link GeoPointInPolygonQuery} can communicate its bounding box to {@link GeoPointInBBoxQuery}. */
+class GeoBoundingBox {
+  /** minimum longitude value (in degrees) */
+  public final double minLon;
+  /** minimum latitude value (in degrees) */
+  public final double maxLon;
+  /** maximum longitude value (in degrees) */
+  public final double minLat;
+  /** maximum latitude value (in degrees) */
+  public final double maxLat;
+
+  /**
+   * Constructs a bounding box by first validating the provided latitude and longitude coordinates
+   */
+  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 " + minLon);
+    }
+    if (GeoUtils.isValidLat(minLat) == false) {
+      throw new IllegalArgumentException("invalid minLat " + minLat);
+    }
+    if (GeoUtils.isValidLat(maxLat) == false) {
+      throw new IllegalArgumentException("invalid maxLat " + minLat);
+    }
+    this.minLon = minLon;
+    this.maxLon = maxLon;
+    this.minLat = minLat;
+    this.maxLat = maxLat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointDistanceQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointDistanceQuery.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointDistanceQuery.java
new file mode 100644
index 0000000..bd3dbdd
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointDistanceQuery.java
@@ -0,0 +1,189 @@
+/*
+ * 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.geopoint.search;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding;
+import org.apache.lucene.spatial.util.GeoDistanceUtils;
+import org.apache.lucene.spatial.util.GeoRect;
+import org.apache.lucene.spatial.util.GeoUtils;
+
+/** Implements a simple point distance query on a GeoPoint field. This is based on
+ * {@link GeoPointInBBoxQuery} and is implemented using a two phase approach. First,
+ * like {@code GeoPointInBBoxQueryImpl} candidate terms are queried using the numeric ranges based on
+ * the morton codes of the min and max lat/lon pairs that intersect the boundary of the point-radius
+ * circle. Terms
+ * passing this initial filter are then passed to a secondary {@code postFilter} method that verifies whether the
+ * decoded lat/lon point fall within the specified query distance (see {@link org.apache.lucene.util.SloppyMath#haversin}.
+ * All morton value comparisons are subject to the same precision tolerance defined in
+ * {@value org.apache.lucene.spatial.util.GeoEncodingUtils#TOLERANCE} and distance comparisons are subject to the accuracy of the
+ * haversine formula (from R.W. Sinnott, "Virtues of the Haversine", Sky and Telescope, vol. 68, no. 2, 1984, p. 159)
+ *
+ * <p>Note: This query currently uses haversine which is a sloppy distance calculation (see above reference). For large
+ * queries one can expect upwards of 400m error. Vincenty shrinks this to ~40m error but pays a penalty for computing
+ * using the spheroid
+ *
+ * @lucene.experimental */
+public class GeoPointDistanceQuery extends GeoPointInBBoxQuery {
+  /** longitude value (in degrees) for query location */
+  protected final double centerLon;
+  /** latitude value (in degrees) for query location */
+  protected final double centerLat;
+  /** distance (in meters) from lon, lat center location */
+  protected final double radiusMeters;
+
+  /**
+   * Constructs a Query for all {@link org.apache.lucene.spatial.geopoint.document.GeoPointField} types within a
+   * distance (in meters) from a given point
+   **/
+  public GeoPointDistanceQuery(final String field, final double centerLon, final double centerLat, final double radiusMeters) {
+    this(field, TermEncoding.PREFIX, centerLon, centerLat, radiusMeters);
+  }
+
+  public GeoPointDistanceQuery(final String field, final TermEncoding termEncoding, final double centerLon, final double centerLat, final double radiusMeters) {
+    this(field, termEncoding, GeoUtils.circleToBBox(centerLon, centerLat, radiusMeters), centerLon, centerLat, radiusMeters);
+  }
+
+  private GeoPointDistanceQuery(final String field, final TermEncoding termEncoding, final GeoRect bbox, final double centerLon,
+                                final double centerLat, final double radiusMeters) {
+    super(field, termEncoding, bbox.minLon, bbox.minLat, bbox.maxLon, bbox.maxLat);
+    {
+      // check longitudinal overlap (restrict distance to maximum longitudinal radius)
+      // todo this restriction technically shouldn't be needed,
+      // its only purpose is to ensure the bounding box doesn't self overlap.
+      final double maxRadius = GeoDistanceUtils.maxRadialDistanceMeters(centerLon, centerLat);
+      if (radiusMeters > maxRadius) {
+        throw new IllegalArgumentException("radiusMeters " + radiusMeters + " exceeds maxRadius [" + maxRadius
+            + "] at location [" + centerLon + " " + centerLat + "]");
+      }
+    }
+
+    if (GeoUtils.isValidLon(centerLon) == false) {
+      throw new IllegalArgumentException("invalid centerLon " + centerLon);
+    }
+
+    if (GeoUtils.isValidLat(centerLat) == false) {
+      throw new IllegalArgumentException("invalid centerLat " + centerLat);
+    }
+
+    if (radiusMeters <= 0.0) {
+      throw new IllegalArgumentException("invalid radiusMeters " + radiusMeters);
+    }
+
+    this.centerLon = centerLon;
+    this.centerLat = centerLat;
+    this.radiusMeters = radiusMeters;
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) {
+    // query crosses dateline; split into left and right queries
+    if (maxLon < minLon) {
+      BooleanQuery.Builder bqb = new BooleanQuery.Builder();
+
+      // unwrap the longitude iff outside the specified min/max lon range
+      double unwrappedLon = centerLon;
+      if (unwrappedLon > maxLon) {
+        // unwrap left
+        unwrappedLon += -360.0D;
+      }
+      GeoPointDistanceQueryImpl left = new GeoPointDistanceQueryImpl(field, termEncoding, this, unwrappedLon,
+          new GeoRect(GeoUtils.MIN_LON_INCL, maxLon, minLat, maxLat));
+      bqb.add(new BooleanClause(left, BooleanClause.Occur.SHOULD));
+
+      if (unwrappedLon < maxLon) {
+        // unwrap right
+        unwrappedLon += 360.0D;
+      }
+      GeoPointDistanceQueryImpl right = new GeoPointDistanceQueryImpl(field, termEncoding, this, unwrappedLon,
+          new GeoRect(minLon, GeoUtils.MAX_LON_INCL, minLat, maxLat));
+      bqb.add(new BooleanClause(right, BooleanClause.Occur.SHOULD));
+
+      return bqb.build();
+    }
+    return new GeoPointDistanceQueryImpl(field, termEncoding, this, centerLon,
+        new GeoRect(this.minLon, this.maxLon, this.minLat, this.maxLat));
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!(o instanceof GeoPointDistanceQuery)) return false;
+    if (!super.equals(o)) return false;
+
+    GeoPointDistanceQuery that = (GeoPointDistanceQuery) o;
+
+    if (Double.compare(that.centerLat, centerLat) != 0) return false;
+    if (Double.compare(that.centerLon, centerLon) != 0) return false;
+    if (Double.compare(that.radiusMeters, radiusMeters) != 0) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    long temp;
+    temp = Double.doubleToLongBits(centerLon);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    temp = Double.doubleToLongBits(centerLat);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    temp = Double.doubleToLongBits(radiusMeters);
+    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 (!this.field.equals(field)) {
+      sb.append(" field=");
+      sb.append(this.field);
+      sb.append(':');
+    }
+    return sb.append( " Center: [")
+        .append(centerLon)
+        .append(',')
+        .append(centerLat)
+        .append(']')
+        .append(" Distance: ")
+        .append(radiusMeters)
+        .append(" meters")
+        .append("]")
+        .toString();
+  }
+
+  /** getter method for center longitude value */
+  public double getCenterLon() {
+    return this.centerLon;
+  }
+
+  /** getter method for center latitude value */
+  public double getCenterLat() {
+    return this.centerLat;
+  }
+
+  /** getter method for distance value (in meters) */
+  public double getRadiusMeters() {
+    return this.radiusMeters;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointDistanceQueryImpl.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointDistanceQueryImpl.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointDistanceQueryImpl.java
new file mode 100644
index 0000000..f091bcb
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointDistanceQueryImpl.java
@@ -0,0 +1,107 @@
+/*
+ * 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.geopoint.search;
+
+import org.apache.lucene.search.MultiTermQuery;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding;
+import org.apache.lucene.spatial.util.GeoRect;
+import org.apache.lucene.spatial.util.GeoRelationUtils;
+import org.apache.lucene.util.SloppyMath;
+
+/** Package private implementation for the public facing GeoPointDistanceQuery delegate class.
+ *
+ *    @lucene.experimental
+ */
+final class GeoPointDistanceQueryImpl extends GeoPointInBBoxQueryImpl {
+  private final GeoPointDistanceQuery distanceQuery;
+  private final double centerLon;
+
+  GeoPointDistanceQueryImpl(final String field, final TermEncoding termEncoding, final GeoPointDistanceQuery q,
+                            final double centerLonUnwrapped, final GeoRect bbox) {
+    super(field, termEncoding, bbox.minLon, bbox.minLat, bbox.maxLon, bbox.maxLat);
+    distanceQuery = q;
+    centerLon = centerLonUnwrapped;
+  }
+
+  @Override
+  public void setRewriteMethod(MultiTermQuery.RewriteMethod method) {
+    throw new UnsupportedOperationException("cannot change rewrite method");
+  }
+
+  @Override
+  protected CellComparator newCellComparator() {
+    return new GeoPointRadiusCellComparator(this);
+  }
+
+  private final class GeoPointRadiusCellComparator extends CellComparator {
+    GeoPointRadiusCellComparator(GeoPointDistanceQueryImpl query) {
+      super(query);
+    }
+
+    @Override
+    protected boolean cellCrosses(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+      return GeoRelationUtils.rectCrossesCircle(minLon, minLat, maxLon, maxLat,
+          centerLon, distanceQuery.centerLat, distanceQuery.radiusMeters, true);
+    }
+
+    @Override
+    protected boolean cellWithin(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+      return GeoRelationUtils.rectWithinCircle(minLon, minLat, maxLon, maxLat,
+          centerLon, distanceQuery.centerLat, distanceQuery.radiusMeters, true);
+    }
+
+    @Override
+    protected boolean cellIntersectsShape(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+      return cellCrosses(minLon, minLat, maxLon, maxLat);
+    }
+
+    /**
+     * The two-phase query approach. The parent {@link GeoPointTermsEnum} class matches
+     * encoded terms that fall within the minimum bounding box of the point-radius circle. Those documents that pass
+     * the initial bounding box filter are then post filter compared to the provided distance using the
+     * {@link org.apache.lucene.util.SloppyMath#haversin} method.
+     */
+    @Override
+    protected boolean postFilter(final double lon, final double lat) {
+      return (SloppyMath.haversin(distanceQuery.centerLat, centerLon, lat, lon) * 1000.0 <= distanceQuery.radiusMeters);
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!(o instanceof GeoPointDistanceQueryImpl)) return false;
+    if (!super.equals(o)) return false;
+
+    GeoPointDistanceQueryImpl that = (GeoPointDistanceQueryImpl) o;
+
+    if (!distanceQuery.equals(that.distanceQuery)) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + distanceQuery.hashCode();
+    return result;
+  }
+
+  public double getRadiusMeters() {
+    return distanceQuery.getRadiusMeters();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointDistanceRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointDistanceRangeQuery.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointDistanceRangeQuery.java
new file mode 100644
index 0000000..e7faccb
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointDistanceRangeQuery.java
@@ -0,0 +1,116 @@
+/*
+ * 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.geopoint.search;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding;
+
+/** Implements a point distance range query on a GeoPoint field. This is based on
+ * {@code org.apache.lucene.spatial.geopoint.search.GeoPointDistanceQuery} and is implemented using a
+ * {@code org.apache.lucene.search.BooleanClause.MUST_NOT} clause to exclude any points that fall within
+ * minRadiusMeters from the provided point.
+ *
+ *    @lucene.experimental
+ */
+public final class GeoPointDistanceRangeQuery extends GeoPointDistanceQuery {
+  protected final double minRadiusMeters;
+
+  /**
+   * Constructs a query for all {@link org.apache.lucene.spatial.geopoint.document.GeoPointField} types within a minimum / maximum
+   * distance (in meters) range from a given point
+   */
+  public GeoPointDistanceRangeQuery(final String field, final double centerLon, final double centerLat,
+                                    final double minRadiusMeters, final double maxRadiusMeters) {
+    this(field, TermEncoding.PREFIX, centerLon, centerLat, minRadiusMeters, maxRadiusMeters);
+  }
+
+  public GeoPointDistanceRangeQuery(final String field, final TermEncoding termEncoding, final double centerLon, final double centerLat,
+                                    final double minRadiusMeters, final double maxRadius) {
+    super(field, termEncoding, centerLon, centerLat, maxRadius);
+    this.minRadiusMeters = minRadiusMeters;
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) {
+    Query q = super.rewrite(reader);
+    if (minRadiusMeters == 0.0) {
+      return q;
+    }
+
+    // add an exclusion query
+    BooleanQuery.Builder bqb = new BooleanQuery.Builder();
+
+    // create a new exclusion query
+    GeoPointDistanceQuery exclude = new GeoPointDistanceQuery(field, termEncoding, centerLon, centerLat, minRadiusMeters);
+    // full map search
+//    if (radiusMeters >= GeoProjectionUtils.SEMIMINOR_AXIS) {
+//      bqb.add(new BooleanClause(new GeoPointInBBoxQuery(this.field, -180.0, -90.0, 180.0, 90.0), BooleanClause.Occur.MUST));
+//    } else {
+      bqb.add(new BooleanClause(q, BooleanClause.Occur.MUST));
+//    }
+    bqb.add(new BooleanClause(exclude, BooleanClause.Occur.MUST_NOT));
+
+    return bqb.build();
+  }
+
+  @Override
+  public String toString(String field) {
+    final StringBuilder sb = new StringBuilder();
+    sb.append(getClass().getSimpleName());
+    sb.append(':');
+    if (!this.field.equals(field)) {
+      sb.append(" field=");
+      sb.append(this.field);
+      sb.append(':');
+    }
+    return sb.append( " Center: [")
+        .append(centerLon)
+        .append(',')
+        .append(centerLat)
+        .append(']')
+        .append(" From Distance: ")
+        .append(minRadiusMeters)
+        .append(" m")
+        .append(" To Distance: ")
+        .append(radiusMeters)
+        .append(" m")
+        .append(" Lower Left: [")
+        .append(minLon)
+        .append(',')
+        .append(minLat)
+        .append(']')
+        .append(" Upper Right: [")
+        .append(maxLon)
+        .append(',')
+        .append(maxLat)
+        .append("]")
+        .toString();
+  }
+
+  /** getter method for minimum distance */
+  public double getMinRadiusMeters() {
+    return this.minRadiusMeters;
+  }
+
+  /** getter method for maximum distance */
+  public double getMaxRadiusMeters() {
+    return this.radiusMeters;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInBBoxQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInBBoxQuery.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInBBoxQuery.java
new file mode 100644
index 0000000..64e8f76
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInBBoxQuery.java
@@ -0,0 +1,173 @@
+/*
+ * 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.geopoint.search;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.FieldValueQuery;
+import org.apache.lucene.search.LegacyNumericRangeQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding;
+import org.apache.lucene.spatial.util.GeoUtils;
+
+/** Implements a simple bounding box query on a GeoPoint field. This is inspired by
+ * {@link LegacyNumericRangeQuery} and is implemented using a
+ * two phase approach. First, 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 final check that verifies whether
+ * the decoded lat/lon falls within (or on the boundary) of the query bounding box.
+ * The value comparisons are subject to a precision tolerance defined in
+ * {@value org.apache.lucene.spatial.util.GeoEncodingUtils#TOLERANCE}
+ *
+ * NOTES:
+ *    1.  All latitude/longitude values must be in decimal degrees.
+ *    2.  Complex computational geometry (e.g., dateline wrapping) is not supported
+ *    3.  For more advanced GeoSpatial indexing and query operations see spatial module
+ *    4.  This is well suited for small rectangles, large bounding boxes may result
+ *        in many terms, depending whether the bounding box falls on the boundary of
+ *        many cells (degenerate case)
+ *
+ * @lucene.experimental
+ */
+public class GeoPointInBBoxQuery extends Query {
+  protected final String field;
+  protected final double minLon;
+  protected final double minLat;
+  protected final double maxLon;
+  protected final double maxLat;
+  protected final TermEncoding termEncoding;
+
+  /**
+   * Constructs a query for all {@link org.apache.lucene.spatial.geopoint.document.GeoPointField} types that fall within a
+   * defined bounding box
+   */
+  public GeoPointInBBoxQuery(final String field, final double minLon, final double minLat, final double maxLon, final double maxLat) {
+    this(field, TermEncoding.PREFIX, minLon, minLat, maxLon, maxLat);
+  }
+
+  public GeoPointInBBoxQuery(final String field, final TermEncoding termEncoding, final double minLon, final double minLat, final double maxLon, final double maxLat) {
+    this.field = field;
+    this.minLon = minLon;
+    this.minLat = minLat;
+    this.maxLon = maxLon;
+    this.maxLat = maxLat;
+    this.termEncoding = termEncoding;
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) {
+    // short-circuit to match all if specifying the whole map
+    if (minLon == GeoUtils.MIN_LON_INCL && maxLon == GeoUtils.MAX_LON_INCL
+        && minLat == GeoUtils.MIN_LAT_INCL && maxLat == GeoUtils.MAX_LAT_INCL) {
+      // FieldValueQuery is valid since DocValues are *required* for GeoPointField
+      return new FieldValueQuery(field);
+    }
+
+    if (maxLon < minLon) {
+      BooleanQuery.Builder bqb = new BooleanQuery.Builder();
+
+      GeoPointInBBoxQueryImpl left = new GeoPointInBBoxQueryImpl(field, termEncoding, -180.0D, minLat, maxLon, maxLat);
+      bqb.add(new BooleanClause(left, BooleanClause.Occur.SHOULD));
+      GeoPointInBBoxQueryImpl right = new GeoPointInBBoxQueryImpl(field, termEncoding, minLon, minLat, 180.0D, maxLat);
+      bqb.add(new BooleanClause(right, BooleanClause.Occur.SHOULD));
+      return bqb.build();
+    }
+    return new GeoPointInBBoxQueryImpl(field, termEncoding, minLon, minLat, maxLon, maxLat);
+  }
+
+  @Override
+  public String toString(String field) {
+    final StringBuilder sb = new StringBuilder();
+    sb.append(getClass().getSimpleName());
+    sb.append(':');
+    if (!this.field.equals(field)) {
+      sb.append(" field=");
+      sb.append(this.field);
+      sb.append(':');
+    }
+    return sb.append(" Lower Left: [")
+        .append(minLon)
+        .append(',')
+        .append(minLat)
+        .append(']')
+        .append(" Upper Right: [")
+        .append(maxLon)
+        .append(',')
+        .append(maxLat)
+        .append("]")
+        .toString();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!(o instanceof GeoPointInBBoxQuery)) return false;
+    if (!super.equals(o)) return false;
+
+    GeoPointInBBoxQuery that = (GeoPointInBBoxQuery) 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;
+    if (!field.equals(that.field)) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    long temp;
+    result = 31 * result + field.hashCode();
+    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;
+  }
+
+  /** getter method for retrieving the field name */
+  public final String getField() {
+    return this.field;
+  }
+
+  /** getter method for retrieving the minimum longitude (in degrees) */
+  public final double getMinLon() {
+    return this.minLon;
+  }
+
+  /** getter method for retrieving the minimum latitude (in degrees) */
+  public final double getMinLat() {
+    return this.minLat;
+  }
+
+  /** getter method for retrieving the maximum longitude (in degrees) */
+  public final double getMaxLon() {
+    return this.maxLon;
+  }
+
+  /** getter method for retrieving the maximum latitude (in degrees) */
+  public final double getMaxLat() {
+    return this.maxLat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInBBoxQueryImpl.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInBBoxQueryImpl.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInBBoxQueryImpl.java
new file mode 100644
index 0000000..675b2a1
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInBBoxQueryImpl.java
@@ -0,0 +1,156 @@
+/*
+ * 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.geopoint.search;
+
+import org.apache.lucene.search.MultiTermQuery;
+import org.apache.lucene.util.SloppyMath;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding;
+import org.apache.lucene.spatial.util.GeoRelationUtils;
+
+/** Package private implementation for the public facing GeoPointInBBoxQuery delegate class.
+ *
+ *    @lucene.experimental
+ */
+class GeoPointInBBoxQueryImpl extends GeoPointMultiTermQuery {
+  /**
+   * 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 TermEncoding termEncoding, final double minLon, final double minLat, final double maxLon, final double maxLat) {
+    super(field, termEncoding, minLon, minLat, maxLon, maxLat);
+  }
+
+  @Override
+  public void setRewriteMethod(MultiTermQuery.RewriteMethod method) {
+    throw new UnsupportedOperationException("cannot change rewrite method");
+  }
+
+  @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);
+  }
+
+  @Override
+  protected CellComparator newCellComparator() {
+    return new GeoPointInBBoxCellComparator(this);
+  }
+
+  private final class GeoPointInBBoxCellComparator extends CellComparator {
+    GeoPointInBBoxCellComparator(GeoPointMultiTermQuery query) {
+      super(query);
+    }
+
+    /**
+     * 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, GeoPointInBBoxQueryImpl.this.minLon,
+          GeoPointInBBoxQueryImpl.this.minLat, GeoPointInBBoxQueryImpl.this.maxLon, GeoPointInBBoxQueryImpl.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, GeoPointInBBoxQueryImpl.this.minLon,
+          GeoPointInBBoxQueryImpl.this.minLat, GeoPointInBBoxQueryImpl.this.maxLon, GeoPointInBBoxQueryImpl.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/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQuery.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQuery.java
new file mode 100644
index 0000000..6b0d4dd
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQuery.java
@@ -0,0 +1,153 @@
+/*
+ * 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.geopoint.search;
+
+import java.util.Arrays;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding;
+import org.apache.lucene.spatial.util.GeoEncodingUtils;
+import org.apache.lucene.spatial.util.GeoRect;
+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 GeoEncodingUtils#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 GeoPointInBBoxQuery {
+  // polygon position arrays - this avoids the use of any objects or
+  // or geo library dependencies
+  protected final double[] x;
+  protected final double[] y;
+
+  public GeoPointInPolygonQuery(final String field, final double[] polyLons, final double[] polyLats) {
+    this(field, TermEncoding.PREFIX, GeoUtils.polyToBBox(polyLons, polyLats), polyLons, polyLats);
+  }
+
+  /**
+   * Constructs a new GeoPolygonQuery that will match encoded {@link org.apache.lucene.spatial.geopoint.document.GeoPointField} terms
+   * that fall within or on the boundary of the polygon defined by the input parameters.
+   */
+  public GeoPointInPolygonQuery(final String field, final TermEncoding termEncoding, final double[] polyLons, final double[] polyLats) {
+    this(field, termEncoding, GeoUtils.polyToBBox(polyLons, polyLats), polyLons, polyLats);
+  }
+
+  /** Common constructor, used only internally. */
+  private GeoPointInPolygonQuery(final String field, TermEncoding termEncoding, GeoRect bbox, final double[] polyLons, final double[] polyLats) {
+    super(field, termEncoding, 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;
+  }
+
+  /** throw exception if trying to change rewrite method */
+  @Override
+  public Query rewrite(IndexReader reader) {
+    return new GeoPointInPolygonQueryImpl(field, termEncoding, this, this.minLon, this.minLat, this.maxLon, this.maxLat);
+  }
+
+  @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();
+  }
+
+  /**
+   * 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;
+  }
+}


[2/3] lucene-solr git commit: LUCENE-6997: refactor GeoPointField and query classes from lucene.spatial to lucene.spatial.geopoint package

Posted by nk...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQueryImpl.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQueryImpl.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQueryImpl.java
new file mode 100644
index 0000000..35a3950
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointInPolygonQueryImpl.java
@@ -0,0 +1,103 @@
+/*
+ * 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.geopoint.search;
+
+import org.apache.lucene.search.MultiTermQuery;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding;
+import org.apache.lucene.spatial.util.GeoRelationUtils;
+
+/** Package private implementation for the public facing GeoPointInPolygonQuery delegate class.
+ *
+ *    @lucene.experimental
+ */
+final class GeoPointInPolygonQueryImpl extends GeoPointInBBoxQueryImpl {
+  private final GeoPointInPolygonQuery polygonQuery;
+
+  GeoPointInPolygonQueryImpl(final String field, final TermEncoding termEncoding, final GeoPointInPolygonQuery q,
+                             final double minLon, final double minLat, final double maxLon, final double maxLat) {
+    super(field, termEncoding, minLon, minLat, maxLon, maxLat);
+    polygonQuery = q;
+  }
+
+  @Override
+  public void setRewriteMethod(MultiTermQuery.RewriteMethod method) {
+    throw new UnsupportedOperationException("cannot change rewrite method");
+  }
+
+  @Override
+  protected CellComparator newCellComparator() {
+    return new GeoPolygonCellComparator(this);
+  }
+
+  /**
+   * Custom {@code org.apache.lucene.spatial.geopoint.search.GeoPointMultiTermQuery.CellComparator} that computes morton hash
+   * ranges based on the defined edges of the provided polygon.
+   */
+  private final class GeoPolygonCellComparator extends CellComparator {
+    GeoPolygonCellComparator(GeoPointMultiTermQuery query) {
+      super(query);
+    }
+
+    @Override
+    protected boolean cellCrosses(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+      return GeoRelationUtils.rectCrossesPolyApprox(minLon, minLat, maxLon, maxLat, polygonQuery.x, polygonQuery.y,
+          polygonQuery.minLon, polygonQuery.minLat, polygonQuery.maxLon, polygonQuery.maxLat);
+    }
+
+    @Override
+    protected boolean cellWithin(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+      return GeoRelationUtils.rectWithinPolyApprox(minLon, minLat, maxLon, maxLat, polygonQuery.x, polygonQuery.y,
+          polygonQuery.minLon, polygonQuery.minLat, polygonQuery.maxLon, polygonQuery.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 org.apache.lucene.spatial.geopoint.search.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(polygonQuery.x, polygonQuery.y, lat, lon);
+    }
+  }
+
+  @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;
+
+    GeoPointInPolygonQueryImpl that = (GeoPointInPolygonQueryImpl) o;
+
+    return !(polygonQuery != null ? !polygonQuery.equals(that.polygonQuery) : that.polygonQuery != null);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + (polygonQuery != null ? polygonQuery.hashCode() : 0);
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointMultiTermQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointMultiTermQuery.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointMultiTermQuery.java
new file mode 100644
index 0000000..b83d680
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointMultiTermQuery.java
@@ -0,0 +1,166 @@
+package org.apache.lucene.spatial.geopoint.search;
+
+/*
+ * 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 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.util.AttributeSource;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding;
+import org.apache.lucene.spatial.util.GeoEncodingUtils;
+import org.apache.lucene.spatial.util.GeoRelationUtils;
+import org.apache.lucene.spatial.util.GeoUtils;
+import org.apache.lucene.util.SloppyMath;
+
+/**
+ * TermQuery for GeoPointField for overriding {@link org.apache.lucene.search.MultiTermQuery} methods specific to
+ * Geospatial operations
+ *
+ * @lucene.experimental
+ */
+abstract class GeoPointMultiTermQuery extends MultiTermQuery {
+  // simple bounding box optimization - no objects used to avoid dependencies
+  protected final double minLon;
+  protected final double minLat;
+  protected final double maxLon;
+  protected final double maxLat;
+  protected final short maxShift;
+  protected final TermEncoding termEncoding;
+  protected final CellComparator cellComparator;
+
+  /**
+   * Constructs a query matching terms that cannot be represented with a single
+   * Term.
+   */
+  public GeoPointMultiTermQuery(String field, final TermEncoding termEncoding, 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);
+    }
+
+    final long minHash = GeoEncodingUtils.mortonHash(minLon, minLat);
+    final long maxHash = GeoEncodingUtils.mortonHash(maxLon, maxLat);
+    this.minLon = GeoEncodingUtils.mortonUnhashLon(minHash);
+    this.minLat = GeoEncodingUtils.mortonUnhashLat(minHash);
+    this.maxLon = GeoEncodingUtils.mortonUnhashLon(maxHash);
+    this.maxLat = GeoEncodingUtils.mortonUnhashLat(maxHash);
+
+    this.maxShift = computeMaxShift();
+    this.termEncoding = termEncoding;
+    this.cellComparator = newCellComparator();
+
+    this.rewriteMethod = GEO_CONSTANT_SCORE_REWRITE;
+  }
+
+  public static final RewriteMethod GEO_CONSTANT_SCORE_REWRITE = new RewriteMethod() {
+    @Override
+    public Query rewrite(IndexReader reader, MultiTermQuery query) {
+      return new GeoPointTermQueryConstantScoreWrapper<>((GeoPointMultiTermQuery)query);
+    }
+  };
+
+  @Override @SuppressWarnings("unchecked")
+  protected TermsEnum getTermsEnum(final Terms terms, AttributeSource atts) throws IOException {
+    return GeoPointTermsEnum.newInstance(terms.iterator(), this);
+  }
+
+  /**
+   * Computes the maximum shift based on the diagonal distance of the bounding box
+   */
+  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)
+    final short shiftFactor;
+
+    // compute diagonal distance
+    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);
+  }
+
+  /**
+   * Abstract method to construct the class that handles all geo point relations
+   * (e.g., GeoPointInPolygon)
+   */
+  abstract protected CellComparator newCellComparator();
+
+  /**
+   * Base class for all geo point relation comparators
+   */
+  static abstract class CellComparator {
+    protected final GeoPointMultiTermQuery geoPointQuery;
+
+    CellComparator(GeoPointMultiTermQuery query) {
+      this.geoPointQuery = query;
+    }
+
+    /**
+     * 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, geoPointQuery.minLon, geoPointQuery.minLat,
+          geoPointQuery.maxLon, geoPointQuery.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(geoPointQuery.minLon, geoPointQuery.minLat, geoPointQuery.maxLon,
+          geoPointQuery.maxLat, minLon, minLat, maxLon, maxLat);
+    }
+
+    /**
+     * Determine whether the quad-cell crosses the shape
+     */
+    abstract protected boolean cellCrosses(final double minLon, final double minLat, final double maxLon, final double maxLat);
+
+    /**
+     * Determine whether quad-cell is within the shape
+     */
+    abstract protected 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}
+     */
+    abstract protected boolean cellIntersectsShape(final double minLon, final double minLat, final double maxLon, final double maxLat);
+
+    abstract protected boolean postFilter(final double lon, final double lat);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointNumericTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointNumericTermsEnum.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointNumericTermsEnum.java
new file mode 100644
index 0000000..120df7d
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointNumericTermsEnum.java
@@ -0,0 +1,161 @@
+package org.apache.lucene.spatial.geopoint.search;
+
+/*
+ * 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.Collections;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField;
+import org.apache.lucene.spatial.util.GeoEncodingUtils;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.LegacyNumericUtils;
+
+/**
+ * Decomposes a given {@link GeoPointMultiTermQuery} into a set of terms that represent the query criteria using
+ * {@link org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding#NUMERIC} method defined by
+ * {@link org.apache.lucene.analysis.LegacyNumericTokenStream}. The terms are then enumerated by the
+ * {@link GeoPointTermQueryConstantScoreWrapper} and all docs whose GeoPoint fields match the prefix terms or
+ * pass the {@link GeoPointMultiTermQuery.CellComparator#postFilter} criteria are returned in the resulting DocIdSet.
+ *
+ *  @lucene.experimental
+ */
+@Deprecated
+final class GeoPointNumericTermsEnum extends GeoPointTermsEnum {
+  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)
+  private final short DETAIL_LEVEL;
+
+  GeoPointNumericTermsEnum(final TermsEnum tenum, final GeoPointMultiTermQuery query) {
+    super(tenum, query);
+    DETAIL_LEVEL = (short)(((GeoEncodingUtils.BITS<<1)-this.maxShift)/2);
+    computeRange(0L, (short) (((GeoEncodingUtils.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 = GeoEncodingUtils.mortonUnhashLon(start);
+    final double minLat = GeoEncodingUtils.mortonUnhashLat(start);
+    final double maxLon = GeoEncodingUtils.mortonUnhashLon(end);
+    final double maxLat = GeoEncodingUtils.mortonUnhashLat(end);
+
+    final short level = (short)((GeoEncodingUtils.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 && relationImpl.cellWithin(minLon, minLat, maxLon, maxLat);
+    if (within || (level == DETAIL_LEVEL && relationImpl.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 && relationImpl.cellIntersectsMBR(minLon, minLat, maxLon, maxLat)) {
+      computeRange(start, (short) (res - 1));
+    }
+  }
+
+  @Override
+  protected final BytesRef peek() {
+    rangeBounds.get(0).fillBytesRef(this.nextSubRangeBRB);
+    return nextSubRangeBRB.get();
+  }
+
+  @Override
+  protected void nextRange() {
+    currentRange = rangeBounds.remove(0);
+    super.nextRange();
+  }
+
+  @Override
+  protected final BytesRef nextSeekTerm(BytesRef term) {
+    while (hasNext()) {
+      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) > 0) {
+        nextRange();
+        if (!rangeBounds.isEmpty()) {
+          continue;
+        }
+      }
+      // never seek backwards, so use current term if lower bound is smaller
+      return (term != null && term.compareTo(currentCell) > 0) ? term : currentCell;
+    }
+
+    // no more sub-range enums available
+    assert rangeBounds.isEmpty();
+    return null;
+  }
+
+  @Override
+  protected final boolean hasNext() {
+    return rangeBounds.isEmpty() == false;
+  }
+
+  /**
+   * Internal class to represent a range along the space filling curve
+   */
+  protected final class Range extends BaseRange {
+    Range(final long lower, final short shift, boolean boundary) {
+      super(lower, shift, boundary);
+    }
+
+    /**
+     * 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.
+     */
+    @Override
+    protected void fillBytesRef(BytesRefBuilder result) {
+      assert result != null;
+      LegacyNumericUtils.longToPrefixCoded(start, shift, result);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointPrefixTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointPrefixTermsEnum.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointPrefixTermsEnum.java
new file mode 100644
index 0000000..fbaf68e
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointPrefixTermsEnum.java
@@ -0,0 +1,237 @@
+package org.apache.lucene.spatial.geopoint.search;
+
+/*
+ * 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.index.TermsEnum;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.spatial.geopoint.document.GeoPointField;
+import org.apache.lucene.spatial.util.GeoEncodingUtils;
+
+import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonHash;
+import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLat;
+import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLon;
+import static org.apache.lucene.spatial.util.GeoEncodingUtils.geoCodedToPrefixCoded;
+import static org.apache.lucene.spatial.util.GeoEncodingUtils.prefixCodedToGeoCoded;
+import static org.apache.lucene.spatial.util.GeoEncodingUtils.getPrefixCodedShift;
+
+/**
+ * Decomposes a given {@link GeoPointMultiTermQuery} into a set of terms that represent the query criteria using
+ * {@link org.apache.lucene.spatial.geopoint.document.GeoPointField.TermEncoding#PREFIX} method defined by
+ * {@link GeoPointField}. The terms are then enumerated by the
+ * {@link GeoPointTermQueryConstantScoreWrapper} and all docs whose GeoPoint fields match the prefix terms or pass
+ * the {@link GeoPointMultiTermQuery.CellComparator#postFilter} criteria are returned in the
+ * resulting DocIdSet.
+ *
+ *  @lucene.experimental
+ */
+final class GeoPointPrefixTermsEnum extends GeoPointTermsEnum {
+  private final long start;
+
+  private short shift;
+
+  // current range as long
+  private long currStart;
+  private long currEnd;
+
+  private final Range nextRange = new Range(-1, shift, true);
+
+  private boolean hasNext = false;
+
+  private boolean withinOnly = false;
+  private long lastWithin;
+
+  public GeoPointPrefixTermsEnum(final TermsEnum tenum, final GeoPointMultiTermQuery query) {
+    super(tenum, query);
+    this.start = mortonHash(query.minLon, query.minLat);
+    this.currentRange = new Range(0, shift, true);
+    // start shift at maxShift value (from computeMaxShift)
+    this.shift = maxShift;
+    final long mask = (1L << shift) - 1;
+    this.currStart = start & ~mask;
+    this.currEnd = currStart | mask;
+  }
+
+  private boolean within(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+    return relationImpl.cellWithin(minLon, minLat, maxLon, maxLat);
+  }
+
+  private boolean boundary(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+    return shift == maxShift && relationImpl.cellIntersectsShape(minLon, minLat, maxLon, maxLat);
+  }
+
+  private boolean nextWithin() {
+    if (withinOnly == false) {
+      return false;
+    }
+    currStart += (1L << shift);
+    setNextRange(false);
+    currentRange.set(nextRange);
+    hasNext = true;
+
+    withinOnly = lastWithin != currStart;
+    if (withinOnly == false) advanceVariables();
+    return true;
+  }
+
+  private void nextRelation() {
+    double minLon = mortonUnhashLon(currStart);
+    double minLat = mortonUnhashLat(currStart);
+    double maxLon;
+    double maxLat;
+    boolean isWithin;
+    do {
+      maxLon = mortonUnhashLon(currEnd);
+      maxLat = mortonUnhashLat(currEnd);
+
+      // within or a boundary
+      if ((isWithin = within(minLon, minLat, maxLon, maxLat) == true) || boundary(minLon, minLat, maxLon, maxLat) == true) {
+        final int m;
+        if (isWithin == false || (m = shift % GeoPointField.PRECISION_STEP) == 0) {
+          setNextRange(isWithin == false);
+          advanceVariables();
+          break;
+        } else if (shift < 54) {
+          withinOnly = true;
+          shift = (short)(shift - m);
+          lastWithin = currEnd & ~((1L << shift) - 1);
+          setNextRange(false);
+          break;
+        }
+      }
+
+      // within cell but not at a depth factor of PRECISION_STEP
+      if (isWithin == true || (relationImpl.cellIntersectsMBR(minLon, minLat, maxLon , maxLat) == true && shift != maxShift)) {
+        // descend: currStart need not change since shift handles end of range
+        currEnd = currStart | (1L<<--shift) - 1;
+      } else {
+        advanceVariables();
+        minLon = mortonUnhashLon(currStart);
+        minLat = mortonUnhashLat(currStart);
+      }
+    } while(shift < 63);
+  }
+
+  private void setNextRange(final boolean boundary) {
+    nextRange.start = currStart;
+    nextRange.shift = shift;
+    nextRange.boundary = boundary;
+  }
+
+  private void advanceVariables() {
+    /** set next variables */
+    long shiftMask = 1L << shift;
+    // pop-up if shift bit is set
+    while ( (currStart & shiftMask) == shiftMask) {
+      shiftMask = 1L << ++shift;
+    }
+    final long shiftMOne = shiftMask - 1;
+    currStart = currStart & ~shiftMOne | shiftMask;
+    currEnd = currStart | shiftMOne;
+  }
+
+  @Override
+  protected final BytesRef peek() {
+    nextRange.fillBytesRef(nextSubRangeBRB);
+    return super.peek();
+  }
+
+  protected void seek(long term, short res) {
+    if (term < currStart && res < maxShift) {
+      throw new IllegalArgumentException("trying to seek backwards");
+    } else if (term == currStart) {
+      return;
+    }
+    shift = res;
+    currStart = term;
+    currEnd = currStart | ((1L<<shift)-1);
+    withinOnly = false;
+  }
+
+  @Override
+  protected void nextRange() {
+    hasNext = false;
+    super.nextRange();
+  }
+
+  @Override
+  protected final boolean hasNext() {
+    if (hasNext == true || nextWithin()) {
+      return true;
+    }
+    nextRelation();
+    if (currentRange.compareTo(nextRange) != 0) {
+      currentRange.set(nextRange);
+      return (hasNext = true);
+    }
+    return false;
+  }
+
+  @Override
+  protected final BytesRef nextSeekTerm(BytesRef term) {
+    while (hasNext()) {
+      nextRange();
+      if (term == null) {
+        return currentCell;
+      }
+
+      final int comparison = term.compareTo(currentCell);
+      if (comparison > 0) {
+        seek(GeoEncodingUtils.prefixCodedToGeoCoded(term), (short)(64-GeoEncodingUtils.getPrefixCodedShift(term)));
+        continue;
+      }
+      return currentCell;
+    }
+
+    // no more sub-range enums available
+    return null;
+  }
+
+  @Override
+  protected AcceptStatus accept(BytesRef term) {
+    // range < term or range is null
+    while (currentCell == null || term.compareTo(currentCell) > 0) {
+      // no more ranges, be gone
+      if (hasNext() == false) {
+        return AcceptStatus.END;
+      }
+
+      // peek next range, if the range > term then seek
+      final int peekCompare = term.compareTo(peek());
+      if (peekCompare < 0) {
+        return AcceptStatus.NO_AND_SEEK;
+      } else if (peekCompare > 0) {
+        seek(prefixCodedToGeoCoded(term), (short)(64 - getPrefixCodedShift(term)));
+      }
+      nextRange();
+    }
+    return AcceptStatus.YES;
+  }
+
+  protected final class Range extends BaseRange {
+    public Range(final long start, final short res, final boolean boundary) {
+      super(start, res, boundary);
+    }
+
+    @Override
+    protected void fillBytesRef(BytesRefBuilder result) {
+      assert result != null;
+      geoCodedToPrefixCoded(start, shift, result);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
new file mode 100644
index 0000000..c8d3e55
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermQueryConstantScoreWrapper.java
@@ -0,0 +1,153 @@
+/*
+ * 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.geopoint.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.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.TwoPhaseIterator;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.SparseFixedBitSet;
+
+import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLat;
+import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonUnhashLon;
+
+
+/**
+ * Custom ConstantScoreWrapper for {@code GeoPointMultiTermQuery} that cuts over to DocValues
+ * for post filtering boundary ranges. Multi-valued GeoPoint documents are supported.
+ *
+ * @lucene.experimental
+ */
+final class GeoPointTermQueryConstantScoreWrapper <Q extends GeoPointMultiTermQuery> 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) {
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        final Terms terms = context.reader().terms(query.getField());
+        if (terms == null) {
+          return null;
+        }
+
+        final GeoPointTermsEnum termsEnum = (GeoPointTermsEnum)(query.getTermsEnum(terms, null));
+        assert termsEnum != null;
+
+        LeafReader reader = context.reader();
+        // approximation (postfiltering has not yet been applied)
+        DocIdSetBuilder builder = new DocIdSetBuilder(reader.maxDoc());
+        // subset of documents that need no postfiltering, this is purely an optimization
+        final BitSet preApproved;
+        // dumb heuristic: if the field is really sparse, use a sparse impl
+        if (terms.getDocCount() * 100L < reader.maxDoc()) {
+          preApproved = new SparseFixedBitSet(reader.maxDoc());
+        } else {
+          preApproved = new FixedBitSet(reader.maxDoc());
+        }
+        PostingsEnum docs = null;
+
+        while (termsEnum.next() != null) {
+          docs = termsEnum.postings(docs, PostingsEnum.NONE);
+          // boundary terms need post filtering
+          if (termsEnum.boundaryTerm()) {
+            builder.add(docs);
+          } else {
+            int docId;
+            while ((docId = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+              builder.add(docId);
+              preApproved.set(docId);
+            }
+          }
+        }
+
+        DocIdSet set = builder.build();
+        final DocIdSetIterator disi = set.iterator();
+        if (disi == null) {
+          return null;
+        }
+
+        // return two-phase iterator using docvalues to postfilter candidates
+        SortedNumericDocValues sdv = reader.getSortedNumericDocValues(query.getField());
+        TwoPhaseIterator iterator = new TwoPhaseIterator(disi) {
+          @Override
+          public boolean matches() throws IOException {
+            int docId = disi.docID();
+            if (preApproved.get(docId)) {
+              return true;
+            } else {
+              sdv.setDocument(docId);
+              int count = sdv.count();
+              for (int i = 0; i < count; i++) {
+                long hash = sdv.valueAt(i);
+                if (termsEnum.postFilter(mortonUnhashLon(hash), mortonUnhashLat(hash))) {
+                  return true;
+                }
+              }
+              return false;
+            }
+          }
+
+          @Override
+          public float matchCost() {
+            return 20; // TODO: make this fancier
+          }
+        };
+        return new ConstantScoreScorer(this, score(), iterator);
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermsEnum.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermsEnum.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermsEnum.java
new file mode 100644
index 0000000..9f9e251
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/GeoPointTermsEnum.java
@@ -0,0 +1,140 @@
+/*
+ * 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.geopoint.search;
+
+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.geopoint.document.GeoPointField.TermEncoding;
+import org.apache.lucene.spatial.geopoint.search.GeoPointMultiTermQuery.CellComparator;
+
+/**
+ * Base class for {@link GeoPointNumericTermsEnum} and {@link GeoPointPrefixTermsEnum} which compares
+ * candidate GeoPointField encoded terms against terms matching the defined query criteria.
+ *
+ *  @lucene.experimental
+ */
+abstract class GeoPointTermsEnum extends FilteredTermsEnum {
+  protected final short maxShift;
+
+  protected BaseRange currentRange;
+  protected BytesRef currentCell;
+  protected final BytesRefBuilder currentCellBRB = new BytesRefBuilder();
+  protected final BytesRefBuilder nextSubRangeBRB = new BytesRefBuilder();
+
+  protected final CellComparator relationImpl;
+
+  GeoPointTermsEnum(final TermsEnum tenum, final GeoPointMultiTermQuery query) {
+    super(tenum);
+    this.maxShift = query.maxShift;
+    this.relationImpl = query.cellComparator;
+  }
+
+  static GeoPointTermsEnum newInstance(final TermsEnum terms, final GeoPointMultiTermQuery query) {
+    if (query.termEncoding == TermEncoding.PREFIX) {
+      return new GeoPointPrefixTermsEnum(terms, query);
+    } else if (query.termEncoding == TermEncoding.NUMERIC) {
+      return new GeoPointNumericTermsEnum(terms, query);
+    }
+    throw new IllegalArgumentException("Invalid GeoPoint TermEncoding " + query.termEncoding);
+  }
+
+  public boolean boundaryTerm() {
+    if (currentCell == null) {
+      throw new IllegalStateException("GeoPointTermsEnum empty or not initialized");
+    }
+    return currentRange.boundary;
+  }
+
+  protected BytesRef peek() {
+    return nextSubRangeBRB.get();
+  }
+
+  abstract protected boolean hasNext();
+
+  protected void nextRange() {
+    currentRange.fillBytesRef(currentCellBRB);
+    currentCell = currentCellBRB.get();
+  }
+
+  /**
+   * 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) > 0) {
+      if (hasNext() == false) {
+        return AcceptStatus.END;
+      }
+      // peek next sub-range, only seek if the current term is smaller than next lower bound
+      if (term.compareTo(peek()) < 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 boolean postFilter(final double lon, final double lat) {
+    return relationImpl.postFilter(lon, lat);
+  }
+
+  /**
+   * Internal class to represent a range along the space filling curve
+   */
+  abstract class BaseRange implements Comparable<BaseRange> {
+    protected short shift;
+    protected long start;
+    protected boolean boundary;
+
+    BaseRange(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.
+     */
+    abstract protected void fillBytesRef(BytesRefBuilder result);
+
+    @Override
+    public int compareTo(BaseRange other) {
+      final int result = Short.compare(this.shift, other.shift);
+      if (result == 0) {
+        return Long.compare(this.start, other.start);
+      }
+      return result;
+    }
+
+    protected void set(BaseRange other) {
+      this.start = other.start;
+      this.shift = other.shift;
+      this.boundary = other.boundary;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/package-info.java b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/search/package-info.java
new file mode 100644
index 0000000..0d923df
--- /dev/null
+++ b/lucene/spatial/src/java/org/apache/lucene/spatial/geopoint/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.geopoint.search;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoBoundingBox.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoBoundingBox.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoBoundingBox.java
deleted file mode 100644
index f26f8b1..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoBoundingBox.java
+++ /dev/null
@@ -1,53 +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.search;
-
-import org.apache.lucene.spatial.util.GeoUtils;
-
-/** NOTE: package private; just used so {@link GeoPointInPolygonQuery} can communicate its bounding box to {@link GeoPointInBBoxQuery}. */
-class GeoBoundingBox {
-  /** minimum longitude value (in degrees) */
-  public final double minLon;
-  /** minimum latitude value (in degrees) */
-  public final double maxLon;
-  /** maximum longitude value (in degrees) */
-  public final double minLat;
-  /** maximum latitude value (in degrees) */
-  public final double maxLat;
-
-  /**
-   * Constructs a bounding box by first validating the provided latitude and longitude coordinates
-   */
-  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 " + minLon);
-    }
-    if (GeoUtils.isValidLat(minLat) == false) {
-      throw new IllegalArgumentException("invalid minLat " + minLat);
-    }
-    if (GeoUtils.isValidLat(maxLat) == false) {
-      throw new IllegalArgumentException("invalid maxLat " + minLat);
-    }
-    this.minLon = minLon;
-    this.maxLon = maxLon;
-    this.minLat = minLat;
-    this.maxLat = maxLat;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointDistanceQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointDistanceQuery.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointDistanceQuery.java
deleted file mode 100644
index 24b7f8f..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointDistanceQuery.java
+++ /dev/null
@@ -1,189 +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.search;
-
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.spatial.document.GeoPointField.TermEncoding;
-import org.apache.lucene.spatial.util.GeoDistanceUtils;
-import org.apache.lucene.spatial.util.GeoRect;
-import org.apache.lucene.spatial.util.GeoUtils;
-
-/** Implements a simple point distance query on a GeoPoint field. This is based on
- * {@link GeoPointInBBoxQuery} and is implemented using a two phase approach. First,
- * like {@code GeoPointInBBoxQueryImpl} candidate terms are queried using the numeric ranges based on
- * the morton codes of the min and max lat/lon pairs that intersect the boundary of the point-radius
- * circle. Terms
- * passing this initial filter are then passed to a secondary {@code postFilter} method that verifies whether the
- * decoded lat/lon point fall within the specified query distance (see {@link org.apache.lucene.util.SloppyMath#haversin}.
- * All morton value comparisons are subject to the same precision tolerance defined in
- * {@value org.apache.lucene.spatial.util.GeoEncodingUtils#TOLERANCE} and distance comparisons are subject to the accuracy of the
- * haversine formula (from R.W. Sinnott, "Virtues of the Haversine", Sky and Telescope, vol. 68, no. 2, 1984, p. 159)
- *
- * <p>Note: This query currently uses haversine which is a sloppy distance calculation (see above reference). For large
- * queries one can expect upwards of 400m error. Vincenty shrinks this to ~40m error but pays a penalty for computing
- * using the spheroid
- *
- * @lucene.experimental */
-public class GeoPointDistanceQuery extends GeoPointInBBoxQuery {
-  /** longitude value (in degrees) for query location */
-  protected final double centerLon;
-  /** latitude value (in degrees) for query location */
-  protected final double centerLat;
-  /** distance (in meters) from lon, lat center location */
-  protected final double radiusMeters;
-
-  /**
-   * Constructs a Query for all {@link org.apache.lucene.spatial.document.GeoPointField} types within a
-   * distance (in meters) from a given point
-   **/
-  public GeoPointDistanceQuery(final String field, final double centerLon, final double centerLat, final double radiusMeters) {
-    this(field, TermEncoding.PREFIX, centerLon, centerLat, radiusMeters);
-  }
-
-  public GeoPointDistanceQuery(final String field, final TermEncoding termEncoding, final double centerLon, final double centerLat, final double radiusMeters) {
-    this(field, termEncoding, GeoUtils.circleToBBox(centerLon, centerLat, radiusMeters), centerLon, centerLat, radiusMeters);
-  }
-
-  private GeoPointDistanceQuery(final String field, final TermEncoding termEncoding, final GeoRect bbox, final double centerLon,
-                                final double centerLat, final double radiusMeters) {
-    super(field, termEncoding, bbox.minLon, bbox.minLat, bbox.maxLon, bbox.maxLat);
-    {
-      // check longitudinal overlap (restrict distance to maximum longitudinal radius)
-      // todo this restriction technically shouldn't be needed,
-      // its only purpose is to ensure the bounding box doesn't self overlap.
-      final double maxRadius = GeoDistanceUtils.maxRadialDistanceMeters(centerLon, centerLat);
-      if (radiusMeters > maxRadius) {
-        throw new IllegalArgumentException("radiusMeters " + radiusMeters + " exceeds maxRadius [" + maxRadius
-            + "] at location [" + centerLon + " " + centerLat + "]");
-      }
-    }
-
-    if (GeoUtils.isValidLon(centerLon) == false) {
-      throw new IllegalArgumentException("invalid centerLon " + centerLon);
-    }
-
-    if (GeoUtils.isValidLat(centerLat) == false) {
-      throw new IllegalArgumentException("invalid centerLat " + centerLat);
-    }
-
-    if (radiusMeters <= 0.0) {
-      throw new IllegalArgumentException("invalid radiusMeters " + radiusMeters);
-    }
-
-    this.centerLon = centerLon;
-    this.centerLat = centerLat;
-    this.radiusMeters = radiusMeters;
-  }
-
-  @Override
-  public Query rewrite(IndexReader reader) {
-    // query crosses dateline; split into left and right queries
-    if (maxLon < minLon) {
-      BooleanQuery.Builder bqb = new BooleanQuery.Builder();
-
-      // unwrap the longitude iff outside the specified min/max lon range
-      double unwrappedLon = centerLon;
-      if (unwrappedLon > maxLon) {
-        // unwrap left
-        unwrappedLon += -360.0D;
-      }
-      GeoPointDistanceQueryImpl left = new GeoPointDistanceQueryImpl(field, termEncoding, this, unwrappedLon,
-          new GeoRect(GeoUtils.MIN_LON_INCL, maxLon, minLat, maxLat));
-      bqb.add(new BooleanClause(left, BooleanClause.Occur.SHOULD));
-
-      if (unwrappedLon < maxLon) {
-        // unwrap right
-        unwrappedLon += 360.0D;
-      }
-      GeoPointDistanceQueryImpl right = new GeoPointDistanceQueryImpl(field, termEncoding, this, unwrappedLon,
-          new GeoRect(minLon, GeoUtils.MAX_LON_INCL, minLat, maxLat));
-      bqb.add(new BooleanClause(right, BooleanClause.Occur.SHOULD));
-
-      return bqb.build();
-    }
-    return new GeoPointDistanceQueryImpl(field, termEncoding, this, centerLon,
-        new GeoRect(this.minLon, this.maxLon, this.minLat, this.maxLat));
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (!(o instanceof GeoPointDistanceQuery)) return false;
-    if (!super.equals(o)) return false;
-
-    GeoPointDistanceQuery that = (GeoPointDistanceQuery) o;
-
-    if (Double.compare(that.centerLat, centerLat) != 0) return false;
-    if (Double.compare(that.centerLon, centerLon) != 0) return false;
-    if (Double.compare(that.radiusMeters, radiusMeters) != 0) return false;
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int result = super.hashCode();
-    long temp;
-    temp = Double.doubleToLongBits(centerLon);
-    result = 31 * result + (int) (temp ^ (temp >>> 32));
-    temp = Double.doubleToLongBits(centerLat);
-    result = 31 * result + (int) (temp ^ (temp >>> 32));
-    temp = Double.doubleToLongBits(radiusMeters);
-    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 (!this.field.equals(field)) {
-      sb.append(" field=");
-      sb.append(this.field);
-      sb.append(':');
-    }
-    return sb.append( " Center: [")
-        .append(centerLon)
-        .append(',')
-        .append(centerLat)
-        .append(']')
-        .append(" Distance: ")
-        .append(radiusMeters)
-        .append(" meters")
-        .append("]")
-        .toString();
-  }
-
-  /** getter method for center longitude value */
-  public double getCenterLon() {
-    return this.centerLon;
-  }
-
-  /** getter method for center latitude value */
-  public double getCenterLat() {
-    return this.centerLat;
-  }
-
-  /** getter method for distance value (in meters) */
-  public double getRadiusMeters() {
-    return this.radiusMeters;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointDistanceQueryImpl.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointDistanceQueryImpl.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointDistanceQueryImpl.java
deleted file mode 100644
index 39fc696..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointDistanceQueryImpl.java
+++ /dev/null
@@ -1,107 +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.search;
-
-import org.apache.lucene.search.MultiTermQuery;
-import org.apache.lucene.spatial.document.GeoPointField.TermEncoding;
-import org.apache.lucene.spatial.util.GeoRect;
-import org.apache.lucene.spatial.util.GeoRelationUtils;
-import org.apache.lucene.util.SloppyMath;
-
-/** Package private implementation for the public facing GeoPointDistanceQuery delegate class.
- *
- *    @lucene.experimental
- */
-final class GeoPointDistanceQueryImpl extends GeoPointInBBoxQueryImpl {
-  private final GeoPointDistanceQuery distanceQuery;
-  private final double centerLon;
-
-  GeoPointDistanceQueryImpl(final String field, final TermEncoding termEncoding, final GeoPointDistanceQuery q,
-                            final double centerLonUnwrapped, final GeoRect bbox) {
-    super(field, termEncoding, bbox.minLon, bbox.minLat, bbox.maxLon, bbox.maxLat);
-    distanceQuery = q;
-    centerLon = centerLonUnwrapped;
-  }
-
-  @Override
-  public void setRewriteMethod(MultiTermQuery.RewriteMethod method) {
-    throw new UnsupportedOperationException("cannot change rewrite method");
-  }
-
-  @Override
-  protected CellComparator newCellComparator() {
-    return new GeoPointRadiusCellComparator(this);
-  }
-
-  private final class GeoPointRadiusCellComparator extends CellComparator {
-    GeoPointRadiusCellComparator(GeoPointDistanceQueryImpl query) {
-      super(query);
-    }
-
-    @Override
-    protected boolean cellCrosses(final double minLon, final double minLat, final double maxLon, final double maxLat) {
-      return GeoRelationUtils.rectCrossesCircle(minLon, minLat, maxLon, maxLat,
-          centerLon, distanceQuery.centerLat, distanceQuery.radiusMeters, true);
-    }
-
-    @Override
-    protected boolean cellWithin(final double minLon, final double minLat, final double maxLon, final double maxLat) {
-      return GeoRelationUtils.rectWithinCircle(minLon, minLat, maxLon, maxLat,
-          centerLon, distanceQuery.centerLat, distanceQuery.radiusMeters, true);
-    }
-
-    @Override
-    protected boolean cellIntersectsShape(final double minLon, final double minLat, final double maxLon, final double maxLat) {
-      return cellCrosses(minLon, minLat, maxLon, maxLat);
-    }
-
-    /**
-     * The two-phase query approach. The parent {@link GeoPointTermsEnum} class matches
-     * encoded terms that fall within the minimum bounding box of the point-radius circle. Those documents that pass
-     * the initial bounding box filter are then post filter compared to the provided distance using the
-     * {@link org.apache.lucene.util.SloppyMath#haversin} method.
-     */
-    @Override
-    protected boolean postFilter(final double lon, final double lat) {
-      return (SloppyMath.haversin(distanceQuery.centerLat, centerLon, lat, lon) * 1000.0 <= distanceQuery.radiusMeters);
-    }
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (!(o instanceof GeoPointDistanceQueryImpl)) return false;
-    if (!super.equals(o)) return false;
-
-    GeoPointDistanceQueryImpl that = (GeoPointDistanceQueryImpl) o;
-
-    if (!distanceQuery.equals(that.distanceQuery)) return false;
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int result = super.hashCode();
-    result = 31 * result + distanceQuery.hashCode();
-    return result;
-  }
-
-  public double getRadiusMeters() {
-    return distanceQuery.getRadiusMeters();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointDistanceRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointDistanceRangeQuery.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointDistanceRangeQuery.java
deleted file mode 100644
index eefdc79..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointDistanceRangeQuery.java
+++ /dev/null
@@ -1,116 +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.search;
-
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.spatial.document.GeoPointField.TermEncoding;
-
-/** Implements a point distance range query on a GeoPoint field. This is based on
- * {@code org.apache.lucene.spatial.search.GeoPointDistanceQuery} and is implemented using a
- * {@code org.apache.lucene.search.BooleanClause.MUST_NOT} clause to exclude any points that fall within
- * minRadiusMeters from the provided point.
- *
- *    @lucene.experimental
- */
-public final class GeoPointDistanceRangeQuery extends GeoPointDistanceQuery {
-  protected final double minRadiusMeters;
-
-  /**
-   * Constructs a query for all {@link org.apache.lucene.spatial.document.GeoPointField} types within a minimum / maximum
-   * distance (in meters) range from a given point
-   */
-  public GeoPointDistanceRangeQuery(final String field, final double centerLon, final double centerLat,
-                                    final double minRadiusMeters, final double maxRadiusMeters) {
-    this(field, TermEncoding.PREFIX, centerLon, centerLat, minRadiusMeters, maxRadiusMeters);
-  }
-
-  public GeoPointDistanceRangeQuery(final String field, final TermEncoding termEncoding, final double centerLon, final double centerLat,
-                                    final double minRadiusMeters, final double maxRadius) {
-    super(field, termEncoding, centerLon, centerLat, maxRadius);
-    this.minRadiusMeters = minRadiusMeters;
-  }
-
-  @Override
-  public Query rewrite(IndexReader reader) {
-    Query q = super.rewrite(reader);
-    if (minRadiusMeters == 0.0) {
-      return q;
-    }
-
-    // add an exclusion query
-    BooleanQuery.Builder bqb = new BooleanQuery.Builder();
-
-    // create a new exclusion query
-    GeoPointDistanceQuery exclude = new GeoPointDistanceQuery(field, termEncoding, centerLon, centerLat, minRadiusMeters);
-    // full map search
-//    if (radiusMeters >= GeoProjectionUtils.SEMIMINOR_AXIS) {
-//      bqb.add(new BooleanClause(new GeoPointInBBoxQuery(this.field, -180.0, -90.0, 180.0, 90.0), BooleanClause.Occur.MUST));
-//    } else {
-      bqb.add(new BooleanClause(q, BooleanClause.Occur.MUST));
-//    }
-    bqb.add(new BooleanClause(exclude, BooleanClause.Occur.MUST_NOT));
-
-    return bqb.build();
-  }
-
-  @Override
-  public String toString(String field) {
-    final StringBuilder sb = new StringBuilder();
-    sb.append(getClass().getSimpleName());
-    sb.append(':');
-    if (!this.field.equals(field)) {
-      sb.append(" field=");
-      sb.append(this.field);
-      sb.append(':');
-    }
-    return sb.append( " Center: [")
-        .append(centerLon)
-        .append(',')
-        .append(centerLat)
-        .append(']')
-        .append(" From Distance: ")
-        .append(minRadiusMeters)
-        .append(" m")
-        .append(" To Distance: ")
-        .append(radiusMeters)
-        .append(" m")
-        .append(" Lower Left: [")
-        .append(minLon)
-        .append(',')
-        .append(minLat)
-        .append(']')
-        .append(" Upper Right: [")
-        .append(maxLon)
-        .append(',')
-        .append(maxLat)
-        .append("]")
-        .toString();
-  }
-
-  /** getter method for minimum distance */
-  public double getMinRadiusMeters() {
-    return this.minRadiusMeters;
-  }
-
-  /** getter method for maximum distance */
-  public double getMaxRadiusMeters() {
-    return this.radiusMeters;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInBBoxQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInBBoxQuery.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInBBoxQuery.java
deleted file mode 100644
index 0e6d5bf..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInBBoxQuery.java
+++ /dev/null
@@ -1,173 +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.search;
-
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.FieldValueQuery;
-import org.apache.lucene.search.LegacyNumericRangeQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.spatial.document.GeoPointField.TermEncoding;
-import org.apache.lucene.spatial.util.GeoUtils;
-
-/** Implements a simple bounding box query on a GeoPoint field. This is inspired by
- * {@link LegacyNumericRangeQuery} and is implemented using a
- * two phase approach. First, 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 final check that verifies whether
- * the decoded lat/lon falls within (or on the boundary) of the query bounding box.
- * The value comparisons are subject to a precision tolerance defined in
- * {@value org.apache.lucene.spatial.util.GeoEncodingUtils#TOLERANCE}
- *
- * NOTES:
- *    1.  All latitude/longitude values must be in decimal degrees.
- *    2.  Complex computational geometry (e.g., dateline wrapping) is not supported
- *    3.  For more advanced GeoSpatial indexing and query operations see spatial module
- *    4.  This is well suited for small rectangles, large bounding boxes may result
- *        in many terms, depending whether the bounding box falls on the boundary of
- *        many cells (degenerate case)
- *
- * @lucene.experimental
- */
-public class GeoPointInBBoxQuery extends Query {
-  protected final String field;
-  protected final double minLon;
-  protected final double minLat;
-  protected final double maxLon;
-  protected final double maxLat;
-  protected final TermEncoding termEncoding;
-
-  /**
-   * Constructs a query for all {@link org.apache.lucene.spatial.document.GeoPointField} types that fall within a
-   * defined bounding box
-   */
-  public GeoPointInBBoxQuery(final String field, final double minLon, final double minLat, final double maxLon, final double maxLat) {
-    this(field, TermEncoding.PREFIX, minLon, minLat, maxLon, maxLat);
-  }
-
-  public GeoPointInBBoxQuery(final String field, final TermEncoding termEncoding, final double minLon, final double minLat, final double maxLon, final double maxLat) {
-    this.field = field;
-    this.minLon = minLon;
-    this.minLat = minLat;
-    this.maxLon = maxLon;
-    this.maxLat = maxLat;
-    this.termEncoding = termEncoding;
-  }
-
-  @Override
-  public Query rewrite(IndexReader reader) {
-    // short-circuit to match all if specifying the whole map
-    if (minLon == GeoUtils.MIN_LON_INCL && maxLon == GeoUtils.MAX_LON_INCL
-        && minLat == GeoUtils.MIN_LAT_INCL && maxLat == GeoUtils.MAX_LAT_INCL) {
-      // FieldValueQuery is valid since DocValues are *required* for GeoPointField
-      return new FieldValueQuery(field);
-    }
-
-    if (maxLon < minLon) {
-      BooleanQuery.Builder bqb = new BooleanQuery.Builder();
-
-      GeoPointInBBoxQueryImpl left = new GeoPointInBBoxQueryImpl(field, termEncoding, -180.0D, minLat, maxLon, maxLat);
-      bqb.add(new BooleanClause(left, BooleanClause.Occur.SHOULD));
-      GeoPointInBBoxQueryImpl right = new GeoPointInBBoxQueryImpl(field, termEncoding, minLon, minLat, 180.0D, maxLat);
-      bqb.add(new BooleanClause(right, BooleanClause.Occur.SHOULD));
-      return bqb.build();
-    }
-    return new GeoPointInBBoxQueryImpl(field, termEncoding, minLon, minLat, maxLon, maxLat);
-  }
-
-  @Override
-  public String toString(String field) {
-    final StringBuilder sb = new StringBuilder();
-    sb.append(getClass().getSimpleName());
-    sb.append(':');
-    if (!this.field.equals(field)) {
-      sb.append(" field=");
-      sb.append(this.field);
-      sb.append(':');
-    }
-    return sb.append(" Lower Left: [")
-        .append(minLon)
-        .append(',')
-        .append(minLat)
-        .append(']')
-        .append(" Upper Right: [")
-        .append(maxLon)
-        .append(',')
-        .append(maxLat)
-        .append("]")
-        .toString();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (!(o instanceof GeoPointInBBoxQuery)) return false;
-    if (!super.equals(o)) return false;
-
-    GeoPointInBBoxQuery that = (GeoPointInBBoxQuery) 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;
-    if (!field.equals(that.field)) return false;
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int result = super.hashCode();
-    long temp;
-    result = 31 * result + field.hashCode();
-    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;
-  }
-
-  /** getter method for retrieving the field name */
-  public final String getField() {
-    return this.field;
-  }
-
-  /** getter method for retrieving the minimum longitude (in degrees) */
-  public final double getMinLon() {
-    return this.minLon;
-  }
-
-  /** getter method for retrieving the minimum latitude (in degrees) */
-  public final double getMinLat() {
-    return this.minLat;
-  }
-
-  /** getter method for retrieving the maximum longitude (in degrees) */
-  public final double getMaxLon() {
-    return this.maxLon;
-  }
-
-  /** getter method for retrieving the maximum latitude (in degrees) */
-  public final double getMaxLat() {
-    return this.maxLat;
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7d8f0127/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
deleted file mode 100644
index 8bce0f0..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInBBoxQueryImpl.java
+++ /dev/null
@@ -1,156 +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.search;
-
-import org.apache.lucene.search.MultiTermQuery;
-import org.apache.lucene.util.SloppyMath;
-import org.apache.lucene.spatial.document.GeoPointField;
-import org.apache.lucene.spatial.document.GeoPointField.TermEncoding;
-import org.apache.lucene.spatial.util.GeoRelationUtils;
-
-/** Package private implementation for the public facing GeoPointInBBoxQuery delegate class.
- *
- *    @lucene.experimental
- */
-class GeoPointInBBoxQueryImpl extends GeoPointMultiTermQuery {
-  /**
-   * 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 TermEncoding termEncoding, final double minLon, final double minLat, final double maxLon, final double maxLat) {
-    super(field, termEncoding, minLon, minLat, maxLon, maxLat);
-  }
-
-  @Override
-  public void setRewriteMethod(MultiTermQuery.RewriteMethod method) {
-    throw new UnsupportedOperationException("cannot change rewrite method");
-  }
-
-  @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);
-  }
-
-  @Override
-  protected CellComparator newCellComparator() {
-    return new GeoPointInBBoxCellComparator(this);
-  }
-
-  private final class GeoPointInBBoxCellComparator extends CellComparator {
-    GeoPointInBBoxCellComparator(GeoPointMultiTermQuery query) {
-      super(query);
-    }
-
-    /**
-     * 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, GeoPointInBBoxQueryImpl.this.minLon,
-          GeoPointInBBoxQueryImpl.this.minLat, GeoPointInBBoxQueryImpl.this.maxLon, GeoPointInBBoxQueryImpl.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, GeoPointInBBoxQueryImpl.this.minLon,
-          GeoPointInBBoxQueryImpl.this.minLat, GeoPointInBBoxQueryImpl.this.maxLon, GeoPointInBBoxQueryImpl.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/7d8f0127/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
deleted file mode 100644
index eb8c50c..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInPolygonQuery.java
+++ /dev/null
@@ -1,153 +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.search;
-
-import java.util.Arrays;
-
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.spatial.document.GeoPointField.TermEncoding;
-import org.apache.lucene.spatial.util.GeoEncodingUtils;
-import org.apache.lucene.spatial.util.GeoRect;
-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 GeoEncodingUtils#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 GeoPointInBBoxQuery {
-  // polygon position arrays - this avoids the use of any objects or
-  // or geo library dependencies
-  protected final double[] x;
-  protected final double[] y;
-
-  public GeoPointInPolygonQuery(final String field, final double[] polyLons, final double[] polyLats) {
-    this(field, TermEncoding.PREFIX, GeoUtils.polyToBBox(polyLons, polyLats), polyLons, polyLats);
-  }
-
-  /**
-   * 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 TermEncoding termEncoding, final double[] polyLons, final double[] polyLats) {
-    this(field, termEncoding, GeoUtils.polyToBBox(polyLons, polyLats), polyLons, polyLats);
-  }
-
-  /** Common constructor, used only internally. */
-  private GeoPointInPolygonQuery(final String field, TermEncoding termEncoding, GeoRect bbox, final double[] polyLons, final double[] polyLats) {
-    super(field, termEncoding, 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;
-  }
-
-  /** throw exception if trying to change rewrite method */
-  @Override
-  public Query rewrite(IndexReader reader) {
-    return new GeoPointInPolygonQueryImpl(field, termEncoding, this, this.minLon, this.minLat, this.maxLon, this.maxLat);
-  }
-
-  @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();
-  }
-
-  /**
-   * 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/7d8f0127/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInPolygonQueryImpl.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInPolygonQueryImpl.java b/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInPolygonQueryImpl.java
deleted file mode 100644
index f0fe87b..0000000
--- a/lucene/spatial/src/java/org/apache/lucene/spatial/search/GeoPointInPolygonQueryImpl.java
+++ /dev/null
@@ -1,103 +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.search;
-
-import org.apache.lucene.search.MultiTermQuery;
-import org.apache.lucene.spatial.document.GeoPointField.TermEncoding;
-import org.apache.lucene.spatial.util.GeoRelationUtils;
-
-/** Package private implementation for the public facing GeoPointInPolygonQuery delegate class.
- *
- *    @lucene.experimental
- */
-final class GeoPointInPolygonQueryImpl extends GeoPointInBBoxQueryImpl {
-  private final GeoPointInPolygonQuery polygonQuery;
-
-  GeoPointInPolygonQueryImpl(final String field, final TermEncoding termEncoding, final GeoPointInPolygonQuery q,
-                             final double minLon, final double minLat, final double maxLon, final double maxLat) {
-    super(field, termEncoding, minLon, minLat, maxLon, maxLat);
-    polygonQuery = q;
-  }
-
-  @Override
-  public void setRewriteMethod(MultiTermQuery.RewriteMethod method) {
-    throw new UnsupportedOperationException("cannot change rewrite method");
-  }
-
-  @Override
-  protected CellComparator newCellComparator() {
-    return new GeoPolygonCellComparator(this);
-  }
-
-  /**
-   * Custom {@code org.apache.lucene.spatial.search.GeoPointMultiTermQuery.CellComparator} that computes morton hash
-   * ranges based on the defined edges of the provided polygon.
-   */
-  private final class GeoPolygonCellComparator extends CellComparator {
-    GeoPolygonCellComparator(GeoPointMultiTermQuery query) {
-      super(query);
-    }
-
-    @Override
-    protected boolean cellCrosses(final double minLon, final double minLat, final double maxLon, final double maxLat) {
-      return GeoRelationUtils.rectCrossesPolyApprox(minLon, minLat, maxLon, maxLat, polygonQuery.x, polygonQuery.y,
-          polygonQuery.minLon, polygonQuery.minLat, polygonQuery.maxLon, polygonQuery.maxLat);
-    }
-
-    @Override
-    protected boolean cellWithin(final double minLon, final double minLat, final double maxLon, final double maxLat) {
-      return GeoRelationUtils.rectWithinPolyApprox(minLon, minLat, maxLon, maxLat, polygonQuery.x, polygonQuery.y,
-          polygonQuery.minLon, polygonQuery.minLat, polygonQuery.maxLon, polygonQuery.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 org.apache.lucene.spatial.search.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(polygonQuery.x, polygonQuery.y, lat, lon);
-    }
-  }
-
-  @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;
-
-    GeoPointInPolygonQueryImpl that = (GeoPointInPolygonQueryImpl) o;
-
-    return !(polygonQuery != null ? !polygonQuery.equals(that.polygonQuery) : that.polygonQuery != null);
-  }
-
-  @Override
-  public int hashCode() {
-    int result = super.hashCode();
-    result = 31 * result + (polygonQuery != null ? polygonQuery.hashCode() : 0);
-    return result;
-  }
-}