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/11/08 19:18:20 UTC

svn commit: r1713278 [2/3] - in /lucene/dev/trunk/lucene: ./ backward-codecs/src/java/org/apache/lucene/codecs/lucene54/ codecs/src/java/org/apache/lucene/codecs/simpletext/ core/src/java/org/apache/lucene/codecs/ core/src/java/org/apache/lucene/codecs...

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java?rev=1713278&r1=1713277&r2=1713278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/bkd/TestBKD.java Sun Nov  8 18:18:19 2015
@@ -91,11 +91,11 @@ public class TestBKD extends LuceneTestC
               }
 
               if (max < queryMin || min > queryMax) {
-                return Relation.QUERY_OUTSIDE_CELL;
+                return Relation.CELL_OUTSIDE_QUERY;
               } else if (min >= queryMin && max <= queryMax) {
                 return Relation.CELL_INSIDE_QUERY;
               } else {
-                return Relation.QUERY_CROSSES_CELL;
+                return Relation.CELL_CROSSES_QUERY;
               }
             }
           });
@@ -198,14 +198,14 @@ public class TestBKD extends LuceneTestC
                 assert max >= min;
 
                 if (max < queryMin[dim] || min > queryMax[dim]) {
-                  return Relation.QUERY_OUTSIDE_CELL;
+                  return Relation.CELL_OUTSIDE_QUERY;
                 } else if (min < queryMin[dim] || max > queryMax[dim]) {
                   crosses = true;
                 }
               }
 
               if (crosses) {
-                return Relation.QUERY_CROSSES_CELL;
+                return Relation.CELL_CROSSES_QUERY;
               } else {
                 return Relation.CELL_INSIDE_QUERY;
               }
@@ -319,14 +319,14 @@ public class TestBKD extends LuceneTestC
                 assert max.compareTo(min) >= 0;
 
                 if (max.compareTo(queryMin[dim]) < 0 || min.compareTo(queryMax[dim]) > 0) {
-                  return Relation.QUERY_OUTSIDE_CELL;
+                  return Relation.CELL_OUTSIDE_QUERY;
                 } else if (min.compareTo(queryMin[dim]) < 0 || max.compareTo(queryMax[dim]) > 0) {
                   crosses = true;
                 }
               }
 
               if (crosses) {
-                return Relation.QUERY_CROSSES_CELL;
+                return Relation.CELL_CROSSES_QUERY;
               } else {
                 return Relation.CELL_INSIDE_QUERY;
               }
@@ -517,6 +517,87 @@ public class TestBKD extends LuceneTestC
     verify(docValuesArray, docIDsArray, numDims, numBytesPerDim);
   }
 
+  public void testBKDUtilAdd() throws Exception {
+    int iters = atLeast(10000);
+    int numBytes = TestUtil.nextInt(random(), 1, 100);
+    for(int iter=0;iter<iters;iter++) {
+      BigInteger v1 = new BigInteger(8*numBytes-1, random());
+      BigInteger v2 = new BigInteger(8*numBytes-1, random());
+
+      byte[] v1Bytes = new byte[numBytes];
+      byte[] v1RawBytes = v1.toByteArray();
+      assert v1RawBytes.length <= numBytes;
+      System.arraycopy(v1RawBytes, 0, v1Bytes, v1Bytes.length-v1RawBytes.length, v1RawBytes.length);
+
+      byte[] v2Bytes = new byte[numBytes];
+      byte[] v2RawBytes = v2.toByteArray();
+      assert v1RawBytes.length <= numBytes;
+      System.arraycopy(v2RawBytes, 0, v2Bytes, v2Bytes.length-v2RawBytes.length, v2RawBytes.length);
+
+      byte[] result = new byte[numBytes];
+      BKDUtil.add(numBytes, 0, v1Bytes, v2Bytes, result);
+
+      BigInteger sum = v1.add(v2);
+      assertTrue("sum=" + sum + " v1=" + v1 + " v2=" + v2 + " but result=" + new BigInteger(1, result), sum.equals(new BigInteger(1, result)));
+    }
+  }
+
+  public void testIllegalBKDUtilAdd() throws Exception {
+    byte[] bytes = new byte[4];
+    Arrays.fill(bytes, (byte) 0xff);
+    byte[] one = new byte[4];
+    one[3] = 1;
+    try {
+      BKDUtil.add(4, 0, bytes, one, new byte[4]);
+    } catch (IllegalArgumentException iae) {
+      assertEquals("a + b overflows bytesPerDim=4", iae.getMessage());
+    }
+  }
+  
+  public void testBKDUtilSubtract() throws Exception {
+    int iters = atLeast(10000);
+    int numBytes = TestUtil.nextInt(random(), 1, 100);
+    for(int iter=0;iter<iters;iter++) {
+      BigInteger v1 = new BigInteger(8*numBytes-1, random());
+      BigInteger v2 = new BigInteger(8*numBytes-1, random());
+      if (v1.compareTo(v2) < 0) {
+        BigInteger tmp = v1;
+        v1 = v2;
+        v2 = tmp;
+      }
+
+      byte[] v1Bytes = new byte[numBytes];
+      byte[] v1RawBytes = v1.toByteArray();
+      assert v1RawBytes.length <= numBytes: "length=" + v1RawBytes.length + " vs numBytes=" + numBytes;
+      System.arraycopy(v1RawBytes, 0, v1Bytes, v1Bytes.length-v1RawBytes.length, v1RawBytes.length);
+
+      byte[] v2Bytes = new byte[numBytes];
+      byte[] v2RawBytes = v2.toByteArray();
+      assert v2RawBytes.length <= numBytes;
+      assert v2RawBytes.length <= numBytes: "length=" + v2RawBytes.length + " vs numBytes=" + numBytes;
+      System.arraycopy(v2RawBytes, 0, v2Bytes, v2Bytes.length-v2RawBytes.length, v2RawBytes.length);
+
+      byte[] result = new byte[numBytes];
+      BKDUtil.subtract(numBytes, 0, v1Bytes, v2Bytes, result);
+
+      BigInteger diff = v1.subtract(v2);
+
+      assertTrue("diff=" + diff + " vs result=" + new BigInteger(result) + " v1=" + v1 + " v2=" + v2, diff.equals(new BigInteger(result)));
+    }
+  }
+
+  public void testIllegalBKDUtilSubtract() throws Exception {
+    byte[] v1 = new byte[4];
+    v1[3] = (byte) 0xf0;
+    byte[] v2 = new byte[4];
+    v2[3] = (byte) 0xf1;
+    try {
+      BKDUtil.subtract(4, 0, v1, v2, new byte[4]);
+    } catch (IllegalArgumentException iae) {
+      assertEquals("a < b", iae.getMessage());
+    }
+  }
+  
   /** docIDs can be null, for the single valued case, else it maps value to docID */
   private void verify(byte[][][] docValues, int[] docIDs, int numDims, int numBytesPerDim) throws Exception {
     try (Directory dir = getDirectory(docValues.length)) {
@@ -627,7 +708,7 @@ public class TestBKD extends LuceneTestC
               for(int dim=0;dim<numDims;dim++) {
                 if (BKDUtil.compare(numBytesPerDim, maxPacked, dim, queryMin[dim], 0) < 0 ||
                     BKDUtil.compare(numBytesPerDim, minPacked, dim, queryMax[dim], 0) > 0) {
-                  return Relation.QUERY_OUTSIDE_CELL;
+                  return Relation.CELL_OUTSIDE_QUERY;
                 } else if (BKDUtil.compare(numBytesPerDim, minPacked, dim, queryMin[dim], 0) < 0 ||
                            BKDUtil.compare(numBytesPerDim, maxPacked, dim, queryMax[dim], 0) > 0) {
                   crosses = true;
@@ -635,7 +716,7 @@ public class TestBKD extends LuceneTestC
               }
 
               if (crosses) {
-                return Relation.QUERY_CROSSES_CELL;
+                return Relation.CELL_CROSSES_QUERY;
               } else {
                 return Relation.CELL_INSIDE_QUERY;
               }

Modified: lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java?rev=1713278&r1=1713277&r2=1713278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java (original)
+++ lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java Sun Nov  8 18:18:19 2015
@@ -32,8 +32,8 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.DimensionalField;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
@@ -395,7 +395,7 @@ public abstract class SorterTestBase ext
 
                        @Override
                        public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-                         return Relation.QUERY_CROSSES_CELL;
+                         return Relation.CELL_CROSSES_QUERY;
                        }
                      });
   }

Copied: lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/document/DimensionalLatLonField.java (from r1713275, lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointField.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/document/DimensionalLatLonField.java?p2=lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/document/DimensionalLatLonField.java&p1=lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointField.java&r1=1713275&r2=1713278&rev=1713278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointField.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/document/DimensionalLatLonField.java Sun Nov  8 18:18:19 2015
@@ -1,4 +1,4 @@
-package org.apache.lucene.bkdtree;
+package org.apache.lucene.document;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,34 +17,72 @@ package org.apache.lucene.bkdtree;
  * limitations under the License.
  */
 
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.GeoUtils;
+import org.apache.lucene.util.bkd.BKDUtil;
 
-/** Add this to a document to index lat/lon point, but be sure to use {@link BKDTreeDocValuesFormat} for the field. */
-public final class BKDPointField extends Field {
+/** Add this to a document to index lat/lon point dimensionally */
+public final class DimensionalLatLonField extends Field {
 
   public static final FieldType TYPE = new FieldType();
   static {
-    TYPE.setDocValuesType(DocValuesType.SORTED_NUMERIC);
+    TYPE.setDimensions(2, 4);
     TYPE.freeze();
   }
 
   /** 
-   * Creates a new BKDPointField field with the specified lat and lon
+   * Creates a new DimensionalLatLonField with the specified lat and lon
    * @param name field name
    * @param lat double latitude
    * @param lon double longitude
    * @throws IllegalArgumentException if the field name is null or lat or lon are out of bounds
    */
-  public BKDPointField(String name, double lat, double lon) {
+  public DimensionalLatLonField(String name, double lat, double lon) {
     super(name, TYPE);
-    if (BKDTreeWriter.validLat(lat) == false) {
+    if (GeoUtils.isValidLat(lat) == false) {
       throw new IllegalArgumentException("invalid lat (" + lat + "): must be -90 to 90");
     }
-    if (BKDTreeWriter.validLon(lon) == false) {
+    if (GeoUtils.isValidLon(lon) == false) {
       throw new IllegalArgumentException("invalid lon (" + lon + "): must be -180 to 180");
     }
-    fieldsData = Long.valueOf(((long) BKDTreeWriter.encodeLat(lat) << 32) | (BKDTreeWriter.encodeLon(lon) & 0xffffffffL));
+    byte[] bytes = new byte[8];
+    BKDUtil.intToBytes(encodeLat(lat), bytes, 0);
+    BKDUtil.intToBytes(encodeLon(lon), bytes, 1);
+    fieldsData = new BytesRef(bytes);
+  }
+
+  public static final double TOLERANCE = 1E-7;
+
+  private static final int BITS = 32;
+
+  private static final double LON_SCALE = (0x1L<<BITS)/360.0D;
+  private static final double LAT_SCALE = (0x1L<<BITS)/180.0D;
+
+  /** Quantizes double (64 bit) latitude into 32 bits */
+  public static int encodeLat(double lat) {
+    assert GeoUtils.isValidLat(lat): "lat=" + lat;
+    long x = (long) (lat * LAT_SCALE);
+    assert x < Integer.MAX_VALUE: "lat=" + lat + " mapped to Integer.MAX_VALUE + " + (x - Integer.MAX_VALUE);
+    assert x > Integer.MIN_VALUE: "lat=" + lat + " mapped to Integer.MIN_VALUE";
+    return (int) x;
+  }
+
+  /** Quantizes double (64 bit) longitude into 32 bits */
+  public static int encodeLon(double lon) {
+    assert GeoUtils.isValidLon(lon): "lon=" + lon;
+    long x = (long) (lon * LON_SCALE);
+    assert x < Integer.MAX_VALUE;
+    assert x > Integer.MIN_VALUE;
+    return (int) x;
+  }
+
+  /** Turns quantized value from {@link #encodeLat} back into a double. */
+  public static double decodeLat(int x) {
+    return x / LAT_SCALE;
+  }
+
+  /** Turns quantized value from {@link #encodeLon} back into a double. */
+  public static double decodeLon(int x) {
+    return x / LON_SCALE;
   }
 }

Copied: lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/DimensionalPointInPolygonQuery.java (from r1711921, lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInPolygonQuery.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/DimensionalPointInPolygonQuery.java?p2=lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/DimensionalPointInPolygonQuery.java&p1=lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInPolygonQuery.java&r1=1711921&r2=1713278&rev=1713278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInPolygonQuery.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/DimensionalPointInPolygonQuery.java Sun Nov  8 18:18:19 2015
@@ -1,4 +1,4 @@
-package org.apache.lucene.bkdtree;
+package org.apache.lucene.search;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,35 +19,24 @@ package org.apache.lucene.bkdtree;
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.Set;
 
+import org.apache.lucene.document.DimensionalLatLonField;
+import org.apache.lucene.index.DimensionalValues;
+import org.apache.lucene.index.DimensionalValues.IntersectVisitor;
+import org.apache.lucene.index.DimensionalValues.Relation;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.search.ConstantScoreScorer;
-import org.apache.lucene.search.ConstantScoreWeight;
-import org.apache.lucene.search.ConstantScoreScorer;
-import org.apache.lucene.search.ConstantScoreWeight;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.DocIdSetBuilder;
 import org.apache.lucene.util.GeoUtils;
+import org.apache.lucene.util.bkd.BKDUtil;
 
 /** Finds all previously indexed points that fall within the specified polygon.
  *
- *  <p>The field must be indexed with {@link BKDTreeDocValuesFormat}, and {@link BKDPointField} added per document.
+ *  <p>The field must be indexed with using {@link DimensionalLatLonField} added per document.
  *
- *  <p>Because this implementation cannot intersect each cell with the polygon, it will be costly especially for large polygons, as every
- *   possible point must be checked.
- *
- *  <p><b>NOTE</b>: for fastest performance, this allocates FixedBitSet(maxDoc) for each segment.  The score of each hit is the query boost.
- *
- * @lucene.experimental */
+ *  @lucene.experimental */
 
-public class BKDPointInPolygonQuery extends Query {
+public class DimensionalPointInPolygonQuery extends Query {
   final String field;
   final double minLat;
   final double maxLat;
@@ -57,7 +46,7 @@ public class BKDPointInPolygonQuery exte
   final double[] polyLons;
 
   /** The lats/lons must be clockwise or counter-clockwise. */
-  public BKDPointInPolygonQuery(String field, double[] polyLats, double[] polyLons) {
+  public DimensionalPointInPolygonQuery(String field, double[] polyLats, double[] polyLons) {
     this.field = field;
     if (polyLats.length != polyLons.length) {
       throw new IllegalArgumentException("polyLats and polyLons must be equal length");
@@ -83,13 +72,13 @@ public class BKDPointInPolygonQuery exte
     double maxLat = Double.NEGATIVE_INFINITY;
     for(int i=0;i<polyLats.length;i++) {
       double lat = polyLats[i];
-      if (BKDTreeWriter.validLat(lat) == false) {
+      if (GeoUtils.isValidLat(lat) == false) {
         throw new IllegalArgumentException("polyLats[" + i + "]=" + lat + " is not a valid latitude");
       }
       minLat = Math.min(minLat, lat);
       maxLat = Math.max(maxLat, lat);
       double lon = polyLons[i];
-      if (BKDTreeWriter.validLon(lon) == false) {
+      if (GeoUtils.isValidLon(lon) == false) {
         throw new IllegalArgumentException("polyLons[" + i + "]=" + lat + " is not a valid longitude");
       }
       minLon = Math.min(minLon, lon);
@@ -115,42 +104,59 @@ public class BKDPointInPolygonQuery exte
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
         LeafReader reader = context.reader();
-        SortedNumericDocValues sdv = reader.getSortedNumericDocValues(field);
-        if (sdv == null) {
-          // No docs in this segment had this field
+        DimensionalValues values = reader.getDimensionalValues();
+        if (values == null) {
+          // No docs in this segment had any dimensional fields
           return null;
         }
 
-        if (sdv instanceof BKDTreeSortedNumericDocValues == false) {
-          throw new IllegalStateException("field \"" + field + "\" was not indexed with BKDTreeDocValuesFormat: got: " + sdv);
-        }
-        BKDTreeSortedNumericDocValues treeDV = (BKDTreeSortedNumericDocValues) sdv;
-        BKDTreeReader tree = treeDV.getBKDTreeReader();
-        
-        DocIdSet result = tree.intersect(minLat, maxLat, minLon, maxLon,
-                                         new BKDTreeReader.LatLonFilter() {
-                                           @Override
-                                           public boolean accept(double lat, double lon) {
-                                             return GeoUtils.pointInPolygon(polyLons, polyLats, lat, lon);
-                                           }
-
-                                           @Override
-                                           public BKDTreeReader.Relation compare(double cellLatMin, double cellLatMax, double cellLonMin, double cellLonMax) {
-                                             if (GeoUtils.rectWithinPoly(cellLonMin, cellLatMin, cellLonMax, cellLatMax,
-                                                                         polyLons, polyLats,
-                                                                         minLon, minLat, maxLon, maxLat)) {
-                                               return BKDTreeReader.Relation.CELL_INSIDE_SHAPE;
-                                             } else if (GeoUtils.rectCrossesPoly(cellLonMin, cellLatMin, cellLonMax, cellLatMax,
-                                                                                 polyLons, polyLats,
-                                                                                 minLon, minLat, maxLon, maxLat)) {
-                                               return BKDTreeReader.Relation.SHAPE_CROSSES_CELL;
-                                             } else {
-                                               return BKDTreeReader.Relation.SHAPE_OUTSIDE_CELL;
-                                             }
-                                           }
-                                         }, treeDV.delegate);
+        DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc());
+        int[] hitCount = new int[1];
+        values.intersect(field,
+                         new IntersectVisitor() {
+                           @Override
+                           public void visit(int docID) {
+                             hitCount[0]++;
+                             result.add(docID);
+                           }
+
+                           @Override
+                           public void visit(int docID, byte[] packedValue) {
+                             assert packedValue.length == 8;
+                             double lat = DimensionalLatLonField.decodeLat(BKDUtil.bytesToInt(packedValue, 0));
+                             double lon = DimensionalLatLonField.decodeLon(BKDUtil.bytesToInt(packedValue, 1));
+                             if (GeoUtils.pointInPolygon(polyLons, polyLats, lat, lon)) {
+                               hitCount[0]++;
+                               result.add(docID);
+                             }
+                           }
+
+                           @Override
+                           public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                             double cellMinLat = DimensionalLatLonField.decodeLat(BKDUtil.bytesToInt(minPackedValue, 0));
+                             double cellMinLon = DimensionalLatLonField.decodeLon(BKDUtil.bytesToInt(minPackedValue, 1));
+                             double cellMaxLat = DimensionalLatLonField.decodeLat(BKDUtil.bytesToInt(maxPackedValue, 0));
+                             double cellMaxLon = DimensionalLatLonField.decodeLon(BKDUtil.bytesToInt(maxPackedValue, 1));
+
+                             if (cellMinLat <= minLat && cellMaxLat >= maxLat && cellMinLon <= minLon && cellMaxLon >= maxLon) {
+                               // Cell fully encloses the query
+                               return Relation.CELL_CROSSES_QUERY;
+                             } else  if (GeoUtils.rectWithinPoly(cellMinLon, cellMinLat, cellMaxLon, cellMaxLat,
+                                                                 polyLons, polyLats,
+                                                                 minLon, minLat, maxLon, maxLat)) {
+                               return Relation.CELL_INSIDE_QUERY;
+                             } else if (GeoUtils.rectCrossesPoly(cellMinLon, cellMinLat, cellMaxLon, cellMaxLat,
+                                                                 polyLons, polyLats,
+                                                                 minLon, minLat, maxLon, maxLat)) {
+                               return Relation.CELL_CROSSES_QUERY;
+                             } else {
+                               return Relation.CELL_OUTSIDE_QUERY;
+                             }
+                           }
+                         });
 
-        return new ConstantScoreScorer(this, score(), result.iterator());
+        // NOTE: hitCount[0] will be over-estimate in multi-valued case
+        return new ConstantScoreScorer(this, score(), result.build(hitCount[0]).iterator());
       }
     };
   }
@@ -162,7 +168,7 @@ public class BKDPointInPolygonQuery exte
     if (o == null || getClass() != o.getClass()) return false;
     if (!super.equals(o)) return false;
 
-    BKDPointInPolygonQuery that = (BKDPointInPolygonQuery) o;
+    DimensionalPointInPolygonQuery that = (DimensionalPointInPolygonQuery) o;
 
     if (Arrays.equals(polyLons, that.polyLons) == false) {
       return false;

Copied: lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/DimensionalPointInRectQuery.java (from r1711921, lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInBBoxQuery.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/DimensionalPointInRectQuery.java?p2=lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/DimensionalPointInRectQuery.java&p1=lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInBBoxQuery.java&r1=1711921&r2=1713278&rev=1713278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/bkdtree/BKDPointInBBoxQuery.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/DimensionalPointInRectQuery.java Sun Nov  8 18:18:19 2015
@@ -1,4 +1,4 @@
-package org.apache.lucene.bkdtree;
+package org.apache.lucene.search;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -18,33 +18,25 @@ package org.apache.lucene.bkdtree;
  */
 
 import java.io.IOException;
-import java.util.Set;
 
+import org.apache.lucene.document.DimensionalLatLonField;
+import org.apache.lucene.index.DimensionalValues;
+import org.apache.lucene.index.DimensionalValues.IntersectVisitor;
+import org.apache.lucene.index.DimensionalValues.Relation;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.ConstantScoreScorer;
-import org.apache.lucene.search.ConstantScoreWeight;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.util.GeoUtils;
+import org.apache.lucene.util.bkd.BKDUtil;
 
 /** Finds all previously indexed points that fall within the specified boundings box.
  *
- *  <p>The field must be indexed with {@link BKDTreeDocValuesFormat}, and {@link BKDPointField} added per document.
+ *  <p>The field must be indexed with using {@link DimensionalLatLonField} added per document.
  *
- *  <p><b>NOTE</b>: for fastest performance, this allocates FixedBitSet(maxDoc) for each segment.  The score of each hit is the query boost.
- *
- * @lucene.experimental */
+ *  @lucene.experimental */
 
-public class BKDPointInBBoxQuery extends Query {
+public class DimensionalPointInRectQuery extends Query {
   final String field;
   final double minLat;
   final double maxLat;
@@ -52,18 +44,18 @@ public class BKDPointInBBoxQuery extends
   final double maxLon;
 
   /** Matches all points &gt;= minLon, minLat (inclusive) and &lt; maxLon, maxLat (exclusive). */ 
-  public BKDPointInBBoxQuery(String field, double minLat, double maxLat, double minLon, double maxLon) {
+  public DimensionalPointInRectQuery(String field, double minLat, double maxLat, double minLon, double maxLon) {
     this.field = field;
-    if (BKDTreeWriter.validLat(minLat) == false) {
+    if (GeoUtils.isValidLat(minLat) == false) {
       throw new IllegalArgumentException("minLat=" + minLat + " is not a valid latitude");
     }
-    if (BKDTreeWriter.validLat(maxLat) == false) {
+    if (GeoUtils.isValidLat(maxLat) == false) {
       throw new IllegalArgumentException("maxLat=" + maxLat + " is not a valid latitude");
     }
-    if (BKDTreeWriter.validLon(minLon) == false) {
+    if (GeoUtils.isValidLon(minLon) == false) {
       throw new IllegalArgumentException("minLon=" + minLon + " is not a valid longitude");
     }
-    if (BKDTreeWriter.validLon(maxLon) == false) {
+    if (GeoUtils.isValidLon(maxLon) == false) {
       throw new IllegalArgumentException("maxLon=" + maxLon + " is not a valid longitude");
     }
     this.minLon = minLon;
@@ -82,21 +74,59 @@ public class BKDPointInBBoxQuery extends
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
         LeafReader reader = context.reader();
-        SortedNumericDocValues sdv = reader.getSortedNumericDocValues(field);
-        if (sdv == null) {
-          // No docs in this segment had this field
+        DimensionalValues values = reader.getDimensionalValues();
+        if (values == null) {
+          // No docs in this segment had any dimensional fields
           return null;
         }
 
-        if (sdv instanceof BKDTreeSortedNumericDocValues == false) {
-          throw new IllegalStateException("field \"" + field + "\" was not indexed with BKDTreeDocValuesFormat: got: " + sdv);
-        }
-        BKDTreeSortedNumericDocValues treeDV = (BKDTreeSortedNumericDocValues) sdv;
-        BKDTreeReader tree = treeDV.getBKDTreeReader();
-
-        DocIdSet result = tree.intersect(minLat, maxLat, minLon, maxLon, null, treeDV.delegate);
+        DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc());
+        int[] hitCount = new int[1];
+        values.intersect(field,
+                         new IntersectVisitor() {
+                           @Override
+                           public void grow(int count) {
+                             result.grow(count);
+                           }
+
+                           @Override
+                           public void visit(int docID) {
+                             hitCount[0]++;
+                             result.add(docID);
+                           }
+
+                           @Override
+                           public void visit(int docID, byte[] packedValue) {
+                             assert packedValue.length == 8;
+                             double lat = DimensionalLatLonField.decodeLat(BKDUtil.bytesToInt(packedValue, 0));
+                             double lon = DimensionalLatLonField.decodeLon(BKDUtil.bytesToInt(packedValue, 1));
+                             if (lat >= minLat && lat <= maxLat && lon >= minLon && lon <= maxLon) {
+                               hitCount[0]++;
+                               result.add(docID);
+                             }
+                           }
+
+                           @Override
+                           public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                             double cellMinLat = DimensionalLatLonField.decodeLat(BKDUtil.bytesToInt(minPackedValue, 0));
+                             double cellMinLon = DimensionalLatLonField.decodeLon(BKDUtil.bytesToInt(minPackedValue, 1));
+                             double cellMaxLat = DimensionalLatLonField.decodeLat(BKDUtil.bytesToInt(maxPackedValue, 0));
+                             double cellMaxLon = DimensionalLatLonField.decodeLon(BKDUtil.bytesToInt(maxPackedValue, 1));
+
+                             if (minLat <= cellMinLat && maxLat >= cellMaxLat && minLon <= cellMinLon && maxLon >= cellMaxLon) {
+                               return Relation.CELL_INSIDE_QUERY;
+                             }
+
+                             if (cellMaxLat < minLat || cellMinLat > maxLat || cellMaxLon < minLon || cellMinLon > maxLon) {
+                               return Relation.CELL_OUTSIDE_QUERY;
+                             }
+
+                             return Relation.CELL_CROSSES_QUERY;
+                           }
+                         });
 
-        return new ConstantScoreScorer(this, score(), result.iterator());
+        // NOTE: hitCount[0] will be over-estimate in multi-valued case
+        return new ConstantScoreScorer(this, score(), result.build(hitCount[0]).iterator());
       }
     };
   }
@@ -111,9 +141,9 @@ public class BKDPointInBBoxQuery extends
       q.setDisableCoord(true);
 
       // E.g.: maxLon = -179, minLon = 179
-      BKDPointInBBoxQuery left = new BKDPointInBBoxQuery(field, minLat, maxLat, BKDTreeWriter.MIN_LON_INCL, maxLon);
+      DimensionalPointInRectQuery left = new DimensionalPointInRectQuery(field, minLat, maxLat, GeoUtils.MIN_LON_INCL, maxLon);
       q.add(new BooleanClause(left, BooleanClause.Occur.SHOULD));
-      BKDPointInBBoxQuery right = new BKDPointInBBoxQuery(field, minLat, maxLat, minLon, BKDTreeWriter.MAX_LON_INCL);
+      DimensionalPointInRectQuery right = new DimensionalPointInRectQuery(field, minLat, maxLat, minLon, GeoUtils.MAX_LON_INCL);
       q.add(new BooleanClause(right, BooleanClause.Occur.SHOULD));
       return new ConstantScoreQuery(q.build());
     } else {
@@ -133,8 +163,8 @@ public class BKDPointInBBoxQuery extends
 
   @Override
   public boolean equals(Object other) {
-    if (super.equals(other) && other instanceof BKDPointInBBoxQuery) {
-      final BKDPointInBBoxQuery q = (BKDPointInBBoxQuery) other;
+    if (super.equals(other) && other instanceof DimensionalPointInRectQuery) {
+      final DimensionalPointInRectQuery q = (DimensionalPointInRectQuery) other;
       return field.equals(q.field) &&
         minLat == q.minLat &&
         maxLat == q.maxLat &&

Copied: lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/search/TestDimensionalQueries.java (from r1711921, lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/search/TestDimensionalQueries.java?p2=lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/search/TestDimensionalQueries.java&p1=lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java&r1=1711921&r2=1713278&rev=1713278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/bkdtree/TestBKDTree.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/search/TestDimensionalQueries.java Sun Nov  8 18:18:19 2015
@@ -1,4 +1,4 @@
-package org.apache.lucene.bkdtree;
+package org.apache.lucene.search;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,37 +17,23 @@ package org.apache.lucene.bkdtree;
  * limitations under the License.
  */
 
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.lucene60.Lucene60Codec;
+import org.apache.lucene.document.DimensionalLatLonField;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriterConfig;
-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.util.Accountable;
-import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.BaseGeoPointTestCase;
 import org.apache.lucene.util.GeoRect;
-import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.SloppyMath;
-import org.apache.lucene.util.TestUtil;
 
-// TODO: can test framework assert we don't leak temp files?
-
-public class TestBKDTree extends BaseGeoPointTestCase {
+public class TestDimensionalQueries extends BaseGeoPointTestCase {
 
   @Override
   protected void addPointToDoc(String field, Document doc, double lat, double lon) {
-    doc.add(new BKDPointField(field, lat, lon));
+    doc.add(new DimensionalLatLonField(field, lat, lon));
   }
 
   @Override
-  protected Query newBBoxQuery(String field, GeoRect rect) {
-    return new BKDPointInBBoxQuery(field, rect.minLat, rect.maxLat, rect.minLon, rect.maxLon);
+  protected Query newRectQuery(String field, GeoRect rect) {
+    return new DimensionalPointInRectQuery(field, rect.minLat, rect.maxLat, rect.minLon, rect.maxLon);
   }
 
   @Override
@@ -63,49 +49,33 @@ public class TestBKDTree extends BaseGeo
 
   @Override
   protected Query newPolygonQuery(String field, double[] lats, double[] lons) {
-    return new BKDPointInPolygonQuery(FIELD_NAME, lats, lons);
+    return new DimensionalPointInPolygonQuery(FIELD_NAME, lats, lons);
   }
 
   @Override
-  protected void initIndexWriterConfig(final String fieldName, IndexWriterConfig iwc) {
-    final DocValuesFormat dvFormat = getDocValuesFormat();
-    Codec codec = new Lucene60Codec() {
-        @Override
-        public DocValuesFormat getDocValuesFormatForField(String field) {
-          if (field.equals(fieldName)) {
-            return dvFormat;
-          } else {
-            return super.getDocValuesFormatForField(field);
-          }
-        }
-      };
-    iwc.setCodec(codec);
-  }
-  
-  @Override
   protected Boolean rectContainsPoint(GeoRect rect, double pointLat, double pointLon) {
 
     assert Double.isNaN(pointLat) == false;
 
-    int rectLatMinEnc = BKDTreeWriter.encodeLat(rect.minLat);
-    int rectLatMaxEnc = BKDTreeWriter.encodeLat(rect.maxLat);
-    int rectLonMinEnc = BKDTreeWriter.encodeLon(rect.minLon);
-    int rectLonMaxEnc = BKDTreeWriter.encodeLon(rect.maxLon);
+    int rectLatMinEnc = DimensionalLatLonField.encodeLat(rect.minLat);
+    int rectLatMaxEnc = DimensionalLatLonField.encodeLat(rect.maxLat);
+    int rectLonMinEnc = DimensionalLatLonField.encodeLon(rect.minLon);
+    int rectLonMaxEnc = DimensionalLatLonField.encodeLon(rect.maxLon);
 
-    int pointLatEnc = BKDTreeWriter.encodeLat(pointLat);
-    int pointLonEnc = BKDTreeWriter.encodeLon(pointLon);
+    int pointLatEnc = DimensionalLatLonField.encodeLat(pointLat);
+    int pointLonEnc = DimensionalLatLonField.encodeLon(pointLon);
 
     if (rect.minLon < rect.maxLon) {
       return pointLatEnc >= rectLatMinEnc &&
-        pointLatEnc < rectLatMaxEnc &&
+        pointLatEnc <= rectLatMaxEnc &&
         pointLonEnc >= rectLonMinEnc &&
-        pointLonEnc < rectLonMaxEnc;
+        pointLonEnc <= rectLonMaxEnc;
     } else {
       // Rect crosses dateline:
       return pointLatEnc >= rectLatMinEnc &&
-        pointLatEnc < rectLatMaxEnc &&
+        pointLatEnc <= rectLatMaxEnc &&
         (pointLonEnc >= rectLonMinEnc ||
-         pointLonEnc < rectLonMaxEnc);
+         pointLonEnc <= rectLonMaxEnc);
     }
   }
 
@@ -143,52 +113,12 @@ public class TestBKDTree extends BaseGeo
     boolean small = random().nextBoolean();
     for(int iter=0;iter<iters;iter++) {
       double lat = randomLat(small);
-      double latQuantized = BKDTreeWriter.decodeLat(BKDTreeWriter.encodeLat(lat));
-      assertEquals(lat, latQuantized, BKDTreeWriter.TOLERANCE);
+      double latQuantized = DimensionalLatLonField.decodeLat(DimensionalLatLonField.encodeLat(lat));
+      assertEquals(lat, latQuantized, DimensionalLatLonField.TOLERANCE);
 
       double lon = randomLon(small);
-      double lonQuantized = BKDTreeWriter.decodeLon(BKDTreeWriter.encodeLon(lon));
-      assertEquals(lon, lonQuantized, BKDTreeWriter.TOLERANCE);
-    }
-  }
-
-  public void testEncodeDecodeMax() throws Exception {
-    int x = BKDTreeWriter.encodeLat(Math.nextAfter(90.0, Double.POSITIVE_INFINITY));
-    assertTrue(x < Integer.MAX_VALUE);
-
-    int y = BKDTreeWriter.encodeLon(Math.nextAfter(180.0, Double.POSITIVE_INFINITY));
-    assertTrue(y < Integer.MAX_VALUE);
-  }
-
-  public void testAccountableHasDelegate() throws Exception {
-    Directory dir = getDirectory();
-    IndexWriterConfig iwc = newIndexWriterConfig();
-    iwc.setCodec(TestUtil.alwaysDocValuesFormat(getDocValuesFormat()));
-    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
-    Document doc = new Document();
-    doc.add(new BKDPointField(FIELD_NAME, -18.2861, 147.7));
-    w.addDocument(doc);
-    IndexReader r = w.getReader();
-
-    // We can't wrap with "exotic" readers because the BKD query must see the BKDDVFormat:
-    IndexSearcher s = newSearcher(r, false);
-    // Need to run a query so the DV field is really loaded:
-    TopDocs hits = s.search(new BKDPointInBBoxQuery(FIELD_NAME, -30, 0, 140, 150), 1);
-    assertEquals(1, hits.totalHits);
-    assertTrue(Accountables.toString((Accountable) r.leaves().get(0).reader()).contains("delegate"));
-    IOUtils.close(r, w, dir);
-  }
-
-  private static DocValuesFormat getDocValuesFormat() {
-    int maxPointsInLeaf = TestUtil.nextInt(random(), 16, 2048);
-    int maxPointsSortInHeap = TestUtil.nextInt(random(), maxPointsInLeaf, 1024*1024);
-    if (VERBOSE) {
-      System.out.println("  BKD params: maxPointsInLeaf=" + maxPointsInLeaf + " maxPointsSortInHeap=" + maxPointsSortInHeap);
+      double lonQuantized = DimensionalLatLonField.decodeLon(DimensionalLatLonField.encodeLon(lon));
+      assertEquals(lon, lonQuantized, DimensionalLatLonField.TOLERANCE);
     }
-    return new BKDTreeDocValuesFormat(maxPointsInLeaf, maxPointsSortInHeap);
-  }
-
-  private Directory getDirectory() {     
-    return noVirusChecker(newDirectory());
   }
 }

Modified: lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/search/TestGeoPointQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/search/TestGeoPointQuery.java?rev=1713278&r1=1713277&r2=1713278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/search/TestGeoPointQuery.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/search/TestGeoPointQuery.java Sun Nov  8 18:18:19 2015
@@ -49,12 +49,18 @@ public class TestGeoPointQuery extends B
   private static final double DISTANCE_PCT_ERR = 0.005;
 
   @Override
+  protected boolean forceSmall() {
+    // TODO: GeoUtils are potentially slow if we use small=false with heavy testing
+    return true;
+  }
+
+  @Override
   protected void addPointToDoc(String field, Document doc, double lat, double lon) {
     doc.add(new GeoPointField(field, lon, lat, Field.Store.NO));
   }
 
   @Override
-  protected Query newBBoxQuery(String field, GeoRect rect) {
+  protected Query newRectQuery(String field, GeoRect rect) {
     return new GeoPointInBBoxQuery(field, rect.minLon, rect.minLat, rect.maxLon, rect.maxLat);
   }
 

Modified: lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/util/BaseGeoPointTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/util/BaseGeoPointTestCase.java?rev=1713278&r1=1713277&r2=1713278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/util/BaseGeoPointTestCase.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/util/BaseGeoPointTestCase.java Sun Nov  8 18:18:19 2015
@@ -72,11 +72,18 @@ public abstract class BaseGeoPointTestCa
     originLat = GeoUtils.normalizeLat(GeoUtils.MIN_LAT_INCL + latRange + (GeoUtils.MAX_LAT_INCL - GeoUtils.MIN_LAT_INCL - 2 * latRange) * random().nextDouble());
   }
 
+  /** Return true when testing on a non-small region may be too slow (GeoPoint*Query) */
+  protected boolean forceSmall() {
+    return false;
+  }
+
   // A particularly tricky adversary for BKD tree:
-  @Nightly
   public void testSamePointManyTimes() throws Exception {
+
+    // For GeoPointQuery, only run this test nightly:
+    assumeTrue("GeoPoint*Query is too slow otherwise", TEST_NIGHTLY || forceSmall() == false);
+
     int numPoints = atLeast(1000);
-    // TODO: GeoUtils are potentially slow if we use small=false with heavy testing
     boolean small = random().nextBoolean();
 
     // Every doc has 2 points:
@@ -92,12 +99,13 @@ public abstract class BaseGeoPointTestCa
     verify(small, lats, lons);
   }
 
-  @Nightly
   public void testAllLatEqual() throws Exception {
+
+    // For GeoPointQuery, only run this test nightly:
+    assumeTrue("GeoPoint*Query is too slow otherwise", TEST_NIGHTLY || forceSmall() == false);
+
     int numPoints = atLeast(10000);
-    // TODO: GeoUtils are potentially slow if we use small=false with heavy testing
-    // boolean small = random().nextBoolean();
-    boolean small = true;
+    boolean small = forceSmall() || random().nextBoolean();
     double lat = randomLat(small);
     double[] lats = new double[numPoints];
     double[] lons = new double[numPoints];
@@ -142,12 +150,13 @@ public abstract class BaseGeoPointTestCa
     verify(small, lats, lons);
   }
 
-  @Nightly
   public void testAllLonEqual() throws Exception {
+
+    // For GeoPointQuery, only run this test nightly:
+    assumeTrue("GeoPoint*Query is too slow otherwise", TEST_NIGHTLY || forceSmall() == false);
+
     int numPoints = atLeast(10000);
-    // TODO: GeoUtils are potentially slow if we use small=false with heavy testing
-    // boolean small = random().nextBoolean();
-    boolean small = true;
+    boolean small = forceSmall() || random().nextBoolean();
     double theLon = randomLon(small);
     double[] lats = new double[numPoints];
     double[] lons = new double[numPoints];
@@ -194,8 +203,11 @@ public abstract class BaseGeoPointTestCa
     verify(small, lats, lons);
   }
 
-  @Nightly
   public void testMultiValued() throws Exception {
+
+    // For GeoPointQuery, only run this test nightly:
+    assumeTrue("GeoPoint*Query is too slow otherwise", TEST_NIGHTLY || forceSmall() == false);
+
     int numPoints = atLeast(10000);
     // Every doc has 2 points:
     double[] lats = new double[2*numPoints];
@@ -209,9 +221,7 @@ public abstract class BaseGeoPointTestCa
     iwc.setMergePolicy(newLogMergePolicy());
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
 
-    // TODO: GeoUtils are potentially slow if we use small=false with heavy testing
     boolean small = random().nextBoolean();
-    //boolean small = true;
 
     for (int id=0;id<numPoints;id++) {
       Document doc = new Document();
@@ -231,6 +241,7 @@ public abstract class BaseGeoPointTestCa
       w.addDocument(doc);
     }
 
+    // TODO: share w/ verify; just need parallel array of the expected ids
     if (random().nextBoolean()) {
       w.forceMerge(1);
     }
@@ -245,10 +256,10 @@ public abstract class BaseGeoPointTestCa
       GeoRect rect = randomRect(small, small == false);
 
       if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter + " bbox=" + rect);
+        System.out.println("\nTEST: iter=" + iter + " rect=" + rect);
       }
 
-      Query query = newBBoxQuery(FIELD_NAME, rect);
+      Query query = newRectQuery(FIELD_NAME, rect);
 
       final FixedBitSet hits = new FixedBitSet(r.maxDoc());
       s.search(query, new SimpleCollector() {
@@ -340,7 +351,6 @@ public abstract class BaseGeoPointTestCa
     double[] lats = new double[numPoints];
     double[] lons = new double[numPoints];
 
-    // TODO: GeoUtils are potentially slow if we use small=false with heavy testing
     boolean small = random().nextBoolean();
 
     boolean haveRealDoc = false;
@@ -424,6 +434,7 @@ public abstract class BaseGeoPointTestCa
     } else {
       result = -90 + 180.0 * random().nextDouble();
     }
+    // TODO: we should not do this here!  it weakens the test, and users don't pre-quantize the lat/lons they send us:
     return unscaleLat(scaleLat(result));
   }
 
@@ -434,6 +445,7 @@ public abstract class BaseGeoPointTestCa
     } else {
       result = -180 + 360.0 * random().nextDouble();
     }
+    // TODO: we should not do this here!  it weakens the test, and users don't pre-quantize the lat/lons they send us:
     return unscaleLon(scaleLon(result));
   }
 
@@ -463,7 +475,7 @@ public abstract class BaseGeoPointTestCa
 
   protected abstract void addPointToDoc(String field, Document doc, double lat, double lon);
 
-  protected abstract Query newBBoxQuery(String field, GeoRect bbox);
+  protected abstract Query newRectQuery(String field, GeoRect bbox);
 
   protected abstract Query newDistanceQuery(String field, double centerLat, double centerLon, double radiusMeters);
 
@@ -557,7 +569,6 @@ public abstract class BaseGeoPointTestCa
     if (mbd != -1 && mbd < lats.length/100) {
       iwc.setMaxBufferedDocs(lats.length/100);
     }
-    initIndexWriterConfig(FIELD_NAME, iwc);
     Directory dir;
     if (lats.length > 100000) {
       dir = newFSDirectory(createTempDir(getClass().getSimpleName()));
@@ -631,15 +642,15 @@ public abstract class BaseGeoPointTestCa
               VerifyHits verifyHits;
 
               if (random().nextBoolean()) {
-                // BBox: don't allow dateline crossing when testing small:
-                final GeoRect bbox = randomRect(small, small == false);
+                // Rect: don't allow dateline crossing when testing small:
+                final GeoRect rect = randomRect(small, small == false);
 
-                query = newBBoxQuery(FIELD_NAME, bbox);
+                query = newRectQuery(FIELD_NAME, rect);
 
                 verifyHits = new VerifyHits() {
                     @Override
                     protected Boolean shouldMatch(double pointLat, double pointLon) {
-                      return rectContainsPoint(bbox, pointLat, pointLon);
+                      return rectContainsPoint(rect, pointLat, pointLon);
                     }
                     @Override
                     protected void describe(int docID, double lat, double lon) {

Copied: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Geo3DPointField.java (from r1711921, lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DPointField.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Geo3DPointField.java?p2=lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Geo3DPointField.java&p1=lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DPointField.java&r1=1711921&r2=1713278&rev=1713278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DPointField.java (original)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Geo3DPointField.java Sun Nov  8 18:18:19 2015
@@ -1,4 +1,4 @@
-package org.apache.lucene.bkdtree3d;
+package org.apache.lucene.geo3d;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,24 +17,23 @@ package org.apache.lucene.bkdtree3d;
  * limitations under the License.
  */
 
-import org.apache.lucene.geo3d.PlanetModel;
-import org.apache.lucene.geo3d.GeoPoint;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
-import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.bkd.BKDUtil;
 
-// TODO: allow multi-valued, packing all points into a single BytesRef
-
-/** Add this to a document to index lat/lon point, but be sure to use {@link Geo3DDocValuesFormat} for the field.
-
+/** Add this to a document to index lat/lon or x/y/z point, indexed as a dimensional value.
+ *  Multiple values are allowed: just add multiple Geo3DPointField to the document with the
+ *  same field name.
+ *
  *  @lucene.experimental */
 public final class Geo3DPointField extends Field {
 
   /** Indexing {@link FieldType}. */
   public static final FieldType TYPE = new FieldType();
   static {
-    TYPE.setDocValuesType(DocValuesType.BINARY);
+    TYPE.setDimensions(3, RamUsageEstimator.NUM_BYTES_INT);
     TYPE.freeze();
   }
 
@@ -62,9 +61,9 @@ public final class Geo3DPointField exten
 
   private void fillFieldsData(double planetMax, double x, double y, double z) {
     byte[] bytes = new byte[12];
-    Geo3DDocValuesFormat.writeInt(Geo3DDocValuesFormat.encodeValue(planetMax, x), bytes, 0);
-    Geo3DDocValuesFormat.writeInt(Geo3DDocValuesFormat.encodeValue(planetMax, y), bytes, 4);
-    Geo3DDocValuesFormat.writeInt(Geo3DDocValuesFormat.encodeValue(planetMax, z), bytes, 8);
+    BKDUtil.intToBytes(Geo3DUtil.encodeValue(planetMax, x), bytes, 0);
+    BKDUtil.intToBytes(Geo3DUtil.encodeValue(planetMax, y), bytes, 1);
+    BKDUtil.intToBytes(Geo3DUtil.encodeValue(planetMax, z), bytes, 2);
     fieldsData = new BytesRef(bytes);
   }
 }

Added: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Geo3DUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Geo3DUtil.java?rev=1713278&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Geo3DUtil.java (added)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Geo3DUtil.java Sun Nov  8 18:18:19 2015
@@ -0,0 +1,60 @@
+package org.apache.lucene.geo3d;
+
+/*
+ * 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.
+ */
+
+class Geo3DUtil {
+
+  /** Clips the incoming value to the allowed min/max range before encoding, instead of throwing an exception. */
+  public static int encodeValueLenient(double planetMax, double x) {
+    if (x > planetMax) {
+      x = planetMax;
+    } else if (x < -planetMax) {
+      x = -planetMax;
+    }
+    return encodeValue(planetMax, x);
+  }
+
+  public static int encodeValue(double planetMax, double x) {
+    if (x > planetMax) {
+      throw new IllegalArgumentException("value=" + x + " is out-of-bounds (greater than planetMax=" + planetMax + ")");
+    }
+    if (x < -planetMax) {
+      throw new IllegalArgumentException("value=" + x + " is out-of-bounds (less than than -planetMax=" + -planetMax + ")");
+    }
+    long y = Math.round (x * (Integer.MAX_VALUE / planetMax));
+    assert y >= Integer.MIN_VALUE;
+    assert y <= Integer.MAX_VALUE;
+
+    return (int) y;
+  }
+
+  /** Center decode */
+  public static double decodeValueCenter(double planetMax, int x) {
+    return x * (planetMax / Integer.MAX_VALUE);
+  }
+
+  /** More negative decode, at bottom of cell */
+  public static double decodeValueMin(double planetMax, int x) {
+    return (((double)x) - 0.5) * (planetMax / Integer.MAX_VALUE);
+  }
+  
+  /** More positive decode, at top of cell  */
+  public static double decodeValueMax(double planetMax, int x) {
+    return (((double)x) + 0.5) * (planetMax / Integer.MAX_VALUE);
+  }
+}

Copied: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/PointInGeo3DShapeQuery.java (from r1711041, lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/PointInGeo3DShapeQuery.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/PointInGeo3DShapeQuery.java?p2=lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/PointInGeo3DShapeQuery.java&p1=lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/PointInGeo3DShapeQuery.java&r1=1711041&r2=1713278&rev=1713278&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/PointInGeo3DShapeQuery.java (original)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/PointInGeo3DShapeQuery.java Sun Nov  8 18:18:19 2015
@@ -1,4 +1,4 @@
-package org.apache.lucene.bkdtree3d;
+package org.apache.lucene.geo3d;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,34 +17,25 @@ package org.apache.lucene.bkdtree3d;
  * limitations under the License.
  */
 
-import org.apache.lucene.geo3d.GeoArea;
-import org.apache.lucene.geo3d.GeoAreaFactory;
-import org.apache.lucene.geo3d.GeoShape;
-import org.apache.lucene.geo3d.PlanetModel;
-import org.apache.lucene.geo3d.XYZBounds;
-import org.apache.lucene.index.BinaryDocValues;
+import java.io.IOException;
+
+import org.apache.lucene.index.DimensionalValues;
+import org.apache.lucene.index.DimensionalValues.IntersectVisitor;
+import org.apache.lucene.index.DimensionalValues.Relation;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
-import org.apache.lucene.util.BytesRef;
-
-import java.io.IOException;
+import org.apache.lucene.util.DocIdSetBuilder;
+import org.apache.lucene.util.bkd.BKDUtil;
 
 /** Finds all previously indexed points that fall within the specified polygon.
  *
- *  <p>The field must be indexed with {@link Geo3DDocValuesFormat}, and {@link Geo3DPointField} added per document.
- *
- *  <p>Because this implementation cannot intersect each cell with the polygon, it will be costly especially for large polygons, as every
- *   possible point must be checked.
- *
- *  <p><b>NOTE</b>: for fastest performance, this allocates FixedBitSet(maxDoc) for each segment.  The score of each hit is the query boost.
+ * <p>The field must be indexed using {@link Geo3DPointField}.
  *
  * @lucene.experimental */
 
@@ -71,18 +62,12 @@ public class PointInGeo3DShapeQuery exte
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
         LeafReader reader = context.reader();
-        BinaryDocValues bdv = reader.getBinaryDocValues(field);
-        if (bdv == null) {
-          // No docs in this segment had this field
+        DimensionalValues values = reader.getDimensionalValues();
+        if (values == null) {
           return null;
         }
 
-        if (bdv instanceof Geo3DBinaryDocValues == false) {
-          throw new IllegalStateException("field \"" + field + "\" was not indexed with Geo3DBinaryDocValuesFormat: got: " + bdv);
-        }
-        final Geo3DBinaryDocValues treeDV = (Geo3DBinaryDocValues) bdv;
-        BKD3DTreeReader tree = treeDV.getBKD3DTreeReader();
-
+        /*
         XYZBounds bounds = new XYZBounds();
         shape.getBounds(bounds);
 
@@ -90,6 +75,7 @@ public class PointInGeo3DShapeQuery exte
         if (planetMax != treeDV.planetMax) {
           throw new IllegalStateException(planetModel + " is not the same one used during indexing: planetMax=" + planetMax + " vs indexing planetMax=" + treeDV.planetMax);
         }
+        */
 
         /*
         GeoArea xyzSolid = GeoAreaFactory.makeGeoArea(planetModel,
@@ -103,82 +89,79 @@ public class PointInGeo3DShapeQuery exte
         assert xyzSolid.getRelationship(shape) == GeoArea.WITHIN || xyzSolid.getRelationship(shape) == GeoArea.OVERLAPS: "expected WITHIN (1) or OVERLAPS (2) but got " + xyzSolid.getRelationship(shape) + "; shape="+shape+"; XYZSolid="+xyzSolid;
         */
 
-        DocIdSet result = tree.intersect(Geo3DDocValuesFormat.encodeValueLenient(planetMax, bounds.getMinimumX()),
-                                         Geo3DDocValuesFormat.encodeValueLenient(planetMax, bounds.getMaximumX()),
-                                         Geo3DDocValuesFormat.encodeValueLenient(planetMax, bounds.getMinimumY()),
-                                         Geo3DDocValuesFormat.encodeValueLenient(planetMax, bounds.getMaximumY()),
-                                         Geo3DDocValuesFormat.encodeValueLenient(planetMax, bounds.getMinimumZ()),
-                                         Geo3DDocValuesFormat.encodeValueLenient(planetMax, bounds.getMaximumZ()),
-                                         new BKD3DTreeReader.ValueFilter() {
-                                           @Override
-                                           public boolean accept(int docID) {
-                                             //System.out.println("  accept? docID=" + docID);
-                                             BytesRef bytes = treeDV.get(docID);
-                                             if (bytes == null) {
-                                               //System.out.println("    false (null)");
-                                               return false;
-                                             }
-
-                                             assert bytes.length == 12;
-                                             double x = Geo3DDocValuesFormat.decodeValueCenter(treeDV.planetMax, Geo3DDocValuesFormat.readInt(bytes.bytes, bytes.offset));
-                                             double y = Geo3DDocValuesFormat.decodeValueCenter(treeDV.planetMax, Geo3DDocValuesFormat.readInt(bytes.bytes, bytes.offset+4));
-                                             double z = Geo3DDocValuesFormat.decodeValueCenter(treeDV.planetMax, Geo3DDocValuesFormat.readInt(bytes.bytes, bytes.offset+8));
-                                             // System.out.println("  accept docID=" + docID + " point: x=" + x + " y=" + y + " z=" + z);
-
-                                             // True if x,y,z is within shape
-                                             //System.out.println("    x=" + x + " y=" + y + " z=" + z);
-                                             //System.out.println("    ret: " + shape.isWithin(x, y, z));
-
-                                             return shape.isWithin(x, y, z);
-                                           }
-
-                                           @Override
-                                           public BKD3DTreeReader.Relation compare(int cellXMinEnc, int cellXMaxEnc, int cellYMinEnc, int cellYMaxEnc, int cellZMinEnc, int cellZMaxEnc) {
-                                             assert cellXMinEnc <= cellXMaxEnc;
-                                             assert cellYMinEnc <= cellYMaxEnc;
-                                             assert cellZMinEnc <= cellZMaxEnc;
-
-                                             // Because the BKD tree operates in quantized (64 bit -> 32 bit) space, and the cell bounds
-                                             // here are inclusive, we need to extend the bounds to the largest un-quantized values that
-                                             // could quantize into these bounds.  The encoding (Geo3DDocValuesFormat.encodeValue) does
-                                             // a Math.round from double to long, so e.g. 1.4 -> 1, and -1.4 -> -1:
-                                             double cellXMin = Geo3DDocValuesFormat.decodeValueMin(treeDV.planetMax, cellXMinEnc);
-                                             double cellXMax = Geo3DDocValuesFormat.decodeValueMax(treeDV.planetMax, cellXMaxEnc);
-                                             double cellYMin = Geo3DDocValuesFormat.decodeValueMin(treeDV.planetMax, cellYMinEnc);
-                                             double cellYMax = Geo3DDocValuesFormat.decodeValueMax(treeDV.planetMax, cellYMaxEnc);
-                                             double cellZMin = Geo3DDocValuesFormat.decodeValueMin(treeDV.planetMax, cellZMinEnc);
-                                             double cellZMax = Geo3DDocValuesFormat.decodeValueMax(treeDV.planetMax, cellZMaxEnc);
-                                             //System.out.println("  compare: x=" + cellXMin + "-" + cellXMax + " y=" + cellYMin + "-" + cellYMax + " z=" + cellZMin + "-" + cellZMax);
-
-                                             GeoArea xyzSolid = GeoAreaFactory.makeGeoArea(planetModel, cellXMin, cellXMax, cellYMin, cellYMax, cellZMin, cellZMax);
-
-                                             switch(xyzSolid.getRelationship(shape)) {
-                                             case GeoArea.CONTAINS:
-                                               // Shape fully contains the cell
-                                               //System.out.println("    inside");
-                                               return BKD3DTreeReader.Relation.CELL_INSIDE_SHAPE;
-                                             case GeoArea.OVERLAPS:
-                                               // They do overlap but neither contains the other:
-                                               //System.out.println("    crosses1");
-                                               return BKD3DTreeReader.Relation.SHAPE_CROSSES_CELL;
-                                             case GeoArea.WITHIN:
-                                               // Cell fully contains the shape:
-                                               //System.out.println("    crosses2");
-                                               return BKD3DTreeReader.Relation.SHAPE_INSIDE_CELL;
-                                             case GeoArea.DISJOINT:
-                                               // They do not overlap at all
-                                               //System.out.println("    outside");
-                                               return BKD3DTreeReader.Relation.SHAPE_OUTSIDE_CELL;
-                                             default:
-                                               assert false;
-                                               return BKD3DTreeReader.Relation.SHAPE_CROSSES_CELL;
-                                             }
-                                           }
-                                         });
+        double planetMax = planetModel.getMaximumMagnitude();
+
+        DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc());
 
-        final DocIdSetIterator disi = result.iterator();
+        int[] hitCount = new int[1];
+        values.intersect(field,
+                         new IntersectVisitor() {
+
+                           @Override
+                           public void visit(int docID) {
+                             result.add(docID);
+                             hitCount[0]++;
+                           }
+
+                           @Override
+                           public void visit(int docID, byte[] packedValue) {
+                             assert packedValue.length == 12;
+                             double x = Geo3DUtil.decodeValueCenter(planetMax, BKDUtil.bytesToInt(packedValue, 0));
+                             double y = Geo3DUtil.decodeValueCenter(planetMax, BKDUtil.bytesToInt(packedValue, 1));
+                             double z = Geo3DUtil.decodeValueCenter(planetMax, BKDUtil.bytesToInt(packedValue, 2));
+                             if (shape.isWithin(x, y, z)) {
+                               result.add(docID);
+                               hitCount[0]++;
+                             }
+                           }
+
+                           @Override
+                           public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                             // Because the dimensional format operates in quantized (64 bit -> 32 bit) space, and the cell bounds
+                             // here are inclusive, we need to extend the bounds to the largest un-quantized values that
+                             // could quantize into these bounds.  The encoding (Geo3DUtil.encodeValue) does
+                             // a Math.round from double to long, so e.g. 1.4 -> 1, and -1.4 -> -1:
+                             double xMin = Geo3DUtil.decodeValueMin(planetMax, BKDUtil.bytesToInt(minPackedValue, 0));
+                             double xMax = Geo3DUtil.decodeValueMax(planetMax, BKDUtil.bytesToInt(maxPackedValue, 0));
+                             double yMin = Geo3DUtil.decodeValueMin(planetMax, BKDUtil.bytesToInt(minPackedValue, 1));
+                             double yMax = Geo3DUtil.decodeValueMax(planetMax, BKDUtil.bytesToInt(maxPackedValue, 1));
+                             double zMin = Geo3DUtil.decodeValueMin(planetMax, BKDUtil.bytesToInt(minPackedValue, 2));
+                             double zMax = Geo3DUtil.decodeValueMax(planetMax, BKDUtil.bytesToInt(maxPackedValue, 2));
+
+                             //System.out.println("  compare: x=" + cellXMin + "-" + cellXMax + " y=" + cellYMin + "-" + cellYMax + " z=" + cellZMin + "-" + cellZMax);
+                             assert xMin <= xMax;
+                             assert yMin <= yMax;
+                             assert zMin <= zMax;
+
+                             GeoArea xyzSolid = GeoAreaFactory.makeGeoArea(planetModel, xMin, xMax, yMin, yMax, zMin, zMax);
+
+                             switch(xyzSolid.getRelationship(shape)) {
+                             case GeoArea.CONTAINS:
+                               // Shape fully contains the cell
+                               //System.out.println("    inside");
+                               return Relation.CELL_INSIDE_QUERY;
+                             case GeoArea.OVERLAPS:
+                               // They do overlap but neither contains the other:
+                               //System.out.println("    crosses1");
+                               return Relation.CELL_CROSSES_QUERY;
+                             case GeoArea.WITHIN:
+                               // Cell fully contains the shape:
+                               //System.out.println("    crosses2");
+                               // return Relation.SHAPE_INSIDE_CELL;
+                               return Relation.CELL_CROSSES_QUERY;
+                             case GeoArea.DISJOINT:
+                               // They do not overlap at all
+                               //System.out.println("    outside");
+                               return Relation.CELL_OUTSIDE_QUERY;
+                             default:
+                               assert false;
+                               return Relation.CELL_CROSSES_QUERY;
+                             }
+                           }
+                         });
 
-        return new ConstantScoreScorer(this, score(), disi);
+        // NOTE: hitCount[0] will be over-estimate in multi-valued case
+        return new ConstantScoreScorer(this, score(), result.build(hitCount[0]).iterator());
       }
     };
   }