You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/05/26 21:28:06 UTC

svn commit: r1681821 - in /lucene/dev/branches/LUCENE-6481/lucene: core/src/java/org/apache/lucene/index/ core/src/java/org/apache/lucene/search/ core/src/java/org/apache/lucene/util/ core/src/test/org/apache/lucene/index/ queries/src/java/org/apache/l...

Author: mikemccand
Date: Tue May 26 19:28:05 2015
New Revision: 1681821

URL: http://svn.apache.org/r1681821
Log:
LUCENE-6481: commit latest patch

Added:
    lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/document/
    lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/document/GeoPointField.java   (with props)
    lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointInBBoxQuery.java   (with props)
    lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointInPolygonQuery.java   (with props)
    lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/util/
    lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/util/GeoUtils.java   (with props)
    lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/test/org/apache/lucene/search/TestGeoPointQuery.java   (with props)
Modified:
    lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
    lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
    lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java
    lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/util/BitUtil.java
    lucene/dev/branches/LUCENE-6481/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/dev/branches/LUCENE-6481/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java

Modified: lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1681821&r1=1681820&r2=1681821&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Tue May 26 19:28:05 2015
@@ -631,6 +631,7 @@ public final class SegmentInfos implemen
   void updateGeneration(SegmentInfos other) {
     lastGeneration = other.lastGeneration;
     generation = other.generation;
+    System.out.println("UPDATE GEN: " + lastGeneration + " " + generation);
   }
 
   void setGeneration(long generation) {
@@ -712,6 +713,7 @@ public final class SegmentInfos implemen
 
     pendingCommit = false;
     lastGeneration = generation;
+    System.out.println("finishCommit: " + lastGeneration);
     return dest;
   }
 

Modified: lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java?rev=1681821&r1=1681820&r2=1681821&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java Tue May 26 19:28:05 2015
@@ -400,6 +400,7 @@ final class StandardDirectoryReader exte
       userData = infos.getUserData();
       files = Collections.unmodifiableCollection(infos.files(true));
       generation = infos.getGeneration();
+      System.out.println("getGen: " + generation);
       segmentCount = infos.size();
     }
 

Modified: lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java?rev=1681821&r1=1681820&r2=1681821&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java Tue May 26 19:28:05 2015
@@ -167,8 +167,7 @@ import org.apache.lucene.index.Term; //
 public final class NumericRangeQuery<T extends Number> extends MultiTermQuery {
 
   private NumericRangeQuery(final String field, final int precisionStep, final NumericType dataType,
-    T min, T max, final boolean minInclusive, final boolean maxInclusive
-  ) {
+    T min, T max, final boolean minInclusive, final boolean maxInclusive) {
     super(field);
     if (precisionStep < 1)
       throw new IllegalArgumentException("precisionStep must be >=1");

Modified: lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/util/BitUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/util/BitUtil.java?rev=1681821&r1=1681820&r2=1681821&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/util/BitUtil.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/core/src/java/org/apache/lucene/util/BitUtil.java Tue May 26 19:28:05 2015
@@ -22,6 +22,15 @@ package org.apache.lucene.util; // from
  */
 public final class BitUtil {
 
+  // magic numbers for bit interleaving
+  private static final long MAGIC[] = {
+      0x5555555555555555L, 0x3333333333333333L,
+      0x0F0F0F0F0F0F0F0FL, 0x00FF00FF00FF00FFL,
+      0x0000FFFF0000FFFFL, 0x00000000FFFFFFFFL
+  };
+  // shift values for bit interleaving
+  private static final short SHIFT[] = {1, 2, 4, 8, 16};
+
   private BitUtil() {} // no instance
 
   // The pop methods used to rely on bit-manipulation tricks for speed but it
@@ -102,6 +111,39 @@ public final class BitUtil {
     return v;
   }
 
+  /**
+   * Interleaves the first 32 bits of each long value
+   *
+   * Adapted from: http://graphics.stanford.edu/~seander/bithacks.html#InterleaveBMN
+   */
+  public static long interleave(long v1, long v2) {
+    v1 = (v1 | (v1 << SHIFT[4])) & MAGIC[4];
+    v1 = (v1 | (v1 << SHIFT[3])) & MAGIC[3];
+    v1 = (v1 | (v1 << SHIFT[2])) & MAGIC[2];
+    v1 = (v1 | (v1 << SHIFT[1])) & MAGIC[1];
+    v1 = (v1 | (v1 << SHIFT[0])) & MAGIC[0];
+    v2 = (v2 | (v2 << SHIFT[4])) & MAGIC[4];
+    v2 = (v2 | (v2 << SHIFT[3])) & MAGIC[3];
+    v2 = (v2 | (v2 << SHIFT[2])) & MAGIC[2];
+    v2 = (v2 | (v2 << SHIFT[1])) & MAGIC[1];
+    v2 = (v2 | (v2 << SHIFT[0])) & MAGIC[0];
+
+    return (v2<<1) | v1;
+  }
+
+  /**
+   * Deinterleaves long value back to two concatenated 32bit values
+   */
+  public static long deinterleave(long b) {
+    b &= MAGIC[0];
+    b = (b ^ (b >>> SHIFT[0])) & MAGIC[1];
+    b = (b ^ (b >>> SHIFT[1])) & MAGIC[2];
+    b = (b ^ (b >>> SHIFT[2])) & MAGIC[3];
+    b = (b ^ (b >>> SHIFT[3])) & MAGIC[4];
+    b = (b ^ (b >>> SHIFT[4])) & MAGIC[5];
+    return b;
+  }
+
    /** Same as {@link #zigZagEncode(long)} but on integers. */
    public static int zigZagEncode(int i) {
      return (i >> 31) ^ (i << 1);

Modified: lucene/dev/branches/LUCENE-6481/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1681821&r1=1681820&r2=1681821&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java Tue May 26 19:28:05 2015
@@ -2721,5 +2721,25 @@ public class TestIndexWriter extends Luc
     r.close();
     dir.close();
   }
+
+  public void testCommitWithNRTReader() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    IndexWriter w = new IndexWriter(dir, iwc);
+    w.commit();
+    DirectoryReader r = DirectoryReader.open(w, true);
+    System.out.println("R: " + r.getIndexCommit().getGeneration());
+    w.addDocument(new Document());
+    w.commit();
+    DirectoryReader r2 = DirectoryReader.openIfChanged(r);
+    assertNotNull(r2);
+    System.out.println("R2: " + r2.getIndexCommit().getGeneration());
+    w.addDocument(new Document());
+    w.commit();
+    DirectoryReader r3 = DirectoryReader.openIfChanged(r2);
+    assertNotNull(r3);
+    System.out.println("R3: " + r3.getIndexCommit().getGeneration());
+    IOUtils.close(r, r2, r3, w, dir);
+  }
 }
 

Modified: lucene/dev/branches/LUCENE-6481/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java?rev=1681821&r1=1681820&r2=1681821&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java (original)
+++ lucene/dev/branches/LUCENE-6481/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java Tue May 26 19:28:05 2015
@@ -20,8 +20,6 @@ package org.apache.lucene.queries;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
@@ -199,11 +197,6 @@ public class TermsQuery extends Query im
     return BASE_RAM_BYTES_USED + termData.ramBytesUsed();
   }
 
-  @Override
-  public Collection<Accountable> getChildResources() {
-    return Collections.emptyList();
-  }
-
   private static class TermAndState {
     final String field;
     final TermsEnum termsEnum;

Added: lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/document/GeoPointField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/document/GeoPointField.java?rev=1681821&view=auto
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/document/GeoPointField.java (added)
+++ lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/document/GeoPointField.java Tue May 26 19:28:05 2015
@@ -0,0 +1,105 @@
+package org.apache.lucene.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 org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.util.GeoUtils;
+
+/**
+ * <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.search.GeoPointInBBoxQuery} or {@link org.apache.lucene.search.GeoPointInPolygonQuery}
+ *
+ * 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 {
+  public static final int PRECISION_STEP = 6;
+
+  /**
+   * Type for an GeoPointField that is not stored:
+   * normalization factors, frequencies, and positions are omitted.
+   */
+  public static final FieldType TYPE_NOT_STORED = new FieldType();
+  static {
+    TYPE_NOT_STORED.setTokenized(false);
+    TYPE_NOT_STORED.setOmitNorms(true);
+    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS);
+    TYPE_NOT_STORED.setNumericType(FieldType.NumericType.LONG);
+    TYPE_NOT_STORED.setNumericPrecisionStep(PRECISION_STEP);
+    TYPE_NOT_STORED.freeze();
+  }
+
+  /**
+   * Type for a stored GeoPointField:
+   * normalization factors, frequencies, and positions are omitted.
+   */
+  public static final FieldType TYPE_STORED = new FieldType();
+  static {
+    TYPE_STORED.setTokenized(false);
+    TYPE_STORED.setOmitNorms(true);
+    TYPE_STORED.setIndexOptions(IndexOptions.DOCS);
+    TYPE_STORED.setNumericType(FieldType.NumericType.LONG);
+    TYPE_STORED.setNumericPrecisionStep(PRECISION_STEP);
+    TYPE_STORED.setStored(true);
+    TYPE_STORED.freeze();
+  }
+
+  /** Creates a stored or un-stored GeoPointField with the provided value
+   *  and default <code>precisionStep</code> set to 64 to avoid wasteful
+   *  indexing of lower precision terms.
+   *  @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) {
+    super(name, stored == Store.YES ? TYPE_STORED : TYPE_NOT_STORED);
+    fieldsData = GeoUtils.mortonHash(lon, lat);
+  }
+
+  /** 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 FieldType.NumericType#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);
+    if (type.numericType() != FieldType.NumericType.LONG) {
+      throw new IllegalArgumentException("type.numericType() must be LONG but got " + type.numericType());
+    }
+    fieldsData = GeoUtils.mortonHash(lon, lat);
+  }
+}

Added: lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointInBBoxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointInBBoxQuery.java?rev=1681821&view=auto
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointInBBoxQuery.java (added)
+++ lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointInBBoxQuery.java Tue May 26 19:28:05 2015
@@ -0,0 +1,294 @@
+package org.apache.lucene.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 java.util.Collections;
+import java.util.LinkedList;
+
+import org.apache.lucene.document.GeoPointField;
+import org.apache.lucene.index.FilteredTermsEnum;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.GeoUtils;
+import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util.ToStringUtils;
+
+/** Implements a simple bounding box query on a GeoPoint field. This is inspired by
+ * {@link org.apache.lucene.search.NumericRangeQuery} 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.util.GeoUtils#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, could result
+ *        in visiting every term in terms dictionary (see LUCENE-6481)
+ *
+ * @lucene.experimental
+ */
+public class GeoPointInBBoxQuery 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;
+
+  private static final short DETAIL_LEVEL = 16;
+
+  /**
+   * 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
+   */
+  public GeoPointInBBoxQuery(final String field, final double minLon, final double minLat, final double maxLon, final double maxLat) {
+    super(field);
+    this.minLon = minLon;
+    this.minLat = minLat;
+    this.maxLon = maxLon;
+    this.maxLat = maxLat;
+  }
+
+  @Override @SuppressWarnings("unchecked")
+  protected TermsEnum getTermsEnum(final Terms terms, AttributeSource atts) throws IOException {
+    final Long min = GeoUtils.mortonHash(minLon, minLat);
+    final Long max = Math.abs(GeoUtils.mortonHash(maxLon, maxLat));
+    if (min != null && max != null &&  min.compareTo(max) > 0) {
+      return TermsEnum.EMPTY;
+    }
+    return new GeoBBoxTermsEnum(terms.iterator());
+  }
+
+  @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;
+
+    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;
+
+    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();
+    if (!getField().equals(field)) sb.append(getField()).append(':');
+    return sb.append(" Lower Left: [")
+        .append(minLon)
+        .append(',')
+        .append(minLat)
+        .append(']')
+        .append(" Upper Right: [")
+        .append(maxLon)
+        .append(',')
+        .append(maxLat)
+        .append("]")
+        .append(ToStringUtils.boost(getBoost()))
+        .toString();
+  }
+
+  /**
+   * computes all ranges along a space-filling curve that represents
+   * the given bounding box and enumerates all terms contained within those ranges
+   */
+  protected class GeoBBoxTermsEnum extends FilteredTermsEnum {
+    private Range currentRange;
+    private BytesRef currentLowerBound, currentUpperBound;
+    private final LinkedList<Range> rangeBounds = new LinkedList<>();
+
+    GeoBBoxTermsEnum(final TermsEnum tenum) {
+      super(tenum);
+      computeRange(0L, (short) (((GeoUtils.BITS) << 1) - 1));
+      Collections.sort(rangeBounds);
+    }
+
+    /**
+     * entry point for recursively computing ranges
+     */
+    private final void computeRange(long term, final short shift) {
+      final long split = term | (0x1L<<shift);
+      final long upperMax = term | ((0x1L<<(shift+1))-1);
+      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)
+     * @return
+     */
+    private void relateAndRecurse(final long start, final long end, final short res) {
+      final double minLon = GeoUtils.mortonUnhash(start, true);
+      final double minLat = GeoUtils.mortonUnhash(start, false);
+      final double maxLon = GeoUtils.mortonUnhash(end, true);
+      final double maxLat = GeoUtils.mortonUnhash(end, false);
+
+      final short level = (short)(62-res>>>1);
+
+      final boolean within = isWithin(minLon, minLat, maxLon, maxLat);
+      final boolean bboxIntersects = (within) ? true : intersects(minLon, minLat, maxLon, maxLat);
+
+      if ((within && res%GeoPointField.PRECISION_STEP == 0) || (bboxIntersects && level == DETAIL_LEVEL)) {
+        rangeBounds.add(new Range(start, end, res, level, !within));
+      } else if (bboxIntersects) {
+        computeRange(start, (short)(res - 1));
+      }
+    }
+
+    protected boolean intersects(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+      return GeoUtils.rectIntersects(minLon, minLat, maxLon, maxLat, GeoPointInBBoxQuery.this.minLon,
+          GeoPointInBBoxQuery.this.minLat, GeoPointInBBoxQuery.this.maxLon, GeoPointInBBoxQuery.this.maxLat);
+    }
+
+    protected boolean isWithin(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+      return GeoUtils.rectIsWithin(minLon, minLat, maxLon, maxLat, GeoPointInBBoxQuery.this.minLon,
+          GeoPointInBBoxQuery.this.minLat, GeoPointInBBoxQuery.this.maxLon, GeoPointInBBoxQuery.this.maxLat);
+    }
+
+    private void nextRange() {
+      currentRange = rangeBounds.removeFirst();
+      currentLowerBound = currentRange.lower;
+      assert currentUpperBound == null || currentUpperBound.compareTo(currentRange.lower) <= 0 :
+          "The current upper bound must be <= the new lower bound";
+
+      currentUpperBound = currentRange.upper;
+    }
+
+    @Override
+    protected final BytesRef nextSeekTerm(BytesRef term) {
+      while (!rangeBounds.isEmpty()) {
+        if (currentRange == null)
+          nextRange();
+
+        // if the new upper bound is before the term parameter, the sub-range is never a hit
+        if (term != null && term.compareTo(currentUpperBound) > 0) {
+          nextRange();
+	        if (!rangeBounds.isEmpty())
+	          continue;
+        }
+        // never seek backwards, so use current term if lower bound is smaller
+        return (term != null && term.compareTo(currentLowerBound) > 0) ?
+            term : currentLowerBound;
+      }
+
+      // no more sub-range enums available
+      assert rangeBounds.isEmpty();
+      currentLowerBound = currentUpperBound = null;
+      return null;
+    }
+
+    /**
+     * The two-phase query approach. {@link #nextSeekTerm} is called to obtain the next term that matches a numeric
+     * range of the bounding box. Those terms that pass the initial range filter are then compared against the
+     * decoded min/max latitude and longitude values of the bounding box only if the range is not a "boundary" range
+     * (e.g., a range that straddles the boundary of the bbox).
+     * @param term term for candidate document
+     * @return match status
+     */
+    @Override
+    protected AcceptStatus accept(BytesRef term) {
+      // validate value is in range
+      while (currentUpperBound == null || term.compareTo(currentUpperBound) > 0) {
+        if (rangeBounds.isEmpty())
+          return AcceptStatus.END;
+        // peek next sub-range, only seek if the current term is smaller than next lower bound
+        if (term.compareTo(rangeBounds.getFirst().lower) < 0)
+          return AcceptStatus.NO_AND_SEEK;
+        // step forward to next range without seeking, as next lower range bound is less or equal current term
+        nextRange();
+      }
+
+      // final-filter boundary ranges by bounding box
+      if (currentRange.boundary) {
+        final long val = NumericUtils.prefixCodedToLong(term);
+        final double lon = GeoUtils.mortonUnhash(val, true);
+        final double lat = GeoUtils.mortonUnhash(val, false);
+        if (!GeoUtils.bboxContains(lon, lat, minLon, minLat, maxLon, maxLat)) {
+          return AcceptStatus.NO;
+        }
+      }
+      return AcceptStatus.YES;
+    }
+
+    /**
+     * Internal class to represent a range along the space filling curve
+     */
+    private final class Range implements Comparable<Range> {
+      final BytesRef lower;
+      final BytesRef upper;
+      final short level;
+      final boolean boundary;
+
+      Range(final long lower, final long upper, final short res, final short level, boolean boundary) {
+        this.level = level;
+        this.boundary = boundary;
+
+        BytesRefBuilder brb = new BytesRefBuilder();
+        NumericUtils.longToPrefixCodedBytes(lower, boundary ? 0 : res, brb);
+        this.lower = brb.get();
+        NumericUtils.longToPrefixCodedBytes(upper, boundary ? 0 : res, (brb = new BytesRefBuilder()));
+        this.upper = brb.get();
+      }
+
+      @Override
+      public final int compareTo(Range other) {
+        return this.lower.compareTo(other.lower);
+      }
+    }
+  }
+}

Added: lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointInPolygonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointInPolygonQuery.java?rev=1681821&view=auto
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointInPolygonQuery.java (added)
+++ lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/search/GeoPointInPolygonQuery.java Tue May 26 19:28:05 2015
@@ -0,0 +1,187 @@
+package org.apache.lucene.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.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.GeoUtils;
+import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util.ToStringUtils;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/** Implements a simple point in polygon query on a GeoPoint field. This is based on
+ * {@link GeoPointInBBoxQuery} and is implemented using a
+ * three phase approach. First, like {@link GeoPointInBBoxQuery}
+ * candidate terms are queried using a numeric range based on the morton codes
+ * of the min and max lat/lon pairs. Terms passing this initial filter are passed
+ * to a secondary filter that verifies whether the decoded lat/lon point falls within
+ * (or on the boundary) of the bounding box query. Finally, the remaining candidate
+ * term is passed to the final point in polygon check. All value comparisons are subject
+ * to the same precision tolerance defined in {@value org.apache.lucene.util.GeoUtils#TOLERANCE}
+ *
+ * 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
+  private final double[] x;
+  private final double[] y;
+
+  /**
+   * Constructs a new GeoPolygonQuery that will match encoded {@link org.apache.lucene.document.GeoPointField} terms
+   * that fall within or on the boundary of the polygon defined by the input parameters. This constructor requires a
+   * precomputed bounding box. As an alternative the {@link GeoPointInPolygonQuery#newPolygonQuery} static factory can
+   * be used to compute the bounding box during construction
+   *
+   * @param field the field name
+   * @param minLon lower longitude (x) value of the bounding box optimizer
+   * @param minLat lower latitude (y) value of the bounding box optimizer
+   * @param maxLon upper longitude (x) value of the bounding box optimizer
+   * @param maxLat upper latitude (y) value of the bounding box optimizer
+   * @param polyLons array containing all longitude values for the polygon
+   * @param polyLats array containing all latitude values for the polygon
+   */
+  public GeoPointInPolygonQuery(final String field, final double minLon, final double minLat, final double maxLon,
+                                final double maxLat, final double[] polyLons, final double[] polyLats) {
+    super(field, minLon, minLat, maxLon, maxLat);
+    this.x = polyLons;
+    this.y = polyLats;
+  }
+
+  /**
+   * Static method call to construct a
+   * {@link #GeoPointInPolygonQuery(String, double, double, double, double, double[], double[])
+   * GeoPolygonQuery(field, minLon, minLat, maxLon, maxLat, polyLons, polyLats)} by first computing the bounding
+   * box lat/lon ranges
+   */
+  public static GeoPointInPolygonQuery newPolygonQuery(final String field, final double[] polyLons, final double[] polyLats) {
+    assert polyLons.length == polyLats.length;
+    double minLon, maxLon, minLat, maxLat;
+    int i=1;
+    for(minLon=maxLon=polyLons[0], minLat=maxLat=polyLats[0]; i<polyLons.length; ++i) {
+      minLon = Math.min(polyLons[i], minLon);
+      maxLon = Math.max(polyLons[i], maxLon);
+      minLat = Math.min(polyLats[i], minLat);
+      maxLat = Math.max(polyLats[i], maxLat);
+    }
+    return new GeoPointInPolygonQuery(field, minLon, minLat, maxLon, maxLat, polyLons, polyLats);
+  }
+
+  @Override @SuppressWarnings("unchecked")
+  protected TermsEnum getTermsEnum(final Terms terms, AttributeSource atts) throws IOException {
+    final Long min = GeoUtils.mortonHash(minLon, minLat);
+    final Long max = Math.abs(GeoUtils.mortonHash(maxLon, maxLat));
+    if (min != null && max != null &&  min.compareTo(max) > 0) {
+      return TermsEnum.EMPTY;
+    }
+    return new GeoPolygonTermsEnum(terms.iterator());
+  }
+
+  @Override
+  public final 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 final 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;
+  }
+
+  @Override
+  public String toString(String field) {
+    assert x.length == y.length;
+
+    final StringBuilder sb = new StringBuilder();
+    if (!getField().equals(field)) sb.append(getField()).append(':');
+    sb.append(" Points: ");
+    for (int i=0; i<x.length; ++i) {
+      sb.append("[")
+        .append(x[i])
+        .append(", ")
+        .append(y[i])
+        .append("] ");
+    }
+    sb.append(ToStringUtils.boost(getBoost()));
+
+    return sb.toString();
+  }
+
+  private final class GeoPolygonTermsEnum extends GeoBBoxTermsEnum {
+    GeoPolygonTermsEnum(final TermsEnum tenum) {
+            super(tenum);
+        }
+
+    @Override
+    protected boolean isWithin(final double minLon, final double minLat, final double maxLon, final double maxLat) {
+      return GeoUtils.rectIsWithin(minLon, minLat, maxLon, maxLat, x, y);
+    }
+
+    /**
+     * The two-phase query approach. The parent
+     * {@link org.apache.lucene.search.GeoPointInBBoxQuery.GeoBBoxTermsEnum#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.util.GeoUtils#pointInPolygon} method.
+     *
+     * @param term term for candidate document
+     * @return match status
+     */
+    @Override
+    protected final AcceptStatus accept(BytesRef term) {
+      // first filter by bounding box
+      AcceptStatus status = super.accept(term);
+      assert status != AcceptStatus.YES_AND_SEEK;
+
+      if (status != AcceptStatus.YES) {
+        return status;
+      }
+
+      final long val = NumericUtils.prefixCodedToLong(term);
+      final double lon = GeoUtils.mortonUnhash(val, true);
+      final double lat = GeoUtils.mortonUnhash(val, false);
+      // post-filter by point in polygon
+      if (x!=null && !GeoUtils.pointInPolygon(x, y, lat, lon)) {
+        return AcceptStatus.NO;
+      }
+      return AcceptStatus.YES;
+    }
+  }
+}

Added: lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/util/GeoUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/util/GeoUtils.java?rev=1681821&view=auto
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/util/GeoUtils.java (added)
+++ lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/java/org/apache/lucene/util/GeoUtils.java Tue May 26 19:28:05 2015
@@ -0,0 +1,252 @@
+package org.apache.lucene.util;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Basic reusable geo-spatial utility methods
+ *
+ * @lucene.experimental
+ */
+public final class GeoUtils {
+  // WGS84 earth-ellipsoid major (a) minor (b) radius, (f) flattening and eccentricity (e)
+  private static final double SEMIMAJOR_AXIS = 6_378_137; // [m]
+  private static final double FLATTENING = 1.0/298.257223563;
+  private static final double SEMIMINOR_AXIS = SEMIMAJOR_AXIS * (1.0 - FLATTENING); //6_356_752.31420; // [m]
+  private static final double ECCENTRICITY = StrictMath.sqrt((2.0 - FLATTENING) * FLATTENING);
+  private static final double PI_OVER_2 = StrictMath.PI / 2.0D;
+  private static final double SEMIMAJOR_AXIS2 = SEMIMAJOR_AXIS * SEMIMINOR_AXIS;
+  private static final double SEMIMINOR_AXIS2 = SEMIMINOR_AXIS * SEMIMINOR_AXIS;
+
+  private static final short MIN_LON = -180;
+  private static final short MIN_LAT = -90;
+  public static final short BITS = 31;
+  private static final double LON_SCALE = (0x1L<<BITS)/360.0D;
+  private static final double LAT_SCALE = (0x1L<<BITS)/180.0D;
+  private static final double TOLERANCE = 1E-7;
+
+  // No instance:
+  private GeoUtils() {
+  }
+
+  public static final Long mortonHash(final double lon, final double lat) {
+    return BitUtil.interleave(scale(lon, true), scale(lat, false));
+  }
+
+  public static final double mortonUnhash(final long hash, final boolean isLon) {
+    return unscale(BitUtil.deinterleave((isLon) ? hash : hash >>> 1), isLon);
+  }
+
+  private static long scale(final double val, final boolean lon) {
+    return (long) ((lon == true) ? (val-MIN_LON) * LON_SCALE : (val-MIN_LAT) * LAT_SCALE);
+  }
+
+  public static double unscale(final long val, final boolean lon) {
+    return (lon == true) ? (val / LON_SCALE) + MIN_LON : (val / LAT_SCALE) + MIN_LAT;
+  }
+
+  public static final double compare(final double v1, final double v2) {
+    final double compare = v1-v2;
+    return Math.abs(compare) <= TOLERANCE ? 0 : compare;
+  }
+
+  public static final boolean bboxContains(final double lon, final double lat, final double minLon,
+                                           final double minLat, final double maxLon, final double maxLat) {
+    return (compare(lon, minLon) >= 0 && compare(lon, maxLon) <= 0
+          && compare(lat, minLat) >= 0 && compare(lat, maxLat) <= 0);
+  }
+
+  /**
+   * Converts from geodesic lon lat alt to geocentric earth-centered earth-fixed
+   * @param lon geodesic longitude
+   * @param lat geodesic latitude
+   * @param alt geodesic altitude
+   * @param ecf reusable earth-centered earth-fixed result
+   * @return either a new ecef array or the reusable ecf parameter
+   */
+  public static final double[] llaToECF(double lon, double lat, double alt, double[] ecf) {
+    lon = StrictMath.toRadians(lon);
+    lat = StrictMath.toRadians(lat);
+
+    final double sl = StrictMath.sin(lat);
+    final double s2 = sl*sl;
+    final double cl = StrictMath.cos(lat);
+    final double ge2 = (SEMIMAJOR_AXIS2 - SEMIMINOR_AXIS2)/(SEMIMAJOR_AXIS2);
+
+    if (ecf == null)
+      ecf = new double[3];
+
+    if (lat < -PI_OVER_2 && lat > -1.001D * PI_OVER_2) {
+      lat = -PI_OVER_2;
+    } else if (lat > PI_OVER_2 && lat < 1.001D * PI_OVER_2) {
+      lat = PI_OVER_2;
+    }
+    assert ((lat >= -PI_OVER_2) || (lat <= PI_OVER_2));
+
+    if (lon > StrictMath.PI) {
+      lon -= (2*StrictMath.PI);
+    }
+
+    final double rn = SEMIMAJOR_AXIS / StrictMath.sqrt(1.0D - ge2 * s2);
+    ecf[0] = (rn+alt) * cl * StrictMath.cos(lon);
+    ecf[1] = (rn+alt) * cl * StrictMath.sin(lon);
+    ecf[2] = ((rn*(1.0-ge2))+alt)*sl;
+
+    return ecf;
+  }
+
+  /**
+   * Converts from geocentric earth-centered earth-fixed to geodesic lat/lon/alt
+   * @param x Cartesian x coordinate
+   * @param y Cartesian y coordinate
+   * @param z Cartesian z coordinate
+   * @param lla 0: longitude 1: latitude: 2: altitude
+   * @return double array as 0: longitude 1: latitude 2: altitude
+   */
+  public static final double[] ecfToLLA(final double x, final double y, final double z, double[] lla) {
+    boolean atPole = false;
+    final double ad_c = 1.0026000D;
+    final double e2 = (SEMIMAJOR_AXIS2 - SEMIMINOR_AXIS2)/(SEMIMAJOR_AXIS2);
+    final double ep2 = (SEMIMAJOR_AXIS2 - SEMIMINOR_AXIS2)/(SEMIMINOR_AXIS2);
+    final double cos67P5 = 0.38268343236508977D;
+
+    if (lla == null)
+      lla = new double[3];
+
+    if (x != 0.0) {
+      lla[0] = StrictMath.atan2(y,x);
+    } else {
+      if (y > 0) {
+        lla[0] = PI_OVER_2;
+      } else if (y < 0) {
+        lla[0] = -PI_OVER_2;
+      } else {
+        atPole = true;
+        lla[0] = 0.0D;
+        if (z > 0.0) {
+          lla[1] = PI_OVER_2;
+        } else if (z < 0.0) {
+          lla[1] = -PI_OVER_2;
+        } else {
+          lla[1] = PI_OVER_2;
+          lla[2] = -SEMIMINOR_AXIS;
+          return lla;
+        }
+      }
+    }
+
+    final double w2 = x*x + y*y;
+    final double w = StrictMath.sqrt(w2);
+    final double t0 = z * ad_c;
+    final double s0 = StrictMath.sqrt(t0 * t0 + w2);
+    final double sinB0 = t0 / s0;
+    final double cosB0 = w / s0;
+    final double sin3B0 = sinB0 * sinB0 * sinB0;
+    final double t1 = z + SEMIMINOR_AXIS * ep2 * sin3B0;
+    final double sum = w - SEMIMAJOR_AXIS * e2 * cosB0 * cosB0 * cosB0;
+    final double s1 = StrictMath.sqrt(t1 * t1 + sum * sum);
+    final double sinP1 = t1 / s1;
+    final double cosP1 = sum / s1;
+    final double rn = SEMIMAJOR_AXIS / StrictMath.sqrt(1.0D - e2 * sinP1 * sinP1);
+
+    if (cosP1 >= cos67P5) {
+      lla[2] = w / cosP1 - rn;
+    } else if (cosP1 <= -cos67P5) {
+      lla[2] = w / -cosP1 - rn;
+    } else {
+      lla[2] = z / sinP1 + rn * (e2 - 1.0);
+    }
+    if (!atPole) {
+      lla[1] = StrictMath.atan(sinP1/cosP1);
+    }
+    lla[0] = StrictMath.toDegrees(lla[0]);
+    lla[1] = StrictMath.toDegrees(lla[1]);
+
+    return lla;
+  }
+
+  /**
+   * simple even-odd point in polygon computation
+   *    1.  Determine if point is contained in the longitudinal range
+   *    2.  Determine whether point crosses the edge by computing the latitudinal delta
+   *        between the end-point of a parallel vector (originating at the point) and the
+   *        y-component of the edge sink
+   *
+   * NOTE: Requires polygon point (x,y) order either clockwise or counter-clockwise
+   */
+  public static boolean pointInPolygon(double[] x, double[] y, double lat, double lon) {
+    assert x.length == y.length;
+    boolean inPoly = false;
+    /**
+     * Note: This is using a euclidean coordinate system which could result in
+     * upwards of 110KM error at the equator.
+     * TODO convert coordinates to cylindrical projection (e.g. mercator)
+     */
+    for (int i = 1; i < x.length; i++) {
+      if (x[i] < lon && x[i-1] >= lon || x[i-1] < lon && x[i] >= lon) {
+        if (y[i] + (lon - x[i]) / (x[i-1] - x[i]) * (y[i-1] - y[i]) < lat) {
+          inPoly = !inPoly;
+        }
+      }
+    }
+    return inPoly;
+  }
+
+  public static String geoTermToString(long term) {
+    StringBuilder s = new StringBuilder(64);
+    final int numberOfLeadingZeros = Long.numberOfLeadingZeros(term);
+    for (int i = 0; i < numberOfLeadingZeros; i++) {
+      s.append('0');
+    }
+    if (term != 0)
+      s.append(Long.toBinaryString(term));
+    return s.toString();
+  }
+
+  /**
+   * Computes whether a rectangle is wholly within another rectangle (shared boundaries allowed)
+   */
+  public static boolean rectIsWithin(final double aMinX, final double aMinY, final double aMaxX, final double aMaxY,
+                                     final double bMinX, final double bMinY, final double bMaxX, final double bMaxY) {
+    return !(aMinX < bMinX || aMinY < bMinY || aMaxX > bMaxX || aMaxY > bMaxY);
+  }
+
+  /**
+   * Computes whether a rectangle intersects another rectangle
+   */
+  public static boolean rectContains(final double aMinX, final double aMinY, final double aMaxX, final double aMaxY,
+                                     final double bMinX, final double bMinY, final double bMaxX, final double bMaxY) {
+      return !(bMinX < aMinX || bMinY < aMinY || bMaxX > aMaxX || bMaxY > aMaxY);
+  }
+
+  /**
+   * Computes whether a rectangle contains another rectangle
+   */
+  public static boolean rectIntersects(final double aMinX, final double aMinY, final double aMaxX, final double aMaxY,
+                                       final double bMinX, final double bMinY, final double bMaxX, final double bMaxY) {
+    return !((aMaxX < bMinX || aMinX > bMaxX || aMaxY < bMinY || aMinY > bMaxY) );
+  }
+
+  /**
+   * Computes whether a rectangle is wholly within a given shape (shared boundaries allowed)
+   */
+  public static boolean rectIsWithin(final double rMinX, final double rMinY, final double rMaxX, final double rMaxY,
+                                     final double[] shapeX, final double[] shapeY) {
+    return !(!pointInPolygon(shapeX, shapeY, rMinY, rMinX) || !pointInPolygon(shapeX, shapeY, rMinY, rMaxX) ||
+        !pointInPolygon(shapeX, shapeY, rMaxY, rMaxX) || !pointInPolygon(shapeX, shapeY, rMaxY, rMinX));
+  }
+}
\ No newline at end of file

Added: lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/test/org/apache/lucene/search/TestGeoPointQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/test/org/apache/lucene/search/TestGeoPointQuery.java?rev=1681821&view=auto
==============================================================================
--- lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/test/org/apache/lucene/search/TestGeoPointQuery.java (added)
+++ lucene/dev/branches/LUCENE-6481/lucene/sandbox/src/test/org/apache/lucene/search/TestGeoPointQuery.java Tue May 26 19:28:05 2015
@@ -0,0 +1,339 @@
+package org.apache.lucene.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 java.util.HashSet;
+import java.util.Set;
+
+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.GeoPointField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.MultiDocValues;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.GeoUtils;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Unit testing for basic GeoPoint query logic
+ *
+ * @lucene.experimental
+ */
+public class TestGeoPointQuery extends LuceneTestCase {
+  private static Directory directory = null;
+  private static IndexReader reader = null;
+  private static IndexSearcher searcher = null;
+
+  private static final String FIELD_NAME = "geoField";
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    directory = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), directory,
+            newIndexWriterConfig(new MockAnalyzer(random()))
+                    .setMaxBufferedDocs(TestUtil.nextInt(random(), 100, 1000))
+                    .setMergePolicy(newLogMergePolicy()));
+
+    // create some simple geo points
+    final FieldType storedPoint = new FieldType(GeoPointField.TYPE_STORED);
+    // this is a simple systematic test
+    GeoPointField[] pts = new GeoPointField[] {
+         new GeoPointField(FIELD_NAME, -96.4538113027811, 32.94823588839368, storedPoint),
+         new GeoPointField(FIELD_NAME, -96.7759895324707, 32.7559529921407, storedPoint),
+         new GeoPointField(FIELD_NAME, -96.77701950073242, 32.77866942010977, storedPoint),
+         new GeoPointField(FIELD_NAME, -96.7706036567688, 32.7756745755423, storedPoint),
+         new GeoPointField(FIELD_NAME, -139.73458170890808, 27.703618681345585, storedPoint),
+         new GeoPointField(FIELD_NAME, -96.65084838867188, 33.06047141970814, storedPoint),
+         new GeoPointField(FIELD_NAME, -96.7772, 32.778650, storedPoint)};
+
+    for (GeoPointField p : pts) {
+        Document doc = new Document();
+        doc.add(p);
+        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 TopDocs bboxQuery(double minLon, double minLat, double maxLon, double maxLat, int limit) throws Exception {
+    GeoPointInBBoxQuery q = new GeoPointInBBoxQuery(FIELD_NAME, minLon, minLat, maxLon, maxLat);
+    return searcher.search(q, limit);
+  }
+
+  private TopDocs polygonQuery(double[] lon, double[] lat, int limit) throws Exception {
+    GeoPointInPolygonQuery q = GeoPointInPolygonQuery.newPolygonQuery(FIELD_NAME, lon, lat);
+    return searcher.search(q, limit);
+  }
+
+  @Test
+  public void testBBoxQuery() throws Exception {
+    TopDocs td = bboxQuery(-96.7772, 32.778650, -96.77690000, 32.778950, 5);
+    assertEquals("GeoBoundingBoxQuery failed", 2, td.totalHits);
+  }
+
+  @Test
+  public void testPolyQuery() throws Exception {
+    TopDocs td = polygonQuery( new double[] { -96.7682647, -96.8280029, -96.6288757, -96.4929199,
+        -96.6041564, -96.7449188, -96.76826477}, new double[] { 33.073130, 32.9942669, 32.938386, 33.0374494,
+        33.1369762, 33.1162747, 33.073130}, 5);
+    assertEquals("GeoPolygonQuery failed", td.totalHits, 1);
+  }
+
+  public void testRandomTiny() throws Exception {
+    // Make sure single-leaf-node case is OK:
+    doTestRandom(10);
+  }
+
+  public void testRandom() throws Exception {
+    doTestRandom(10000);
+  }
+
+  @Nightly
+  public void testRandomBig() throws Exception {
+    doTestRandom(1000000);
+  }
+
+  private void doTestRandom(int count) throws Exception {
+
+    int numPoints = atLeast(count);
+
+    if (VERBOSE) {
+      System.out.println("TEST: numPoints=" + numPoints);
+    }
+
+    double[] lats = new double[numPoints];
+    double[] lons = new double[numPoints];
+
+    boolean haveRealDoc = false;
+
+    for (int docID=0;docID<numPoints;docID++) {
+      int x = random().nextInt(20);
+      if (x == 17) {
+        // Some docs don't have a point:
+        lats[docID] = Double.NaN;
+        if (VERBOSE) {
+          //System.out.println("  doc=" + docID + " is missing");
+        }
+        continue;
+      }
+
+      if (docID > 0 && x < 3 && haveRealDoc) {
+        int oldDocID;
+        while (true) {
+          oldDocID = random().nextInt(docID);
+          if (Double.isNaN(lats[oldDocID]) == false) {
+            break;
+          }
+        }
+
+        if (x == 0) {
+          // Identical lat to old point
+          lats[docID] = lats[oldDocID];
+          lons[docID] = randomLon();
+          if (VERBOSE) {
+            //System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + lons[docID] + " (same lat as doc=" + oldDocID + ")");
+          }
+        } else if (x == 1) {
+          // Identical lon to old point
+          lats[docID] = randomLat();
+          lons[docID] = lons[oldDocID];
+          if (VERBOSE) {
+            //System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + lons[docID] + " (same lon as doc=" + oldDocID + ")");
+          }
+        } else {
+          assert x == 2;
+          // Fully identical point:
+          lats[docID] = lats[oldDocID];
+          lons[docID] = lons[oldDocID];
+          if (VERBOSE) {
+            //System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + lons[docID] + " (same lat/lon as doc=" + oldDocID + ")");
+          }
+        }
+      } else {
+        lats[docID] = randomLat();
+        lons[docID] = randomLon();
+        haveRealDoc = true;
+        if (VERBOSE) {
+          //System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + lons[docID]);
+        }
+      }
+    }
+
+    verify(lats, lons);
+  }
+
+  private static void verify(double[] lats, double[] lons) throws IOException {
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    Directory dir;
+    if (lats.length > 100000) {
+      dir = newFSDirectory(createTempDir("TestGeoPointQuery"));
+    } else {
+      dir = newDirectory();
+    }
+    Set<Integer> deleted = new HashSet<>();
+    // RandomIndexWriter is too slow here:
+    IndexWriter w = new IndexWriter(dir, iwc);
+    for(int id=0;id<lats.length;id++) {
+      Document doc = new Document();
+      doc.add(newStringField("id", ""+id, Field.Store.NO));
+      doc.add(new NumericDocValuesField("id", id));
+      if (Double.isNaN(lats[id]) == false) {
+        if (VERBOSE) {
+          System.out.println("  id=" + id + " lat=" + lats[id] + " lon=" + lons[id]);
+        }
+        doc.add(new GeoPointField("point", lons[id], lats[id], Field.Store.NO));
+      } else if (VERBOSE) {
+        System.out.println("  id=" + id + " skipped");
+      }
+      w.addDocument(doc);
+      if (id > 0 && random().nextInt(100) == 42) {
+        int idToDelete = random().nextInt(id);
+        w.deleteDocuments(new Term("id", ""+idToDelete));
+        deleted.add(idToDelete);
+        if (VERBOSE) {
+          System.out.println("  delete id=" + idToDelete);
+        }
+      }
+    }
+    if (random().nextBoolean()) {
+      w.forceMerge(1);
+    }
+    IndexReader r = DirectoryReader.open(w, true);
+    w.close();
+
+    // We can't wrap with "exotic" readers because the BKD query must see the BKDDVFormat:
+    IndexSearcher s = newSearcher(r, false);
+
+    NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
+
+    int iters = atLeast(100);
+    for (int iter=0;iter<iters;iter++) {
+      double lat0 = randomLat();
+      double lat1 = randomLat();
+      double lon0 = randomLon();
+      double lon1 = randomLon();
+
+      if (lat1 < lat0) {
+        double x = lat0;
+        lat0 = lat1;
+        lat1 = x;
+      }
+
+      if (lon1 < lon0) {
+        double x = lon0;
+        lon0 = lon1;
+        lon1 = x;
+      }
+
+      if (VERBOSE) {
+        System.out.println("\nTEST: iter=" + iter + " lat=" + lat0 + " TO " + lat1 + " lon=" + lon0 + " TO " + lon1);
+      }
+
+      // nocommit test "in polygon" query too!
+      Query query = new GeoPointInBBoxQuery("point", lon0, lat0, lon1, lat1);
+
+      final FixedBitSet hits = new FixedBitSet(r.maxDoc());
+      s.search(query, new SimpleCollector() {
+
+          private int docBase;
+
+          @Override
+          public boolean needsScores() {
+            return false;
+          }
+
+          @Override
+          protected void doSetNextReader(LeafReaderContext context) throws IOException {
+            docBase = context.docBase;
+          }
+
+          @Override
+          public void collect(int doc) {
+            hits.set(docBase+doc);
+          }
+        });
+
+      for(int docID=0;docID<r.maxDoc();docID++) {
+        int id = (int) docIDToID.get(docID);
+        boolean expected = deleted.contains(id) == false && rectContainsPointEnc(lat0, lat1, lon0, lon1, lats[id], lons[id]);
+        if (hits.get(docID) != expected) {
+          fail("id=" + id + " docID=" + docID + " lat=" + lats[id] + " lon=" + lons[id] + " expected " + expected + " but got: " + hits.get(docID) + " deleted?=" + deleted.contains(id));
+        }
+      }
+    }
+    IOUtils.close(r, dir);
+  }
+
+  private static boolean rectContainsPointEnc(double rectLatMin, double rectLatMax,
+                                              double rectLonMin, double rectLonMax,
+                                              double pointLat, double pointLon) {
+    if (Double.isNaN(pointLat)) {
+      return false;
+    }
+    /*
+    int rectLatMinEnc = BKDTreeWriter.encodeLat(rectLatMin);
+    int rectLatMaxEnc = BKDTreeWriter.encodeLat(rectLatMax);
+    int rectLonMinEnc = BKDTreeWriter.encodeLon(rectLonMin);
+    int rectLonMaxEnc = BKDTreeWriter.encodeLon(rectLonMax);
+    int pointLatEnc = BKDTreeWriter.encodeLat(pointLat);
+    int pointLonEnc = BKDTreeWriter.encodeLon(pointLon);
+
+    return pointLatEnc >= rectLatMinEnc &&
+      pointLatEnc < rectLatMaxEnc &&
+      pointLonEnc >= rectLonMinEnc &&
+      pointLonEnc < rectLonMaxEnc;
+    */
+    return GeoUtils.bboxContains(pointLon, pointLat, rectLonMin, rectLatMin, rectLonMax, rectLatMax);
+  }
+
+  private static double randomLat() {
+    return -90 + 180.0 * random().nextDouble();
+  }
+
+  private static double randomLon() {
+    return -180 + 360.0 * random().nextDouble();
+  }
+
+}