You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2016/03/03 16:23:13 UTC

[1/9] lucene-solr git commit: LUCENE-7059: always visit 1D points in sorted order; fix tie-break but in BKDWriter; fix BKDWriter to pass on maxMBSortInHeap to the OfflineSorter too (cherry picked from commit 3cbc48e)

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6_0 437dd7a18 -> 84d2bb604


LUCENE-7059: always visit 1D points in sorted order; fix tie-break but in BKDWriter; fix BKDWriter to pass on maxMBSortInHeap to the OfflineSorter too
(cherry picked from commit 3cbc48e)


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

Branch: refs/heads/branch_6_0
Commit: 3a96ed9e3b20ed449818a22c78d2a229afb9be40
Parents: 437dd7a
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Mar 2 21:55:27 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Mar 3 20:48:18 2016 +0530

----------------------------------------------------------------------
 .../org/apache/lucene/index/CheckIndex.java     |  23 +-
 .../org/apache/lucene/index/PointValues.java    |   4 +-
 .../apache/lucene/index/PointValuesWriter.java  |   1 +
 .../index/SlowCompositeReaderWrapper.java       |   2 +-
 .../apache/lucene/search/PointInSetQuery.java   |  13 -
 .../org/apache/lucene/util/OfflineSorter.java   |   2 +-
 .../org/apache/lucene/util/bkd/BKDWriter.java   |  41 ++-
 .../apache/lucene/index/TestDuelingCodecs.java  |   4 +-
 .../apache/lucene/index/TestPointValues.java    |  62 +++++
 .../apache/lucene/search/TestPointQueries.java  |  37 +--
 .../facet/range/TestRangeFacetCounts.java       |   8 +-
 .../org/apache/lucene/index/SorterTestBase.java |   6 +-
 .../lucene/document/TestBigIntegerPoint.java    |   4 +-
 .../lucene/document/TestInetAddressPoint.java   |   4 +-
 .../apache/lucene/document/TestLatLonPoint.java |   2 +-
 .../document/TestLatLonPointDistanceQuery.java  |   2 +-
 .../lucene/search/TestDocValuesRangeQuery.java  |   4 +-
 .../codecs/asserting/AssertingPointFormat.java  |  86 +++++-
 .../lucene/index/BasePointFormatTestCase.java   | 262 +++++++++++--------
 .../org/apache/lucene/util/LuceneTestCase.java  | 111 ++++----
 20 files changed, 440 insertions(+), 238 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index 0c24461..9f711df 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -1706,8 +1706,12 @@ public final class CheckIndex implements Closeable {
             lastMaxPacked.length = bytesPerDim;
             lastMinPacked.length = bytesPerDim;
             scratch.length = bytesPerDim;
+            byte[] lastPackedValue = new byte[dimCount*bytesPerDim];
             values.intersect(fieldInfo.name,
                              new PointValues.IntersectVisitor() {
+
+                               private int lastDocID = -1;
+
                                @Override
                                public void visit(int docID) {
                                  throw new RuntimeException("codec called IntersectVisitor.visit without a packed value for docID=" + docID);
@@ -1725,12 +1729,27 @@ public final class CheckIndex implements Closeable {
 
                                    if (scratch.compareTo(lastMinPacked) < 0) {
                                      // This doc's point, in this dimension, is lower than the minimum value of the last cell checked:
-                                     throw new RuntimeException("packed value " + Arrays.toString(packedValue) + " for docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
+                                     throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
                                    }
                                    if (scratch.compareTo(lastMaxPacked) > 0) {
                                      // This doc's point, in this dimension, is greater than the maximum value of the last cell checked:
-                                     throw new RuntimeException("packed value " + Arrays.toString(packedValue) + " for docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
+                                     throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for docID=" + docID + " is out-of-bounds of the last cell min=" + Arrays.toString(lastMinPackedValue) + " max=" + Arrays.toString(lastMaxPackedValue) + " dim=" + dim);
+                                   }
+                                 }
+
+                                 // In the 1D case, PointValues must make a single in-order sweep through all values, and tie-break by
+                                 // increasing docID:
+                                 if (dimCount == 1) {
+                                   int cmp = StringHelper.compare(bytesPerDim, lastPackedValue, 0, packedValue, 0);
+                                   if (cmp > 0) {
+                                     throw new RuntimeException("packed points value " + Arrays.toString(packedValue) + " for docID=" + docID + " is out-of-order vs the previous document's value " + Arrays.toString(lastPackedValue));
+                                   } else if (cmp == 0) {
+                                     if (docID < lastDocID) {
+                                       throw new RuntimeException("packed points value is the same, but docID=" + docID + " is out of order vs previous docID=" + lastDocID);
+                                     }
                                    }
+                                   System.arraycopy(packedValue, 0, lastPackedValue, 0, bytesPerDim);
+                                   lastDocID = docID;
                                  }
 
                                  status.totalValuePoints++;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/core/src/java/org/apache/lucene/index/PointValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValues.java b/lucene/core/src/java/org/apache/lucene/index/PointValues.java
index 108dde3..34d315f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValues.java
@@ -61,7 +61,9 @@ public abstract class PointValues {
     void visit(int docID) throws IOException;
 
     /** Called for all documents in a leaf cell that crosses the query.  The consumer
-     *  should scrutinize the packedValue to decide whether to accept it. */
+     *  should scrutinize the packedValue to decide whether to accept it.  In the 1D case,
+     *  values are visited in increasing order, and in the case of ties, in increasing
+     *  docID order. */
     void visit(int docID, byte[] packedValue) throws IOException;
 
     /** Called for non-leaf cells to test how the cell relates to the query, to

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
index 283f7bd..2fa8b4f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
@@ -43,6 +43,7 @@ class PointValuesWriter {
     packedValue = new byte[fieldInfo.getPointDimensionCount() * fieldInfo.getPointNumBytes()];
   }
 
+  // TODO: if exactly the same value is added to exactly the same doc, should we dedup?
   public void addPackedValue(int docID, BytesRef value) {
     if (value == null) {
       throw new IllegalArgumentException("field=" + fieldInfo.name + ": point value cannot be null");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java b/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
index db9579c..e44c53c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
@@ -236,7 +236,7 @@ public final class SlowCompositeReaderWrapper extends LeafReader {
   @Override
   public PointValues getPointValues() {
     ensureOpen();
-    return MultiPointValues.get(in);
+    return null;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index 42f5bb1..3d6086c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -172,10 +172,6 @@ public abstract class PointInSetQuery extends Query {
       this.sortedPackedPoints = sortedPackedPoints;
       lastMaxPackedValue = new byte[bytesPerDim];
       scratch.length = bytesPerDim;
-      resetIterator();
-    }
-
-    private void resetIterator() {
       this.iterator = sortedPackedPoints.iterator();
       nextQueryPoint = iterator.next();
     }
@@ -211,15 +207,6 @@ public abstract class PointInSetQuery extends Query {
 
     @Override
     public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-      
-      // NOTE: this is messy ... we need it in cases where a single vistor (us) is shared across multiple leaf readers
-      // (e.g. SlowCompositeReaderWrapper), in which case we need to reset our iterator to re-start the merge sort.  Maybe we should instead
-      // add an explicit .start() to IntersectVisitor, and clarify the semantics that in the 1D case all cells will be visited in order?
-      if (StringHelper.compare(bytesPerDim, lastMaxPackedValue, 0, minPackedValue, 0) > 0) {    
-        resetIterator();
-      }
-      System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, bytesPerDim);
-
       while (nextQueryPoint != null) {
         scratch.bytes = minPackedValue;
         int cmpMin = nextQueryPoint.compareTo(scratch);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
index b146102..283dc1f 100644
--- a/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
@@ -120,7 +120,7 @@ public class OfflineSorter {
       final long minBufferSizeBytes = MIN_BUFFER_SIZE_MB*MB;
       if (sortBufferByteSize <  minBufferSizeBytes
           || totalAvailableBytes > 10 * minBufferSizeBytes) { // lets see if we need/should to grow the heap 
-        if (totalAvailableBytes/2 > minBufferSizeBytes){ // there is enough mem for a reasonable buffer
+        if (totalAvailableBytes/2 > minBufferSizeBytes) { // there is enough mem for a reasonable buffer
           sortBufferByteSize = totalAvailableBytes/2; // grow the heap
         } else {
           //heap seems smallish lets be conservative fall back to the free/2 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
index 4397f2e..9d6ad97 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
@@ -102,6 +102,7 @@ public class BKDWriter implements Closeable {
 
   final TrackingDirectoryWrapper tempDir;
   final String tempFileNamePrefix;
+  final double maxMBSortInHeap;
 
   final byte[] scratchDiff;
   final byte[] scratchPackedValue;
@@ -169,6 +170,8 @@ public class BKDWriter implements Closeable {
 
     // We write first maxPointsSortInHeap in heap, then cutover to offline for additional points:
     heapPointWriter = new HeapPointWriter(16, maxPointsSortInHeap, packedBytesLength);
+
+    this.maxMBSortInHeap = maxMBSortInHeap;
   }
 
   public static void verifyParams(int numDims, int maxPointsInLeafNode, double maxMBSortInHeap) {
@@ -550,7 +553,7 @@ public class BKDWriter implements Closeable {
     //int[] swapCount = new int[1];
     //int[] cmpCount = new int[1];
 
-    //System.out.println("SORT length=" + length);
+    // System.out.println("SORT length=" + length);
 
     // All buffered points are still in heap; just do in-place sort:
     new IntroSorter() {
@@ -623,13 +626,11 @@ public class BKDWriter implements Closeable {
           return cmp;
         }
 
-        // Tie-break
-        cmp = Integer.compare(writer.docIDs[i], writer.docIDs[j]);
-        if (cmp != 0) {
-          return cmp;
-        }
+        // Tie-break by docID:
 
-        return Long.compare(writer.ords[i], writer.ords[j]);
+        // No need to tie break on ord, for the case where the same doc has the same value in a given dimension indexed more than once: it
+        // can't matter at search time since we don't write ords into the index:
+        return Integer.compare(writer.docIDs[i], writer.docIDs[j]);
       }
     }.sort(start, start+length);
     //System.out.println("LEN=" + length + " SWAP=" + swapCount[0] + " CMP=" + cmpCount[0]);
@@ -679,29 +680,23 @@ public class BKDWriter implements Closeable {
             return cmp;
           }
 
-          // Tie-break by docID and then ord:
-          reader.reset(a.bytes, a.offset + packedBytesLength, a.length);
-          final int docIDA = reader.readVInt();
-          final long ordA = reader.readVLong();
+          // Tie-break by docID:
+          reader.reset(a.bytes, a.offset + packedBytesLength + Long.BYTES, a.length);
+          final int docIDA = reader.readInt();
 
-          reader.reset(b.bytes, b.offset + packedBytesLength, b.length);
-          final int docIDB = reader.readVInt();
-          final long ordB = reader.readVLong();
-
-          cmp = Integer.compare(docIDA, docIDB);
-          if (cmp != 0) {
-            return cmp;
-          }
+          reader.reset(b.bytes, b.offset + packedBytesLength + Long.BYTES, b.length);
+          final int docIDB = reader.readInt();
 
-          // TODO: is this really necessary?  If OfflineSorter is stable, we can safely return 0 here, and avoid writing ords?
-          return Long.compare(ordA, ordB);
+          // No need to tie break on ord, for the case where the same doc has the same value in a given dimension indexed more than once: it
+          // can't matter at search time since we don't write ords into the index:
+          return Integer.compare(docIDA, docIDB);
         }
       };
 
       // TODO: this is sort of sneaky way to get the final OfflinePointWriter from OfflineSorter:
       IndexOutput[] lastWriter = new IndexOutput[1];
 
-      OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix, cmp) {
+      OfflineSorter sorter = new OfflineSorter(tempDir, tempFileNamePrefix, cmp, OfflineSorter.BufferSize.megabytes(Math.max(1, (long) maxMBSortInHeap)), OfflineSorter.MAX_TEMPFILES) {
 
           /** We write/read fixed-byte-width file that {@link OfflinePointReader} can read. */
           @Override
@@ -753,7 +748,7 @@ public class BKDWriter implements Closeable {
 
   /** Writes the BKD tree to the provided {@link IndexOutput} and returns the file offset where index was written. */
   public long finish(IndexOutput out) throws IOException {
-    //System.out.println("\nBKDTreeWriter.finish pointCount=" + pointCount + " out=" + out + " heapWriter=" + heapWriter);
+    // System.out.println("\nBKDTreeWriter.finish pointCount=" + pointCount + " out=" + out + " heapWriter=" + heapPointWriter);
 
     // TODO: specialize the 1D case?  it's much faster at indexing time (no partitioning on recruse...)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java b/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
index 62fe28a..b79e638 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
@@ -162,8 +162,8 @@ public class TestDuelingCodecs extends LuceneTestCase {
     createRandomIndex(numdocs, leftWriter, seed);
     createRandomIndex(numdocs, rightWriter, seed);
 
-    leftReader = wrapReader(leftWriter.getReader());
-    rightReader = wrapReader(rightWriter.getReader());
+    leftReader = wrapReader(leftWriter.getReader(), false);
+    rightReader = wrapReader(rightWriter.getReader(), false);
     
     // check that our readers are valid
     TestUtil.checkReader(leftReader);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
index 3e1fe2d..7551d3c 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPointValues.java
@@ -17,8 +17,16 @@
 package org.apache.lucene.index;
 
 
+import java.io.IOException;
+
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.PointFormat;
+import org.apache.lucene.codecs.PointReader;
+import org.apache.lucene.codecs.PointWriter;
+import org.apache.lucene.codecs.lucene60.Lucene60PointReader;
+import org.apache.lucene.codecs.lucene60.Lucene60PointWriter;
 import org.apache.lucene.document.BinaryPoint;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoublePoint;
@@ -26,6 +34,9 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FloatPoint;
 import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.index.PointValues;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
@@ -478,4 +489,55 @@ public class TestPointValues extends LuceneTestCase {
       field.numericValue();
     });
   }
+
+  public void testTieBreakByDocID() throws Exception {
+    Directory dir = newFSDirectory(createTempDir());
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    IndexWriter w = new IndexWriter(dir, iwc);
+    Document doc = new Document();
+    doc.add(new IntPoint("int", 17));
+    for(int i=0;i<300000;i++) {
+      w.addDocument(doc);
+      if (false && random().nextInt(1000) == 17) {
+        w.commit();
+      }
+    }
+
+    IndexReader r = DirectoryReader.open(w);
+
+    for(LeafReaderContext ctx : r.leaves()) {
+      PointValues points = ctx.reader().getPointValues();
+      points.intersect("int",
+                       new IntersectVisitor() {
+
+                         int lastDocID = -1;
+
+                         @Override
+                         public void visit(int docID) {
+                           if (docID < lastDocID) {
+                             fail("docs out of order: docID=" + docID + " but lastDocID=" + lastDocID);
+                           }
+                           lastDocID = docID;
+                         }
+
+                         @Override
+                         public void visit(int docID, byte[] packedValue) {
+                           visit(docID);
+                         }
+
+                         @Override
+                         public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                           if (random().nextBoolean()) {
+                             return Relation.CELL_CROSSES_QUERY;
+                           } else {
+                             return Relation.CELL_INSIDE_QUERY;
+                           }
+                         }
+                       });
+    }
+    
+    r.close();
+    w.close();
+    dir.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
index 8a578fa..fb31792 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -53,11 +53,13 @@ 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.MultiReader;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.PointValues;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.SlowCompositeReaderWrapper;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
@@ -360,7 +362,7 @@ public class TestPointQueries extends LuceneTestCase {
     final IndexReader r = w.getReader();
     w.close();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     int numThreads = TestUtil.nextInt(random(), 2, 5);
 
@@ -627,7 +629,7 @@ public class TestPointQueries extends LuceneTestCase {
     final IndexReader r = w.getReader();
     w.close();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     int numThreads = TestUtil.nextInt(random(), 2, 5);
 
@@ -847,7 +849,7 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     assertEquals(1, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, true, 0L, true)));
     assertEquals(1, s.count(LongPoint.newRangeQuery("value", 0L, true, Long.MAX_VALUE, true)));
@@ -885,7 +887,7 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     assertEquals(1, s.count(BinaryPoint.newRangeQuery("value",
         toUTF8("aaa"),
@@ -950,7 +952,7 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     assertEquals(2, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
     assertEquals(1, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, true, Long.MAX_VALUE, false)));
@@ -976,7 +978,6 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = w.getReader();
 
-    // We can't wrap with "exotic" readers because the query must see the RangeTreeDVFormat:
     IndexSearcher s = newSearcher(r, false);
 
     assertEquals(2, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
@@ -1003,7 +1004,7 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(0, s.count(BinaryPoint.newRangeQuery("value", toUTF8("m"), true, toUTF8("n"), false)));
 
     assertEquals(2, s.count(BinaryPoint.newRangeQuery("value", (byte[]) null, true, null, true)));
@@ -1040,7 +1041,7 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(0, s.count(LongPoint.newRangeQuery("value", 17L, true, 13L, false)));
 
     IOUtils.close(r, w, dir);
@@ -1207,7 +1208,7 @@ public class TestPointQueries extends LuceneTestCase {
     w.addDocument(doc);
 
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(1, s.count(IntPoint.newExactQuery("int", 42)));
     assertEquals(0, s.count(IntPoint.newExactQuery("int", 41)));
 
@@ -1328,7 +1329,7 @@ public class TestPointQueries extends LuceneTestCase {
     final IndexReader r = w.getReader();
     w.close();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     int numThreads = TestUtil.nextInt(random(), 2, 5);
 
@@ -1467,7 +1468,7 @@ public class TestPointQueries extends LuceneTestCase {
     doc.add(new IntPoint("int", 17, 42));
     w.addDocument(doc);
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     assertEquals(0, s.count(newMultiDimIntSetQuery("int", 2, 17, 41)));
     assertEquals(1, s.count(newMultiDimIntSetQuery("int", 2, 17, 42)));
@@ -1490,7 +1491,7 @@ public class TestPointQueries extends LuceneTestCase {
     doc.add(new IntPoint("int", 34, 79));
     w.addDocument(doc);
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     assertEquals(0, s.count(newMultiDimIntSetQuery("int", 2, 17, 41)));
     assertEquals(1, s.count(newMultiDimIntSetQuery("int", 2, 17, 42)));
@@ -1523,7 +1524,7 @@ public class TestPointQueries extends LuceneTestCase {
       w.addDocument(doc);
     }
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     assertEquals(zeroCount, s.count(newMultiDimIntSetQuery("int", 2, 0, 0)));
     assertEquals(10000-zeroCount, s.count(newMultiDimIntSetQuery("int", 2, 1, 1)));
@@ -1573,7 +1574,7 @@ public class TestPointQueries extends LuceneTestCase {
     w.addDocument(doc);
 
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(0, s.count(IntPoint.newSetQuery("int", 16)));
     assertEquals(1, s.count(IntPoint.newSetQuery("int", 17)));
     assertEquals(3, s.count(IntPoint.newSetQuery("int", 17, 97, 42)));
@@ -1634,7 +1635,7 @@ public class TestPointQueries extends LuceneTestCase {
     w.addDocument(doc);
 
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(0, s.count(IntPoint.newSetQuery("int", 16)));
     assertEquals(1, s.count(IntPoint.newSetQuery("int", 17)));
     assertEquals(1, s.count(IntPoint.newSetQuery("int", 17, 97, 42)));
@@ -1685,7 +1686,7 @@ public class TestPointQueries extends LuceneTestCase {
     w.addDocument(doc);
 
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(0, s.count(IntPoint.newSetQuery("int")));
     assertEquals(0, s.count(LongPoint.newSetQuery("long")));
     assertEquals(0, s.count(FloatPoint.newSetQuery("float")));
@@ -1719,7 +1720,7 @@ public class TestPointQueries extends LuceneTestCase {
     }
 
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 0)));
     assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 0, -7)));
     assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 7, 0)));
@@ -1777,7 +1778,7 @@ public class TestPointQueries extends LuceneTestCase {
     }
 
     IndexReader r = DirectoryReader.open(w);
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 0)));
     assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 0, -7)));
     assertEquals(zeroCount, s.count(IntPoint.newSetQuery("int", 7, 0)));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
index f7a1970..c4233c0 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
@@ -218,7 +218,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
 
     final TaxonomyReader tr = new DirectoryTaxonomyReader(tw);
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
 
     if (VERBOSE) {
       System.out.println("TEST: searcher=" + s);
@@ -375,7 +375,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
     }
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     FacetsConfig config = new FacetsConfig();
     
     int numIters = atLeast(10);
@@ -516,7 +516,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
     }
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     FacetsConfig config = new FacetsConfig();
     
     int numIters = atLeast(10);
@@ -671,7 +671,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
     }
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     FacetsConfig config = new FacetsConfig();
     
     int numIters = atLeast(10);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java b/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
index 93caa701..0015c6c 100644
--- a/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
+++ b/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
@@ -174,7 +174,8 @@ public abstract class SorterTestBase extends LuceneTestCase {
     doc.add(new Field(TERM_VECTORS_FIELD, Integer.toString(id), TERM_VECTORS_TYPE));
     byte[] bytes = new byte[4];
     NumericUtils.intToBytes(id, bytes, 0);
-    doc.add(new BinaryPoint(DIMENSIONAL_FIELD, bytes));
+    // TODO: index time sorting doesn't yet support points
+    //doc.add(new BinaryPoint(DIMENSIONAL_FIELD, bytes));
     return doc;
   }
 
@@ -378,6 +379,8 @@ public abstract class SorterTestBase extends LuceneTestCase {
     }
   }
 
+  // TODO: index sorting doesn't yet support points
+  /*
   public void testPoints() throws Exception {
     PointValues values = sortedReader.getPointValues();
     values.intersect(DIMENSIONAL_FIELD,
@@ -398,4 +401,5 @@ public abstract class SorterTestBase extends LuceneTestCase {
                        }
                      });
   }
+  */
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
index 3e8cf3d..f6d407d 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
@@ -40,7 +40,7 @@ public class TestBigIntegerPoint extends LuceneTestCase {
     
     // search and verify we found our doc
     IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(BigIntegerPoint.newExactQuery("field", large)));
     assertEquals(1, searcher.count(BigIntegerPoint.newRangeQuery("field", large.subtract(BigInteger.ONE), false, large.add(BigInteger.ONE), false)));
     assertEquals(1, searcher.count(BigIntegerPoint.newSetQuery("field", large)));
@@ -65,7 +65,7 @@ public class TestBigIntegerPoint extends LuceneTestCase {
     
     // search and verify we found our doc
     IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(BigIntegerPoint.newExactQuery("field", negative)));
     assertEquals(1, searcher.count(BigIntegerPoint.newRangeQuery("field", negative.subtract(BigInteger.ONE), false, negative.add(BigInteger.ONE), false)));
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
index c9be31f..9854001 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
@@ -40,7 +40,7 @@ public class TestInetAddressPoint extends LuceneTestCase {
     
     // search and verify we found our doc
     IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(InetAddressPoint.newExactQuery("field", address)));
     assertEquals(1, searcher.count(InetAddressPoint.newPrefixQuery("field", address, 24)));
     assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("1.2.3.3"), false, InetAddress.getByName("1.2.3.5"), false)));
@@ -66,7 +66,7 @@ public class TestInetAddressPoint extends LuceneTestCase {
     
     // search and verify we found our doc
     IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(InetAddressPoint.newExactQuery("field", address)));
     assertEquals(1, searcher.count(InetAddressPoint.newPrefixQuery("field", address, 64)));
     assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("fec0::f66c"), false, InetAddress.getByName("fec0::f66e"), false)));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
index b67dec8..1d3bfac 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
@@ -38,7 +38,7 @@ public class TestLatLonPoint extends LuceneTestCase {
     
     // search and verify we found our doc
     IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(LatLonPoint.newBoxQuery("field", 18, 19, -66, -65)));
 
     reader.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceQuery.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceQuery.java
index 3d47b44..e37d75e 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceQuery.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPointDistanceQuery.java
@@ -55,7 +55,7 @@ public class TestLatLonPointDistanceQuery extends LuceneTestCase {
     
     // search within 50km and verify we found our doc
     IndexReader reader = writer.getReader();
-    IndexSearcher searcher = newSearcher(reader);
+    IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(LatLonPoint.newDistanceQuery("field", 18, -65, 50_000)));
 
     reader.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
index 1bcadd6..47cd740 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
@@ -60,7 +60,7 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
       }
       iw.commit();
       final IndexReader reader = iw.getReader();
-      final IndexSearcher searcher = newSearcher(reader);
+      final IndexSearcher searcher = newSearcher(reader, false);
       iw.close();
 
       for (int i = 0; i < 100; ++i) {
@@ -188,7 +188,7 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
     }
     iw.commit();
     final IndexReader reader = iw.getReader();
-    final IndexSearcher searcher = newSearcher(reader);
+    final IndexSearcher searcher = newSearcher(reader, false);
     iw.close();
 
     for (int i = 0; i < 100; ++i) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointFormat.java
index 3411c7b..15836de 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPointFormat.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.codecs.asserting;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collection;
 
 import org.apache.lucene.codecs.PointFormat;
@@ -24,9 +25,13 @@ import org.apache.lucene.codecs.PointReader;
 import org.apache.lucene.codecs.PointWriter;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.index.PointValues;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.TestUtil;
 
 /**
@@ -60,6 +65,83 @@ public final class AssertingPointFormat extends PointFormat {
   public PointReader fieldsReader(SegmentReadState state) throws IOException {
     return new AssertingPointReader(in.fieldsReader(state));
   }
+
+  /** Validates in the 1D case that all points are visited in order, and point values are in bounds of the last cell checked */
+  static class AssertingIntersectVisitor implements IntersectVisitor {
+    final IntersectVisitor in;
+    final int numDims;
+    final int bytesPerDim;
+    final byte[] lastDocValue;
+    final byte[] lastMinPackedValue;
+    final byte[] lastMaxPackedValue;
+    private Relation lastCompareResult;
+    private int lastDocID = -1;
+
+    public AssertingIntersectVisitor(int numDims, int bytesPerDim, IntersectVisitor in) {
+      this.in = in;
+      this.numDims = numDims;
+      this.bytesPerDim = bytesPerDim;
+      lastMaxPackedValue = new byte[numDims*bytesPerDim];
+      lastMinPackedValue = new byte[numDims*bytesPerDim];
+      if (numDims == 1) {
+        lastDocValue = new byte[bytesPerDim];
+      } else {
+        lastDocValue = null;
+      }
+    }
+
+    @Override
+    public void visit(int docID) throws IOException {
+      // This method, not filtering each hit, should only be invoked when the cell is inside the query shape:
+      assert lastCompareResult == Relation.CELL_INSIDE_QUERY;
+      in.visit(docID);
+    }
+
+    @Override
+    public void visit(int docID, byte[] packedValue) throws IOException {
+
+      // This method, to filter each doc's value, should only be invoked when the cell crosses the query shape:
+      assert lastCompareResult == PointValues.Relation.CELL_CROSSES_QUERY;
+
+      // This doc's packed value should be contained in the last cell passed to compare:
+      for(int dim=0;dim<numDims;dim++) {
+        assert StringHelper.compare(bytesPerDim, lastMinPackedValue, dim*bytesPerDim, packedValue, dim*bytesPerDim) <= 0: "dim=" + dim + " of " +  numDims;
+        assert StringHelper.compare(bytesPerDim, lastMaxPackedValue, dim*bytesPerDim, packedValue, dim*bytesPerDim) >= 0: "dim=" + dim + " of " +  numDims;
+      }
+
+      // TODO: we should assert that this "matches" whatever relation the last call to compare had returned
+      assert packedValue.length == numDims * bytesPerDim;
+      if (numDims == 1) {
+        int cmp = StringHelper.compare(bytesPerDim, lastDocValue, 0, packedValue, 0);
+        if (cmp < 0) {
+          // ok
+        } else if (cmp == 0) {
+          assert lastDocID <= docID: "doc ids are out of order when point values are the same!";
+        } else {
+          // out of order!
+          assert false: "point values are out of order";
+        }
+        System.arraycopy(packedValue, 0, lastDocValue, 0, bytesPerDim);
+      }
+      in.visit(docID, packedValue);
+    }
+
+    @Override
+    public void grow(int count) {
+      in.grow(count);
+    }
+
+    @Override
+    public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+      for(int dim=0;dim<numDims;dim++) {
+        assert StringHelper.compare(bytesPerDim, minPackedValue, dim*bytesPerDim, maxPackedValue, dim*bytesPerDim) <= 0;
+      }
+      System.arraycopy(maxPackedValue, 0, lastMaxPackedValue, 0, numDims*bytesPerDim);
+      System.arraycopy(minPackedValue, 0, lastMinPackedValue, 0, numDims*bytesPerDim);
+      lastCompareResult = in.compare(minPackedValue, maxPackedValue);
+      return lastCompareResult;
+    }
+  }
   
   static class AssertingPointReader extends PointReader {
     private final PointReader in;
@@ -80,8 +162,8 @@ public final class AssertingPointFormat extends PointFormat {
 
     @Override
     public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
-      // TODO: wrap the visitor and make sure things are being reasonable
-      in.intersect(fieldName, visitor);
+      in.intersect(fieldName,
+                   new AssertingIntersectVisitor(in.getNumDimensions(fieldName), in.getBytesPerDimension(fieldName), visitor));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java
index 1a81138..5113bcd 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BasePointFormatTestCase.java
@@ -163,29 +163,31 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
     w.close();
     DirectoryReader r = DirectoryReader.open(dir);
     assertEquals(1, r.numDocs());
-    PointValues values = MultiPointValues.get(r);
     Bits liveDocs = MultiFields.getLiveDocs(r);
-    NumericDocValues idValues = MultiDocValues.getNumericValues(r, "id");
-
-    if (values != null) {
-      BitSet seen = new BitSet();
-      values.intersect("dim",
-                       new IntersectVisitor() {
-                         @Override
-                         public Relation compare(byte[] minPacked, byte[] maxPacked) {
-                           return Relation.CELL_CROSSES_QUERY;
-                         }
-                         public void visit(int docID) {
-                           throw new IllegalStateException();
-                         }
-                         public void visit(int docID, byte[] packedValue) {
-                           if (liveDocs.get(docID)) {
-                             seen.set(docID);
+
+    for(LeafReaderContext ctx : r.leaves()) {
+      PointValues values = ctx.reader().getPointValues();
+      NumericDocValues idValues = ctx.reader().getNumericDocValues("id");
+      if (values != null) {
+        BitSet seen = new BitSet();
+        values.intersect("dim",
+                         new IntersectVisitor() {
+                           @Override
+                           public Relation compare(byte[] minPacked, byte[] maxPacked) {
+                             return Relation.CELL_CROSSES_QUERY;
+                           }
+                           public void visit(int docID) {
+                             throw new IllegalStateException();
+                           }
+                           public void visit(int docID, byte[] packedValue) {
+                             if (liveDocs.get(docID)) {
+                               seen.set(docID);
+                             }
+                             assertEquals(idValues.get(docID), NumericUtils.bytesToInt(packedValue, 0));
                            }
-                           assertEquals(idValues.get(docID), NumericUtils.bytesToInt(packedValue, 0));
-                         }
-                       });
-      assertEquals(0, seen.cardinality());
+                         });
+        assertEquals(0, seen.cardinality());
+      }
     }
     IOUtils.close(r, dir);
   }
@@ -361,8 +363,6 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
       DirectoryReader r = w.getReader();
       w.close();
 
-      PointValues dimValues = MultiPointValues.get(r);
-
       int iters = atLeast(100);
       for(int iter=0;iter<iters;iter++) {
         if (VERBOSE) {
@@ -386,50 +386,59 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
         }
 
         final BitSet hits = new BitSet();
-        dimValues.intersect("field", new IntersectVisitor() {
-            @Override
-            public void visit(int docID) {
-              hits.set(docID);
-              //System.out.println("visit docID=" + docID);
-            }
+        for(LeafReaderContext ctx : r.leaves()) {
+          PointValues dimValues = ctx.reader().getPointValues();
+          if (dimValues == null) {
+            continue;
+          }
 
-            @Override
-            public void visit(int docID, byte[] packedValue) {
-              //System.out.println("visit check docID=" + docID);
-              for(int dim=0;dim<numDims;dim++) {
-                BigInteger x = NumericUtils.bytesToBigInt(packedValue, dim * numBytesPerDim, numBytesPerDim);
-                if (x.compareTo(queryMin[dim]) < 0 || x.compareTo(queryMax[dim]) > 0) {
-                  //System.out.println("  no");
-                  return;
-                }
+          final int docBase = ctx.docBase;
+          
+          dimValues.intersect("field", new IntersectVisitor() {
+              @Override
+              public void visit(int docID) {
+                hits.set(docBase+docID);
+                //System.out.println("visit docID=" + docID);
               }
 
-              //System.out.println("  yes");
-              hits.set(docID);
-            }
-
-            @Override
-            public Relation compare(byte[] minPacked, byte[] maxPacked) {
-              boolean crosses = false;
-              for(int dim=0;dim<numDims;dim++) {
-                BigInteger min = NumericUtils.bytesToBigInt(minPacked, dim * numBytesPerDim, numBytesPerDim);
-                BigInteger max = NumericUtils.bytesToBigInt(maxPacked, dim * numBytesPerDim, numBytesPerDim);
-                assert max.compareTo(min) >= 0;
-
-                if (max.compareTo(queryMin[dim]) < 0 || min.compareTo(queryMax[dim]) > 0) {
-                  return Relation.CELL_OUTSIDE_QUERY;
-                } else if (min.compareTo(queryMin[dim]) < 0 || max.compareTo(queryMax[dim]) > 0) {
-                  crosses = true;
+              @Override
+              public void visit(int docID, byte[] packedValue) {
+                //System.out.println("visit check docID=" + docID);
+                for(int dim=0;dim<numDims;dim++) {
+                  BigInteger x = NumericUtils.bytesToBigInt(packedValue, dim * numBytesPerDim, numBytesPerDim);
+                  if (x.compareTo(queryMin[dim]) < 0 || x.compareTo(queryMax[dim]) > 0) {
+                    //System.out.println("  no");
+                    return;
+                  }
                 }
+
+                //System.out.println("  yes");
+                hits.set(docBase+docID);
               }
 
-              if (crosses) {
-                return Relation.CELL_CROSSES_QUERY;
-              } else {
-                return Relation.CELL_INSIDE_QUERY;
+              @Override
+              public Relation compare(byte[] minPacked, byte[] maxPacked) {
+                boolean crosses = false;
+                for(int dim=0;dim<numDims;dim++) {
+                  BigInteger min = NumericUtils.bytesToBigInt(minPacked, dim * numBytesPerDim, numBytesPerDim);
+                  BigInteger max = NumericUtils.bytesToBigInt(maxPacked, dim * numBytesPerDim, numBytesPerDim);
+                  assert max.compareTo(min) >= 0;
+
+                  if (max.compareTo(queryMin[dim]) < 0 || min.compareTo(queryMax[dim]) > 0) {
+                    return Relation.CELL_OUTSIDE_QUERY;
+                  } else if (min.compareTo(queryMin[dim]) < 0 || max.compareTo(queryMax[dim]) > 0) {
+                    crosses = true;
+                  }
+                }
+
+                if (crosses) {
+                  return Relation.CELL_CROSSES_QUERY;
+                } else {
+                  return Relation.CELL_INSIDE_QUERY;
+                }
               }
-            }
-          });
+            });
+        }
 
         for(int docID=0;docID<numDocs;docID++) {
           BigInteger[] docValues = docs[docID];
@@ -665,24 +674,39 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
         System.out.println("TEST: reader=" + r);
       }
 
-      PointValues dimValues = MultiPointValues.get(r);
-      if (VERBOSE) {
-        System.out.println("  dimValues=" + dimValues);
-      }
-      assertNotNull(dimValues);
-
       NumericDocValues idValues = MultiDocValues.getNumericValues(r, "id");
       Bits liveDocs = MultiFields.getLiveDocs(r);
 
       // Verify min/max values are correct:
-      byte[] minValues = dimValues.getMinPackedValue("field");
-      byte[] maxValues = dimValues.getMaxPackedValue("field");
+      byte[] minValues = new byte[numDims*numBytesPerDim];
+      Arrays.fill(minValues, (byte) 0xff);
+
+      byte[] maxValues = new byte[numDims*numBytesPerDim];
+
+      for(LeafReaderContext ctx : r.leaves()) {
+        PointValues dimValues = ctx.reader().getPointValues();
+        if (dimValues == null) {
+          continue;
+        }
+
+        byte[] leafMinValues = dimValues.getMinPackedValue("field");
+        byte[] leafMaxValues = dimValues.getMaxPackedValue("field");
+        for(int dim=0;dim<numDims;dim++) {
+          if (StringHelper.compare(numBytesPerDim, leafMinValues, dim*numBytesPerDim, minValues, dim*numBytesPerDim) < 0) {
+            System.arraycopy(leafMinValues, dim*numBytesPerDim, minValues, dim*numBytesPerDim, numBytesPerDim);
+          }
+          if (StringHelper.compare(numBytesPerDim, leafMaxValues, dim*numBytesPerDim, maxValues, dim*numBytesPerDim) > 0) {
+            System.arraycopy(leafMaxValues, dim*numBytesPerDim, maxValues, dim*numBytesPerDim, numBytesPerDim);
+          }
+        }
+      }
+
       byte[] scratch = new byte[numBytesPerDim];
       for(int dim=0;dim<numDims;dim++) {
-        System.arraycopy(minValues, dim*numBytesPerDim, scratch, 0, scratch.length);
+        System.arraycopy(minValues, dim*numBytesPerDim, scratch, 0, numBytesPerDim);
         //System.out.println("dim=" + dim + " expectedMin=" + new BytesRef(expectedMinValues[dim]) + " min=" + new BytesRef(scratch));
         assertTrue(Arrays.equals(expectedMinValues[dim], scratch));
-        System.arraycopy(maxValues, dim*numBytesPerDim, scratch, 0, scratch.length);
+        System.arraycopy(maxValues, dim*numBytesPerDim, scratch, 0, numBytesPerDim);
         //System.out.println("dim=" + dim + " expectedMax=" + new BytesRef(expectedMaxValues[dim]) + " max=" + new BytesRef(scratch));
         assertTrue(Arrays.equals(expectedMaxValues[dim], scratch));
       }
@@ -716,58 +740,68 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
 
         final BitSet hits = new BitSet();
 
-        dimValues.intersect("field", new PointValues.IntersectVisitor() {
-            @Override
-            public void visit(int docID) {
-              if (liveDocs == null || liveDocs.get(docID)) {
-                hits.set((int) idValues.get(docID));
-              }
-              //System.out.println("visit docID=" + docID);
-            }
+        for(LeafReaderContext ctx : r.leaves()) {
+          PointValues dimValues = ctx.reader().getPointValues();
+          if (dimValues == null) {
+            continue;
+          }
 
-            @Override
-            public void visit(int docID, byte[] packedValue) {
-              if (liveDocs != null && liveDocs.get(docID) == false) {
-                return;
-              }
-              //System.out.println("visit check docID=" + docID + " id=" + idValues.get(docID));
-              for(int dim=0;dim<numDims;dim++) {
-                //System.out.println("  dim=" + dim + " value=" + new BytesRef(packedValue, dim*numBytesPerDim, numBytesPerDim));
-                if (StringHelper.compare(numBytesPerDim, packedValue, dim*numBytesPerDim, queryMin[dim], 0) < 0 ||
-                    StringHelper.compare(numBytesPerDim, packedValue, dim*numBytesPerDim, queryMax[dim], 0) > 0) {
-                  //System.out.println("  no");
-                  return;
+          final int docBase = ctx.docBase;
+
+          dimValues.intersect("field", new PointValues.IntersectVisitor() {
+              @Override
+              public void visit(int docID) {
+                if (liveDocs == null || liveDocs.get(docBase+docID)) {
+                  hits.set((int) idValues.get(docBase+docID));
                 }
+                //System.out.println("visit docID=" + docID);
               }
 
-              //System.out.println("  yes");
-              hits.set((int) idValues.get(docID));
-            }
+              @Override
+              public void visit(int docID, byte[] packedValue) {
+                if (liveDocs != null && liveDocs.get(docBase+docID) == false) {
+                  return;
+                }
 
-            @Override
-            public Relation compare(byte[] minPacked, byte[] maxPacked) {
-              boolean crosses = false;
-              //System.out.println("compare");
-              for(int dim=0;dim<numDims;dim++) {
-                if (StringHelper.compare(numBytesPerDim, maxPacked, dim*numBytesPerDim, queryMin[dim], 0) < 0 ||
-                    StringHelper.compare(numBytesPerDim, minPacked, dim*numBytesPerDim, queryMax[dim], 0) > 0) {
-                  //System.out.println("  query_outside_cell");
-                  return Relation.CELL_OUTSIDE_QUERY;
-                } else if (StringHelper.compare(numBytesPerDim, minPacked, dim*numBytesPerDim, queryMin[dim], 0) < 0 ||
-                           StringHelper.compare(numBytesPerDim, maxPacked, dim*numBytesPerDim, queryMax[dim], 0) > 0) {
-                  crosses = true;
+                //System.out.println("visit check docID=" + docID + " id=" + idValues.get(docID));
+                for(int dim=0;dim<numDims;dim++) {
+                  //System.out.println("  dim=" + dim + " value=" + new BytesRef(packedValue, dim*numBytesPerDim, numBytesPerDim));
+                  if (StringHelper.compare(numBytesPerDim, packedValue, dim*numBytesPerDim, queryMin[dim], 0) < 0 ||
+                      StringHelper.compare(numBytesPerDim, packedValue, dim*numBytesPerDim, queryMax[dim], 0) > 0) {
+                    //System.out.println("  no");
+                    return;
+                  }
                 }
+
+                //System.out.println("  yes");
+                hits.set((int) idValues.get(docBase+docID));
               }
 
-              if (crosses) {
-                //System.out.println("  query_crosses_cell");
-                return Relation.CELL_CROSSES_QUERY;
-              } else {
-                //System.out.println("  cell_inside_query");
-                return Relation.CELL_INSIDE_QUERY;
+              @Override
+              public Relation compare(byte[] minPacked, byte[] maxPacked) {
+                boolean crosses = false;
+                //System.out.println("compare");
+                for(int dim=0;dim<numDims;dim++) {
+                  if (StringHelper.compare(numBytesPerDim, maxPacked, dim*numBytesPerDim, queryMin[dim], 0) < 0 ||
+                      StringHelper.compare(numBytesPerDim, minPacked, dim*numBytesPerDim, queryMax[dim], 0) > 0) {
+                    //System.out.println("  query_outside_cell");
+                    return Relation.CELL_OUTSIDE_QUERY;
+                  } else if (StringHelper.compare(numBytesPerDim, minPacked, dim*numBytesPerDim, queryMin[dim], 0) < 0 ||
+                             StringHelper.compare(numBytesPerDim, maxPacked, dim*numBytesPerDim, queryMax[dim], 0) > 0) {
+                    crosses = true;
+                  }
+                }
+
+                if (crosses) {
+                  //System.out.println("  query_crosses_cell");
+                  return Relation.CELL_CROSSES_QUERY;
+                } else {
+                  //System.out.println("  cell_inside_query");
+                  return Relation.CELL_INSIDE_QUERY;
+                }
               }
-            }
-          });
+            });
+        }
 
         BitSet expected = new BitSet();
         for(int ord=0;ord<numValues;ord++) {
@@ -845,7 +879,7 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
     w.forceMerge(1);
 
     DirectoryReader r = w.getReader();
-    IndexSearcher s = newSearcher(r);
+    IndexSearcher s = newSearcher(r, false);
     assertEquals(2, s.count(IntPoint.newExactQuery("int1", 17)));
     assertEquals(2, s.count(IntPoint.newExactQuery("int2", 42)));
     r.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a96ed9e/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index 462e73e..79eb6280 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -1625,6 +1625,10 @@ public abstract class LuceneTestCase extends Assert {
   }
 
   public static IndexReader wrapReader(IndexReader r) throws IOException {
+    return wrapReader(r, true);
+  }
+
+  public static IndexReader wrapReader(IndexReader r, boolean allowSlowCompositeReader) throws IOException {
     Random random = random();
       
     // TODO: remove this, and fix those tests to wrap before putting slow around:
@@ -1632,10 +1636,12 @@ public abstract class LuceneTestCase extends Assert {
     for (int i = 0, c = random.nextInt(6)+1; i < c; i++) {
       switch(random.nextInt(6)) {
       case 0:
-        if (VERBOSE) {
-          System.out.println("NOTE: LuceneTestCase.wrapReader: wrapping previous reader=" + r + " with SlowCompositeReaderWrapper.wrap");
+        if (allowSlowCompositeReader) {
+          if (VERBOSE) {
+            System.out.println("NOTE: LuceneTestCase.wrapReader: wrapping previous reader=" + r + " with SlowCompositeReaderWrapper.wrap");
+          }
+          r = SlowCompositeReaderWrapper.wrap(r);
         }
-        r = SlowCompositeReaderWrapper.wrap(r);
         break;
       case 1:
         // will create no FC insanity in atomic case, as ParallelLeafReader has own cache key:
@@ -1656,22 +1662,24 @@ public abstract class LuceneTestCase extends Assert {
         r = new FCInvisibleMultiReader(r);
         break;
       case 3:
-        final LeafReader ar = SlowCompositeReaderWrapper.wrap(r);
-        final List<String> allFields = new ArrayList<>();
-        for (FieldInfo fi : ar.getFieldInfos()) {
-          allFields.add(fi.name);
-        }
-        Collections.shuffle(allFields, random);
-        final int end = allFields.isEmpty() ? 0 : random.nextInt(allFields.size());
-        final Set<String> fields = new HashSet<>(allFields.subList(0, end));
-        // will create no FC insanity as ParallelLeafReader has own cache key:
-        if (VERBOSE) {
-          System.out.println("NOTE: LuceneTestCase.wrapReader: wrapping previous reader=" + r + " with ParallelLeafReader(SlowCompositeReaderWapper)");
+        if (allowSlowCompositeReader) {
+          final LeafReader ar = SlowCompositeReaderWrapper.wrap(r);
+          final List<String> allFields = new ArrayList<>();
+          for (FieldInfo fi : ar.getFieldInfos()) {
+            allFields.add(fi.name);
+          }
+          Collections.shuffle(allFields, random);
+          final int end = allFields.isEmpty() ? 0 : random.nextInt(allFields.size());
+          final Set<String> fields = new HashSet<>(allFields.subList(0, end));
+          // will create no FC insanity as ParallelLeafReader has own cache key:
+          if (VERBOSE) {
+            System.out.println("NOTE: LuceneTestCase.wrapReader: wrapping previous reader=" + r + " with ParallelLeafReader(SlowCompositeReaderWapper)");
+          }
+          r = new ParallelLeafReader(
+                                     new FieldFilterLeafReader(ar, fields, false),
+                                     new FieldFilterLeafReader(ar, fields, true)
+                                     );
         }
-        r = new ParallelLeafReader(
-                                   new FieldFilterLeafReader(ar, fields, false),
-                                   new FieldFilterLeafReader(ar, fields, true)
-                                   );
         break;
       case 4:
         // Häckidy-Hick-Hack: a standard Reader will cause FC insanity, so we use
@@ -1701,7 +1709,9 @@ public abstract class LuceneTestCase extends Assert {
       }
     }
     if (wasOriginallyAtomic) {
-      r = SlowCompositeReaderWrapper.wrap(r);
+      if (allowSlowCompositeReader) {
+        r = SlowCompositeReaderWrapper.wrap(r);
+      }
     } else if ((r instanceof CompositeReader) && !(r instanceof FCInvisibleMultiReader)) {
       // prevent cache insanity caused by e.g. ParallelCompositeReader, to fix we wrap one more time:
       r = new FCInvisibleMultiReader(r);
@@ -2588,40 +2598,45 @@ public abstract class LuceneTestCase extends Assert {
   }
 
   // naive silly memory heavy uninversion!!  maps docID -> packed values (a Set because a given doc can be multi-valued)
-  private Map<Integer,Set<BytesRef>> uninvert(String fieldName, PointValues points) throws IOException {
+  private Map<Integer,Set<BytesRef>> uninvert(String fieldName, IndexReader reader) throws IOException {
     final Map<Integer,Set<BytesRef>> docValues = new HashMap<>();
-    points.intersect(fieldName, new PointValues.IntersectVisitor() {
-        @Override
-        public void visit(int docID) {
-          throw new UnsupportedOperationException();
-        }
+    for(LeafReaderContext ctx : reader.leaves()) {
 
-        @Override
-        public void visit(int docID, byte[] packedValue) throws IOException {
-          if (docValues.containsKey(docID) == false) {
-            docValues.put(docID, new HashSet<BytesRef>());
-          }
-          docValues.get(docID).add(new BytesRef(packedValue.clone()));
-        }
+      PointValues points = ctx.reader().getPointValues();
+      if (points == null) {
+        continue;
+      }
+
+      points.intersect(fieldName,
+                       new PointValues.IntersectVisitor() {
+                         @Override
+                         public void visit(int docID) {
+                           throw new UnsupportedOperationException();
+                         }
+
+                         @Override
+                         public void visit(int docID, byte[] packedValue) throws IOException {
+                           int topDocID = ctx.docBase + docID;
+                           if (docValues.containsKey(topDocID) == false) {
+                             docValues.put(topDocID, new HashSet<BytesRef>());
+                           }
+                           docValues.get(topDocID).add(new BytesRef(packedValue.clone()));
+                         }
+
+                         @Override
+                         public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                           // We pretend our query shape is so hairy that it crosses every single cell:
+                           return PointValues.Relation.CELL_CROSSES_QUERY;
+                         }
+                       });
+    }
 
-        @Override
-        public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-          // We pretend our query shape is so hairy that it crosses every single cell:
-          return PointValues.Relation.CELL_CROSSES_QUERY;
-        }
-      });
     return docValues;
   }
 
   public void assertPointsEquals(String info, IndexReader leftReader, IndexReader rightReader) throws IOException {
-    assertPointsEquals(info,
-                       MultiFields.getMergedFieldInfos(leftReader),
-                       MultiPointValues.get(leftReader),
-                       MultiFields.getMergedFieldInfos(rightReader),
-                       MultiPointValues.get(rightReader));
-  }
-
-  public void assertPointsEquals(String info, FieldInfos fieldInfos1, PointValues points1, FieldInfos fieldInfos2, PointValues points2) throws IOException {
+    FieldInfos fieldInfos1 = MultiFields.getMergedFieldInfos(leftReader);
+    FieldInfos fieldInfos2 = MultiFields.getMergedFieldInfos(rightReader);
     for(FieldInfo fieldInfo1 : fieldInfos1) {
       if (fieldInfo1.getPointDimensionCount() != 0) {
         FieldInfo fieldInfo2 = fieldInfos2.fieldInfo(fieldInfo1.name);
@@ -2631,8 +2646,8 @@ public abstract class LuceneTestCase extends Assert {
         assertEquals(info, fieldInfo2.getPointNumBytes(), fieldInfo2.getPointNumBytes());
 
         assertEquals(info + " field=" + fieldInfo1.name,
-                     uninvert(fieldInfo1.name, points1),
-                     uninvert(fieldInfo1.name, points2));
+                     uninvert(fieldInfo1.name, leftReader),
+                     uninvert(fieldInfo1.name, rightReader));
       }
     }
 


[2/9] lucene-solr git commit: SOLR-8145: Fix position of OOM killer script when starting Solr in the background (cherry picked from commit e1033d9)

Posted by sh...@apache.org.
SOLR-8145: Fix position of OOM killer script when starting Solr in the background
(cherry picked from commit e1033d9)


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

Branch: refs/heads/branch_6_0
Commit: cc8af0d3e8e9bbd933947dae7307d0b09eb146da
Parents: 3a96ed9
Author: thelabdude <th...@gmail.com>
Authored: Wed Mar 2 23:52:27 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Mar 3 20:48:21 2016 +0530

----------------------------------------------------------------------
 solr/bin/solr | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cc8af0d3/solr/bin/solr
----------------------------------------------------------------------
diff --git a/solr/bin/solr b/solr/bin/solr
index 904c2c3..9cf8ae5 100755
--- a/solr/bin/solr
+++ b/solr/bin/solr
@@ -1420,8 +1420,9 @@ function launch_solr() {
     exec "$JAVA" "${SOLR_START_OPTS[@]}" $SOLR_ADDL_ARGS -jar start.jar "${SOLR_JETTY_CONFIG[@]}"
   else
     # run Solr in the background
-    nohup "$JAVA" "${SOLR_START_OPTS[@]}" $SOLR_ADDL_ARGS -jar start.jar \
-	"-XX:OnOutOfMemoryError=$SOLR_TIP/bin/oom_solr.sh $SOLR_PORT $SOLR_LOGS_DIR" "${SOLR_JETTY_CONFIG[@]}" \
+    nohup "$JAVA" "${SOLR_START_OPTS[@]}" $SOLR_ADDL_ARGS \
+	"-XX:OnOutOfMemoryError=$SOLR_TIP/bin/oom_solr.sh $SOLR_PORT $SOLR_LOGS_DIR" \
+        -jar start.jar "${SOLR_JETTY_CONFIG[@]}" \
 	1>"$SOLR_LOGS_DIR/solr-$SOLR_PORT-console.log" 2>&1 & echo $! > "$SOLR_PID_DIR/solr-$SOLR_PORT.pid"
 
     # no lsof on cygwin though


[7/9] lucene-solr git commit: LUCENE-7060: Spatial4j 0.6 upgrade. Package com.spatial4j.core -> org.locationtech.spatial4j (cherry picked from commit 569b6ca) (cherry picked from commit b90dbd4)

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/PortedSolr3Test.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/PortedSolr3Test.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/PortedSolr3Test.java
index 8506c86..a081497 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/PortedSolr3Test.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/PortedSolr3Test.java
@@ -22,10 +22,10 @@ import java.util.List;
 import java.util.Set;
 
 import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;
 import org.apache.lucene.spatial.prefix.TermQueryPrefixTreeStrategy;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/QueryEqualsHashCodeTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/QueryEqualsHashCodeTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/QueryEqualsHashCodeTest.java
index b1a5e54..5dbb8f8 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/QueryEqualsHashCodeTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/QueryEqualsHashCodeTest.java
@@ -19,8 +19,8 @@ package org.apache.lucene.spatial;
 import java.util.ArrayList;
 import java.util.Collection;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.spatial.bbox.BBoxStrategy;
 import org.apache.lucene.spatial.composite.CompositeSpatialStrategy;
 import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialArgsTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialArgsTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialArgsTest.java
index 09b5d46..094953a 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialArgsTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialArgsTest.java
@@ -16,8 +16,8 @@
  */
 package org.apache.lucene.spatial;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.spatial.query.SpatialArgs;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialExample.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialExample.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialExample.java
index 1bd7159..76e0200 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialExample.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialExample.java
@@ -18,10 +18,10 @@ package org.apache.lucene.spatial;
 
 import java.io.IOException;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestCase.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestCase.java
index 94e5a8e..529e98b 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestCase.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestCase.java
@@ -24,10 +24,10 @@ import java.util.Map;
 import java.util.Random;
 import java.util.logging.Logger;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestData.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestData.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestData.java
index 27d47b3..06a68ce 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestData.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestData.java
@@ -16,8 +16,8 @@
  */
 package org.apache.lucene.spatial;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Shape;
 
 import java.io.BufferedReader;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestQuery.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestQuery.java
index bac90cf..47e9c12 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestQuery.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/SpatialTestQuery.java
@@ -16,7 +16,7 @@
  */
 package org.apache.lucene.spatial;
 
-import com.spatial4j.core.context.SpatialContext;
+import org.locationtech.spatial4j.context.SpatialContext;
 
 import org.apache.lucene.spatial.query.SpatialArgs;
 import org.apache.lucene.spatial.query.SpatialArgsParser;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/StrategyTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/StrategyTestCase.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/StrategyTestCase.java
index 00e437b..df37d18 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/StrategyTestCase.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/StrategyTestCase.java
@@ -29,8 +29,8 @@ import java.util.List;
 import java.util.Set;
 import java.util.logging.Logger;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.StoredField;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/TestTestFramework.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/TestTestFramework.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/TestTestFramework.java
index d31fdf6..6af7467 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/TestTestFramework.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/TestTestFramework.java
@@ -16,8 +16,8 @@
  */
 package org.apache.lucene.spatial;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Rectangle;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Rectangle;
 import org.apache.lucene.spatial.query.SpatialArgsParser;
 import org.apache.lucene.spatial.query.SpatialOperation;
 import org.apache.lucene.util.LuceneTestCase;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/bbox/TestBBoxStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/bbox/TestBBoxStrategy.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/bbox/TestBBoxStrategy.java
index 6140996..20a7202 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/bbox/TestBBoxStrategy.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/bbox/TestBBoxStrategy.java
@@ -19,12 +19,12 @@ package org.apache.lucene.spatial.bbox;
 import java.io.IOException;
 
 import com.carrotsearch.randomizedtesting.annotations.Repeat;
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.context.SpatialContextFactory;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.impl.RectangleImpl;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.context.SpatialContextFactory;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.impl.RectangleImpl;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.IndexOptions;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/composite/CompositeStrategyTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/composite/CompositeStrategyTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/composite/CompositeStrategyTest.java
index 8e1bb51..7d49e8b 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/composite/CompositeStrategyTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/composite/CompositeStrategyTest.java
@@ -19,12 +19,12 @@ package org.apache.lucene.spatial.composite;
 import java.io.IOException;
 
 import com.carrotsearch.randomizedtesting.annotations.Repeat;
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.context.SpatialContextFactory;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.impl.RectangleImpl;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.context.SpatialContextFactory;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.impl.RectangleImpl;
 import org.apache.lucene.spatial.prefix.RandomSpatialOpStrategyTestCase;
 import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;
 import org.apache.lucene.spatial.prefix.tree.GeohashPrefixTree;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java
index 7cd4723..33c8a33 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/DateNRStrategyTest.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.util.Calendar;
 
 import com.carrotsearch.randomizedtesting.annotations.Repeat;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.spatial.prefix.tree.DateRangePrefixTree;
 import org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape;
 import org.apache.lucene.spatial.query.SpatialOperation;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/HeatmapFacetCounterTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/HeatmapFacetCounterTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/HeatmapFacetCounterTest.java
index 124af79..2de18cc 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/HeatmapFacetCounterTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/HeatmapFacetCounterTest.java
@@ -21,15 +21,15 @@ import java.util.ArrayList;
 import java.util.List;
 
 import com.carrotsearch.randomizedtesting.annotations.Repeat;
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.context.SpatialContextFactory;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Circle;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
-import com.spatial4j.core.shape.impl.RectangleImpl;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.context.SpatialContextFactory;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Circle;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.impl.RectangleImpl;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TotalHitCountCollector;
 import org.apache.lucene.spatial.StrategyTestCase;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/JtsPolygonTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/JtsPolygonTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/JtsPolygonTest.java
index 09fb3a9..3f1684f 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/JtsPolygonTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/JtsPolygonTest.java
@@ -16,9 +16,9 @@
  */
 package org.apache.lucene.spatial.prefix;
 
-import com.spatial4j.core.context.SpatialContextFactory;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContextFactory;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Field.Store;
@@ -45,7 +45,7 @@ public class JtsPolygonTest extends StrategyTestCase {
     try {
       HashMap<String, String> args = new HashMap<>();
       args.put("spatialContextFactory",
-          "com.spatial4j.core.context.jts.JtsSpatialContextFactory");
+          "org.locationtech.spatial4j.context.jts.JtsSpatialContextFactory");
       ctx = SpatialContextFactory.makeSpatialContext(args, getClass().getClassLoader());
     } catch (NoClassDefFoundError e) {
       assumeTrue("This test requires JTS jar: "+e, false);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
index 11e1d18..514c18e 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/NumberRangeFacetsTest.java
@@ -23,7 +23,7 @@ import java.util.Collections;
 import java.util.List;
 
 import com.carrotsearch.randomizedtesting.annotations.Repeat;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.TermsQuery;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java
index 8db131c..c7e107c 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java
@@ -29,14 +29,14 @@ import java.util.Map;
 import java.util.Set;
 
 import com.carrotsearch.randomizedtesting.annotations.Repeat;
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.context.SpatialContextFactory;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.ShapeCollection;
-import com.spatial4j.core.shape.SpatialRelation;
-import com.spatial4j.core.shape.impl.RectangleImpl;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.context.SpatialContextFactory;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.ShapeCollection;
+import org.locationtech.spatial4j.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.impl.RectangleImpl;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.StoredField;
@@ -56,10 +56,10 @@ import org.junit.Test;
 import static com.carrotsearch.randomizedtesting.RandomizedTest.randomBoolean;
 import static com.carrotsearch.randomizedtesting.RandomizedTest.randomInt;
 import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
-import static com.spatial4j.core.shape.SpatialRelation.CONTAINS;
-import static com.spatial4j.core.shape.SpatialRelation.DISJOINT;
-import static com.spatial4j.core.shape.SpatialRelation.INTERSECTS;
-import static com.spatial4j.core.shape.SpatialRelation.WITHIN;
+import static org.locationtech.spatial4j.shape.SpatialRelation.CONTAINS;
+import static org.locationtech.spatial4j.shape.SpatialRelation.DISJOINT;
+import static org.locationtech.spatial4j.shape.SpatialRelation.INTERSECTS;
+import static org.locationtech.spatial4j.shape.SpatialRelation.WITHIN;
 
 /** Randomized PrefixTree test that considers the fuzziness of the
  * results introduced by grid approximation. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpStrategyTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpStrategyTestCase.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpStrategyTestCase.java
index 87f1071..22c5839 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpStrategyTestCase.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpStrategyTestCase.java
@@ -22,7 +22,7 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.spatial.StrategyTestCase;
 import org.apache.lucene.spatial.query.SpatialArgs;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/TestRecursivePrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/TestRecursivePrefixTreeStrategy.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/TestRecursivePrefixTreeStrategy.java
index a53d52d..f852464 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/TestRecursivePrefixTreeStrategy.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/TestRecursivePrefixTreeStrategy.java
@@ -16,10 +16,10 @@
  */
 package org.apache.lucene.spatial.prefix;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.spatial.SpatialMatchConcern;
 import org.apache.lucene.spatial.StrategyTestCase;
 import org.apache.lucene.spatial.prefix.tree.GeohashPrefixTree;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/TestTermQueryPrefixGridStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/TestTermQueryPrefixGridStrategy.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/TestTermQueryPrefixGridStrategy.java
index 1a912c0..fc131c5 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/TestTermQueryPrefixGridStrategy.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/TestTermQueryPrefixGridStrategy.java
@@ -16,8 +16,8 @@
  */
 package org.apache.lucene.spatial.prefix;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.StoredField;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTreeTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTreeTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTreeTest.java
index 74a989e..12e9744 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTreeTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTreeTest.java
@@ -21,8 +21,8 @@ import java.util.Arrays;
 import java.util.Calendar;
 import java.util.GregorianCalendar;
 
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
 import org.apache.lucene.spatial.prefix.tree.NumberRangePrefixTree.UnitNRShape;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java
index 403b8d1..8a3d79c 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java
@@ -16,10 +16,10 @@
  */
 package org.apache.lucene.spatial.prefix.tree;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Field.Store;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/query/SpatialArgsParserTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/query/SpatialArgsParserTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/query/SpatialArgsParserTest.java
index 93b95f3..9724c65 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/query/SpatialArgsParserTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/query/SpatialArgsParserTest.java
@@ -16,8 +16,8 @@
  */
 package org.apache.lucene.spatial.query;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Rectangle;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Rectangle;
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/serialized/SerializedStrategyTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/serialized/SerializedStrategyTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/serialized/SerializedStrategyTest.java
index bed8339..6a73d23 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/serialized/SerializedStrategyTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/serialized/SerializedStrategyTest.java
@@ -18,7 +18,7 @@ package org.apache.lucene.spatial.serialized;
 
 import java.io.IOException;
 
-import com.spatial4j.core.context.SpatialContext;
+import org.locationtech.spatial4j.context.SpatialContext;
 import org.apache.lucene.spatial.SpatialMatchConcern;
 import org.apache.lucene.spatial.StrategyTestCase;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dRptTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dRptTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dRptTest.java
index 8040a35..d26bb29 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dRptTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dRptTest.java
@@ -21,10 +21,10 @@ import java.util.ArrayList;
 import java.util.List;
 
 import com.carrotsearch.randomizedtesting.annotations.Repeat;
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.spatial.composite.CompositeSpatialStrategy;
 import org.apache.lucene.spatial.prefix.RandomSpatialOpStrategyTestCase;
 import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;
@@ -41,7 +41,7 @@ import org.apache.lucene.geo3d.GeoShape;
 import org.apache.lucene.geo3d.PlanetModel;
 import org.junit.Test;
 
-import static com.spatial4j.core.distance.DistanceUtils.DEGREES_TO_RADIANS;
+import static org.locationtech.spatial4j.distance.DistanceUtils.DEGREES_TO_RADIANS;
 
 public class Geo3dRptTest extends RandomSpatialOpStrategyTestCase {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java
index 58b520d..134b8c7 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeRectRelationTestCase.java
@@ -19,12 +19,12 @@ package org.apache.lucene.spatial.spatial4j;
 import java.util.ArrayList;
 import java.util.List;
 
-import com.spatial4j.core.TestLog;
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Circle;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.RectIntersectionTestHelper;
+import org.locationtech.spatial4j.TestLog;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Circle;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.RectIntersectionTestHelper;
 import org.apache.lucene.geo3d.LatLonBounds;
 import org.apache.lucene.geo3d.GeoBBox;
 import org.apache.lucene.geo3d.GeoBBoxFactory;
@@ -37,7 +37,7 @@ import org.apache.lucene.geo3d.PlanetModel;
 import org.junit.Rule;
 import org.junit.Test;
 
-import static com.spatial4j.core.distance.DistanceUtils.DEGREES_TO_RADIANS;
+import static org.locationtech.spatial4j.distance.DistanceUtils.DEGREES_TO_RADIANS;
 
 public abstract class Geo3dShapeRectRelationTestCase extends RandomizedShapeTestCase {
   protected final static double RADIANS_PER_DEGREE = Math.PI/180.0;
@@ -110,7 +110,6 @@ public abstract class Geo3dShapeRectRelationTestCase extends RandomizedShapeTest
     }
   }
 
-  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-6867")
   @Test
   public void testGeoCircleRect() {
     new Geo3dRectIntersectionTestHelper(ctx) {
@@ -133,7 +132,6 @@ public abstract class Geo3dShapeRectRelationTestCase extends RandomizedShapeTest
     }.testRelateWithRectangle();
   }
 
-  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-6867")
   @Test
   public void testGeoBBoxRect() {
     new Geo3dRectIntersectionTestHelper(ctx) {
@@ -168,7 +166,6 @@ public abstract class Geo3dShapeRectRelationTestCase extends RandomizedShapeTest
     }.testRelateWithRectangle();
   }
 
-  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-6867")
   @Test
   public void testGeoPolygonRect() {
     new Geo3dRectIntersectionTestHelper(ctx) {
@@ -212,7 +209,6 @@ public abstract class Geo3dShapeRectRelationTestCase extends RandomizedShapeTest
     }.testRelateWithRectangle();
   }
 
-  @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/LUCENE-6867")
   @Test
   public void testGeoPathRect() {
     new Geo3dRectIntersectionTestHelper(ctx) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeSphereModelRectRelationTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeSphereModelRectRelationTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeSphereModelRectRelationTest.java
index aac0a0a..2d95823 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeSphereModelRectRelationTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/Geo3dShapeSphereModelRectRelationTest.java
@@ -19,7 +19,7 @@ package org.apache.lucene.spatial.spatial4j;
 import java.util.ArrayList;
 import java.util.List;
 
-import com.spatial4j.core.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Rectangle;
 import org.apache.lucene.geo3d.GeoArea;
 import org.apache.lucene.geo3d.GeoBBox;
 import org.apache.lucene.geo3d.GeoBBoxFactory;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/RandomizedShapeTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/RandomizedShapeTestCase.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/RandomizedShapeTestCase.java
index 40d1b24..0c18f5d 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/RandomizedShapeTestCase.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/RandomizedShapeTestCase.java
@@ -16,17 +16,17 @@
  */
 package org.apache.lucene.spatial.spatial4j;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Circle;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
-import com.spatial4j.core.shape.impl.Range;
-
-import static com.spatial4j.core.shape.SpatialRelation.CONTAINS;
-import static com.spatial4j.core.shape.SpatialRelation.WITHIN;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Circle;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.impl.Range;
+
+import static org.locationtech.spatial4j.shape.SpatialRelation.CONTAINS;
+import static org.locationtech.spatial4j.shape.SpatialRelation.WITHIN;
 
 import org.apache.lucene.util.LuceneTestCase;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/geo3d/GeoPointTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/geo3d/GeoPointTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/geo3d/GeoPointTest.java
index 1d559da..4446474 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/geo3d/GeoPointTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/spatial4j/geo3d/GeoPointTest.java
@@ -21,7 +21,7 @@ import org.apache.lucene.geo3d.PlanetModel;
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.Test;
 
-import com.spatial4j.core.distance.DistanceUtils;
+import org.locationtech.spatial4j.distance.DistanceUtils;
 
 import static com.carrotsearch.randomizedtesting.RandomizedTest.randomFloat;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/vector/TestPointVectorStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/vector/TestPointVectorStrategy.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/vector/TestPointVectorStrategy.java
index d62a0a8..69f8c4d 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/vector/TestPointVectorStrategy.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/vector/TestPointVectorStrategy.java
@@ -16,9 +16,9 @@
  */
 package org.apache.lucene.spatial.vector;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Circle;
-import com.spatial4j.core.shape.Point;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Circle;
+import org.locationtech.spatial4j.shape.Point;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.spatial.SpatialMatchConcern;
 import org.apache.lucene.spatial.StrategyTestCase;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/ivy.xml
----------------------------------------------------------------------
diff --git a/solr/core/ivy.xml b/solr/core/ivy.xml
index 2936c5b..5f8706f 100644
--- a/solr/core/ivy.xml
+++ b/solr/core/ivy.xml
@@ -36,7 +36,7 @@
     <dependency org="commons-cli" name="commons-cli" rev="${/commons-cli/commons-cli}" conf="compile"/>
     <dependency org="commons-lang" name="commons-lang" rev="${/commons-lang/commons-lang}" conf="compile"/>
     <dependency org="com.google.guava" name="guava" rev="${/com.google.guava/guava}" conf="compile"/>
-    <dependency org="com.spatial4j" name="spatial4j" rev="${/com.spatial4j/spatial4j}" conf="compile"/>
+    <dependency org="org.locationtech.spatial4j" name="spatial4j" rev="${/org.locationtech.spatial4j/spatial4j}" conf="compile"/>
     <dependency org="org.antlr" name="antlr4-runtime" rev="${/org.antlr/antlr4-runtime}"/>
     <dependency org="org.apache.commons" name="commons-math3" rev="${/org.apache.commons/commons-math3}" conf="test"/>
     <dependency org="org.ow2.asm" name="asm" rev="${/org.ow2.asm/asm}" conf="compile"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/handler/component/SpatialHeatmapFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SpatialHeatmapFacets.java b/solr/core/src/java/org/apache/solr/handler/component/SpatialHeatmapFacets.java
index dc1b9af..4ad882c 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/SpatialHeatmapFacets.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/SpatialHeatmapFacets.java
@@ -32,8 +32,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.spatial.prefix.HeatmapFacetCounter;
 import org.apache.lucene.spatial.prefix.PrefixTreeStrategy;
 import org.apache.lucene.spatial.query.SpatialArgs;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java b/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
index 83fd447..222f0b8 100644
--- a/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
@@ -57,12 +57,12 @@ import org.slf4j.LoggerFactory;
 import com.google.common.base.Throwables;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.context.SpatialContextFactory;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.context.SpatialContextFactory;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
 
 /**
  * Abstract base class for Solr FieldTypes based on a Lucene 4 {@link SpatialStrategy}.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/schema/AbstractSpatialPrefixTreeFieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/AbstractSpatialPrefixTreeFieldType.java b/solr/core/src/java/org/apache/solr/schema/AbstractSpatialPrefixTreeFieldType.java
index 164398b..abf55f1 100644
--- a/solr/core/src/java/org/apache/solr/schema/AbstractSpatialPrefixTreeFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/AbstractSpatialPrefixTreeFieldType.java
@@ -31,7 +31,7 @@ import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTreeFactory;
 import org.apache.lucene.spatial.query.SpatialArgsParser;
 import org.apache.solr.util.MapListener;
 
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Shape;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/schema/BBoxField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/BBoxField.java b/solr/core/src/java/org/apache/solr/schema/BBoxField.java
index 2f282c8..f655e68 100644
--- a/solr/core/src/java/org/apache/solr/schema/BBoxField.java
+++ b/solr/core/src/java/org/apache/solr/schema/BBoxField.java
@@ -22,7 +22,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 
-import com.spatial4j.core.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Rectangle;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.spatial.bbox.BBoxOverlapRatioValueSource;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/schema/DateRangeField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/DateRangeField.java b/solr/core/src/java/org/apache/solr/schema/DateRangeField.java
index 38b6800..95b441a 100644
--- a/solr/core/src/java/org/apache/solr/schema/DateRangeField.java
+++ b/solr/core/src/java/org/apache/solr/schema/DateRangeField.java
@@ -22,7 +22,7 @@ import java.util.Date;
 import java.util.List;
 import java.util.Map;
 
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Shape;
 
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.search.Query;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/schema/GeoHashField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/GeoHashField.java b/solr/core/src/java/org/apache/solr/schema/GeoHashField.java
index 7deae5f..2baf72e 100644
--- a/solr/core/src/java/org/apache/solr/schema/GeoHashField.java
+++ b/solr/core/src/java/org/apache/solr/schema/GeoHashField.java
@@ -18,10 +18,10 @@ package org.apache.solr.schema;
 
 import java.io.IOException;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.io.GeohashUtils;
-import com.spatial4j.core.shape.Point;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.io.GeohashUtils;
+import org.locationtech.spatial4j.shape.Point;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.LiteralValueSource;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/schema/LatLonType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/LatLonType.java b/solr/core/src/java/org/apache/solr/schema/LatLonType.java
index 8592c5a..c30729a 100644
--- a/solr/core/src/java/org/apache/solr/schema/LatLonType.java
+++ b/solr/core/src/java/org/apache/solr/schema/LatLonType.java
@@ -47,10 +47,10 @@ import org.apache.solr.search.QParser;
 import org.apache.solr.search.SpatialOptions;
 import org.apache.solr.util.SpatialUtils;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/schema/PointType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/PointType.java b/solr/core/src/java/org/apache/solr/schema/PointType.java
index 9345231..b2f15ab 100644
--- a/solr/core/src/java/org/apache/solr/schema/PointType.java
+++ b/solr/core/src/java/org/apache/solr/schema/PointType.java
@@ -21,7 +21,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import com.spatial4j.core.distance.DistanceUtils;
+import org.locationtech.spatial4j.distance.DistanceUtils;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.queries.function.ValueSource;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/schema/RptWithGeometrySpatialField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/RptWithGeometrySpatialField.java b/solr/core/src/java/org/apache/solr/schema/RptWithGeometrySpatialField.java
index fe4cedc..b633174 100644
--- a/solr/core/src/java/org/apache/solr/schema/RptWithGeometrySpatialField.java
+++ b/solr/core/src/java/org/apache/solr/schema/RptWithGeometrySpatialField.java
@@ -20,9 +20,9 @@ import java.io.IOException;
 import java.lang.ref.WeakReference;
 import java.util.Map;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.jts.JtsGeometry;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.jts.JtsGeometry;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
index 2c05f33..0830267 100644
--- a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
+++ b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
@@ -16,7 +16,7 @@
  */
 package org.apache.solr.search;
 
-import com.spatial4j.core.distance.DistanceUtils;
+import org.locationtech.spatial4j.distance.DistanceUtils;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/search/function/distance/GeoDistValueSourceParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/function/distance/GeoDistValueSourceParser.java b/solr/core/src/java/org/apache/solr/search/function/distance/GeoDistValueSourceParser.java
index 1f4ea34..956550c 100644
--- a/solr/core/src/java/org/apache/solr/search/function/distance/GeoDistValueSourceParser.java
+++ b/solr/core/src/java/org/apache/solr/search/function/distance/GeoDistValueSourceParser.java
@@ -20,9 +20,9 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Point;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Point;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.ConstNumberSource;
 import org.apache.lucene.queries.function.valuesource.DoubleConstValueSource;
@@ -41,7 +41,7 @@ import org.apache.solr.util.SpatialUtils;
 
 /**
  * Parses "geodist" creating {@link HaversineConstFunction} or {@link HaversineFunction}
- * or calling {@link SpatialStrategy#makeDistanceValueSource(com.spatial4j.core.shape.Point,double)}.
+ * or calling {@link SpatialStrategy#makeDistanceValueSource(org.locationtech.spatial4j.shape.Point,double)}.
  */
 public class GeoDistValueSourceParser extends ValueSourceParser {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/search/function/distance/GeohashFunction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/function/distance/GeohashFunction.java b/solr/core/src/java/org/apache/solr/search/function/distance/GeohashFunction.java
index c4a7bd5..b00f2fd 100644
--- a/solr/core/src/java/org/apache/solr/search/function/distance/GeohashFunction.java
+++ b/solr/core/src/java/org/apache/solr/search/function/distance/GeohashFunction.java
@@ -18,7 +18,7 @@ package org.apache.solr.search.function.distance;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
-import com.spatial4j.core.io.GeohashUtils;
+import org.locationtech.spatial4j.io.GeohashUtils;
 
 import java.util.Map;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/search/function/distance/GeohashHaversineFunction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/function/distance/GeohashHaversineFunction.java b/solr/core/src/java/org/apache/solr/search/function/distance/GeohashHaversineFunction.java
index 4dd49e4..915db33 100644
--- a/solr/core/src/java/org/apache/solr/search/function/distance/GeohashHaversineFunction.java
+++ b/solr/core/src/java/org/apache/solr/search/function/distance/GeohashHaversineFunction.java
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 package org.apache.solr.search.function.distance;
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.distance.GeodesicSphereDistCalc;
-import com.spatial4j.core.io.GeohashUtils;
-import com.spatial4j.core.shape.Point;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.distance.GeodesicSphereDistCalc;
+import org.locationtech.spatial4j.io.GeohashUtils;
+import org.locationtech.spatial4j.shape.Point;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/search/function/distance/HaversineConstFunction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/function/distance/HaversineConstFunction.java b/solr/core/src/java/org/apache/solr/search/function/distance/HaversineConstFunction.java
index f9ac062..4b68f5c 100644
--- a/solr/core/src/java/org/apache/solr/search/function/distance/HaversineConstFunction.java
+++ b/solr/core/src/java/org/apache/solr/search/function/distance/HaversineConstFunction.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 package org.apache.solr.search.function.distance;
-import com.spatial4j.core.distance.DistanceUtils;
+import org.locationtech.spatial4j.distance.DistanceUtils;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
@@ -26,7 +26,7 @@ import org.apache.lucene.search.IndexSearcher;
 import java.io.IOException;
 import java.util.Map;
 
-import static com.spatial4j.core.distance.DistanceUtils.DEGREES_TO_RADIANS;
+import static org.locationtech.spatial4j.distance.DistanceUtils.DEGREES_TO_RADIANS;
 
 /**
  * Haversine function with one point constant

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/search/function/distance/HaversineFunction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/function/distance/HaversineFunction.java b/solr/core/src/java/org/apache/solr/search/function/distance/HaversineFunction.java
index c0eb044..20ed5a7 100644
--- a/solr/core/src/java/org/apache/solr/search/function/distance/HaversineFunction.java
+++ b/solr/core/src/java/org/apache/solr/search/function/distance/HaversineFunction.java
@@ -21,7 +21,7 @@ import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
 import org.apache.lucene.queries.function.valuesource.MultiValueSource;
 import org.apache.lucene.search.IndexSearcher;
-import com.spatial4j.core.distance.DistanceUtils;
+import org.locationtech.spatial4j.distance.DistanceUtils;
 import org.apache.solr.common.SolrException;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/util/DistanceUnits.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/DistanceUnits.java b/solr/core/src/java/org/apache/solr/util/DistanceUnits.java
index 63f43e4..a2163ac 100644
--- a/solr/core/src/java/org/apache/solr/util/DistanceUnits.java
+++ b/solr/core/src/java/org/apache/solr/util/DistanceUnits.java
@@ -21,7 +21,7 @@ import java.util.Map;
 import java.util.Set;
 
 import com.google.common.collect.ImmutableMap;
-import com.spatial4j.core.distance.DistanceUtils;
+import org.locationtech.spatial4j.distance.DistanceUtils;
 import org.apache.solr.schema.AbstractSpatialFieldType;
 
 /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/java/org/apache/solr/util/SpatialUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/SpatialUtils.java b/solr/core/src/java/org/apache/solr/util/SpatialUtils.java
index 1b05d01..9f6019c 100644
--- a/solr/core/src/java/org/apache/solr/util/SpatialUtils.java
+++ b/solr/core/src/java/org/apache/solr/util/SpatialUtils.java
@@ -18,11 +18,11 @@ package org.apache.solr.util;
 
 import java.text.ParseException;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.exception.InvalidShapeException;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.exception.InvalidShapeException;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.solr.common.SolrException;
 
 /** Utility methods pertaining to spatial. */
@@ -32,7 +32,7 @@ public class SpatialUtils {
 
   /**
    * Parses a 'geom' parameter (might also be used to parse shapes for indexing). {@code geomStr} can either be WKT or
-   * a rectangle-range syntax (see {@link #parseRectangle(String, com.spatial4j.core.context.SpatialContext)}.
+   * a rectangle-range syntax (see {@link #parseRectangle(String, org.locationtech.spatial4j.context.SpatialContext)}.
    */
   public static Shape parseGeomSolrException(String geomStr, SpatialContext ctx) {
     if (geomStr.length() == 0) {
@@ -100,7 +100,7 @@ public class SpatialUtils {
     return idx;
   }
 
-  /** Calls {@link #parsePoint(String, com.spatial4j.core.context.SpatialContext)} and wraps
+  /** Calls {@link #parsePoint(String, org.locationtech.spatial4j.context.SpatialContext)} and wraps
    * the exception with {@link org.apache.solr.common.SolrException} with a helpful message. */
   public static Point parsePointSolrException(String externalVal, SpatialContext ctx) throws SolrException {
     try {
@@ -116,7 +116,7 @@ public class SpatialUtils {
   /**
    * Parses {@code str} in the format of '[minPoint TO maxPoint]' where {@code minPoint} is the lower left corner
    * and maxPoint is the upper-right corner of the bounding box.  Both corners may optionally be wrapped with a quote
-   * and then it's parsed via {@link #parsePoint(String, com.spatial4j.core.context.SpatialContext)}.
+   * and then it's parsed via {@link #parsePoint(String, org.locationtech.spatial4j.context.SpatialContext)}.
    * @param str Non-null; may *not* have leading or trailing spaces
    * @param ctx Non-null
    * @return the Rectangle
@@ -140,7 +140,7 @@ public class SpatialUtils {
   }
 
   /**
-   * Calls {@link #parseRectangle(String, com.spatial4j.core.context.SpatialContext)} and wraps the exception with
+   * Calls {@link #parseRectangle(String, org.locationtech.spatial4j.context.SpatialContext)} and wraps the exception with
    * {@link org.apache.solr.common.SolrException} with a helpful message.
    */
   public static Rectangle parseRectangeSolrException(String externalVal, SpatialContext ctx) throws SolrException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java b/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java
index 20555c6..2fe3740 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java
@@ -21,10 +21,10 @@ import java.util.Arrays;
 
 import com.carrotsearch.randomizedtesting.RandomizedTest;
 import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
 import org.apache.lucene.spatial.bbox.BBoxStrategy;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/test/org/apache/solr/search/function/distance/DistanceFunctionTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/function/distance/DistanceFunctionTest.java b/solr/core/src/test/org/apache/solr/search/function/distance/DistanceFunctionTest.java
index f33bc2a..b3ad183 100644
--- a/solr/core/src/test/org/apache/solr/search/function/distance/DistanceFunctionTest.java
+++ b/solr/core/src/test/org/apache/solr/search/function/distance/DistanceFunctionTest.java
@@ -15,8 +15,8 @@
  * limitations under the License.
  */
 package org.apache.solr.search.function.distance;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.io.GeohashUtils;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.io.GeohashUtils;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/core/src/test/org/apache/solr/util/DistanceUnitsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/util/DistanceUnitsTest.java b/solr/core/src/test/org/apache/solr/util/DistanceUnitsTest.java
index f16292c..3956371 100644
--- a/solr/core/src/test/org/apache/solr/util/DistanceUnitsTest.java
+++ b/solr/core/src/test/org/apache/solr/util/DistanceUnitsTest.java
@@ -16,7 +16,7 @@
  */
 package org.apache.solr.util;
 
-import com.spatial4j.core.distance.DistanceUtils;
+import org.locationtech.spatial4j.distance.DistanceUtils;
 import org.apache.lucene.util.LuceneTestCase;
 
 public class DistanceUnitsTest extends LuceneTestCase {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/licenses/spatial4j-0.5.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/spatial4j-0.5.jar.sha1 b/solr/licenses/spatial4j-0.5.jar.sha1
deleted file mode 100644
index c81a76c..0000000
--- a/solr/licenses/spatial4j-0.5.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-6e16edaf6b1ba76db7f08c2f3723fce3b358ecc3

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/licenses/spatial4j-0.6.jar.sha1
----------------------------------------------------------------------
diff --git a/solr/licenses/spatial4j-0.6.jar.sha1 b/solr/licenses/spatial4j-0.6.jar.sha1
new file mode 100644
index 0000000..56c02ad
--- /dev/null
+++ b/solr/licenses/spatial4j-0.6.jar.sha1
@@ -0,0 +1 @@
+21b15310bddcfd8c72611c180f20cf23279809a3

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/solr/licenses/spatial4j-NOTICE.txt
----------------------------------------------------------------------
diff --git a/solr/licenses/spatial4j-NOTICE.txt b/solr/licenses/spatial4j-NOTICE.txt
index d7d48d1..779b8df 100644
--- a/solr/licenses/spatial4j-NOTICE.txt
+++ b/solr/licenses/spatial4j-NOTICE.txt
@@ -1,5 +1,133 @@
-Spatial4j
-Copyright 2012-2014 The Apache Software Foundation
+# about.md file
 
-This product includes software developed by
-The Apache Software Foundation (http://www.apache.org/).
+## About This Content
+
+May 22, 2015
+
+### License
+
+The Eclipse Foundation makes available all content in this plug-in ("Content"). Unless otherwise indicated below, the
+Content is provided to you under the terms and conditions of the Apache License, Version 2.0.  A copy of the Apache
+License, Version 2.0 is available at
+[http://www.apache.org/licenses/LICENSE-2.0.txt](http://www.apache.org/licenses/LICENSE-2.0.txt)
+
+If you did not receive this Content directly from the Eclipse Foundation, the Content is being redistributed by another
+party ("Redistributor") and different terms and conditions may apply to your use of any object code in the Content.
+Check the Redistributor’s license that was provided with the Content. If no such license exists, contact the
+Redistributor. Unless otherwise indicated below, the terms and conditions of the Apache License, Version 2.0 still apply
+to any source code in the Content and such source code may be obtained at
+[http://www.eclipse.org](http://www.eclipse.org).
+
+# notice.md file
+
+Note: the below Eclipse user agreement is standard.  It says "Unless otherwise indicated, "... before referring to the
+EPL.  We indicate above that all content is licensed under the ASLv2 license.  -- David Smiley
+
+## Eclipse Foundation Software User Agreement
+
+April 9, 2014
+
+### Usage Of Content
+
+THE ECLIPSE FOUNDATION MAKES AVAILABLE SOFTWARE, DOCUMENTATION, INFORMATION AND/OR OTHER MATERIALS FOR OPEN SOURCE
+PROJECTS (COLLECTIVELY "CONTENT"). USE OF THE CONTENT IS GOVERNED BY THE TERMS AND CONDITIONS OF THIS AGREEMENT AND/OR
+THE TERMS AND CONDITIONS OF LICENSE AGREEMENTS OR NOTICES INDICATED OR REFERENCED BELOW. BY USING THE CONTENT, YOU AGREE
+THAT YOUR USE OF THE CONTENT IS GOVERNED BY THIS AGREEMENT AND/OR THE TERMS AND CONDITIONS OF ANY APPLICABLE LICENSE
+AGREEMENTS OR NOTICES INDICATED OR REFERENCED BELOW. IF YOU DO NOT AGREE TO THE TERMS AND CONDITIONS OF THIS AGREEMENT
+AND THE TERMS AND CONDITIONS OF ANY APPLICABLE LICENSE AGREEMENTS OR NOTICES INDICATED OR REFERENCED BELOW, THEN YOU MAY
+NOT USE THE CONTENT.
+
+### Applicable Licenses
+
+Unless otherwise indicated, all Content made available by the Eclipse Foundation is provided to you under the terms and
+conditions of the Eclipse Public License Version 1.0 ("EPL"). A copy of the EPL is provided with this Content and is
+also available at [http://www.eclipse.org/legal/epl-v10.html](http://www.eclipse.org/legal/epl-v10.html). For purposes
+of the EPL, "Program" will mean the Content.
+
+Content includes, but is not limited to, source code, object code, documentation and other files maintained in the
+Eclipse Foundation source code repository ("Repository") in software modules ("Modules") and made available as
+downloadable archives ("Downloads").
+
+*   Content may be structured and packaged into modules to facilitate delivering, extending, and upgrading the Content.
+    Typical modules may include plug-ins ("Plug-ins"), plug-in fragments ("Fragments"), and features ("Features").
+*   Each Plug-in or Fragment may be packaged as a sub-directory or JAR (Javaâ„¢ ARchive) in a directory named "plugins".
+*   A Feature is a bundle of one or more Plug-ins and/or Fragments and associated material. Each Feature may be packaged
+    as a sub-directory in a directory named "features". Within a Feature, files named "feature.xml" may contain a list
+    of the names and version numbers of the Plug-ins and/or Fragments associated with that Feature.
+*   Features may also include other Features ("Included Features"). Within a Feature, files named "feature.xml" may
+    contain a list of the names and version numbers of Included Features.
+
+The terms and conditions governing Plug-ins and Fragments should be contained in files named "about.html" ("Abouts").
+The terms and conditions governing Features and Included Features should be contained in files named "license.html"
+("Feature Licenses"). Abouts and Feature Licenses may be located in any directory of a Download or Module including, but
+not limited to the following locations:
+
+*   The top-level (root) directory
+*   Plug-in and Fragment directories
+*   Inside Plug-ins and Fragments packaged as JARs
+*   Sub-directories of the directory named "src" of certain Plug-ins
+*   Feature directories
+
+Note: if a Feature made available by the Eclipse Foundation is installed using the Provisioning Technology (as defined
+below), you must agree to a license ("Feature Update License") during the installation process. If the Feature contains
+Included Features, the Feature Update License should either provide you with the terms and conditions governing the
+Included Features or inform you where you can locate them. Feature Update Licenses may be found in the "license"
+property of files named "feature.properties" found within a Feature. Such Abouts, Feature Licenses, and Feature Update
+Licenses contain the terms and conditions (or references to such terms and conditions) that govern your use of the
+associated Content in that directory.
+
+THE ABOUTS, FEATURE LICENSES, AND FEATURE UPDATE LICENSES MAY REFER TO THE EPL OR OTHER LICENSE AGREEMENTS, NOTICES OR
+TERMS AND CONDITIONS. SOME OF THESE OTHER LICENSE AGREEMENTS MAY INCLUDE (BUT ARE NOT LIMITED TO):
+
+*   Eclipse Distribution License Version 1.0 (available at
+    [http://www.eclipse.org/licenses/edl-v1.0.html](http://www.eclipse.org/licenses/edl-v10.html))
+*   Common Public License Version 1.0 (available at
+    [http://www.eclipse.org/legal/cpl-v10.html](http://www.eclipse.org/legal/cpl-v10.html))
+*   Apache Software License 1.1 (available at
+    [http://www.apache.org/licenses/LICENSE](http://www.apache.org/licenses/LICENSE))
+*   Apache Software License 2.0 (available at
+    [http://www.apache.org/licenses/LICENSE-2.0](http://www.apache.org/licenses/LICENSE-2.0))
+*   Mozilla Public License Version 1.1 (available at
+    [http://www.mozilla.org/MPL/MPL-1.1.html](http://www.mozilla.org/MPL/MPL-1.1.html))
+
+IT IS YOUR OBLIGATION TO READ AND ACCEPT ALL SUCH TERMS AND CONDITIONS PRIOR TO USE OF THE CONTENT. If no About, Feature
+License, or Feature Update License is provided, please contact the Eclipse Foundation to determine what terms and
+conditions govern that particular Content.
+
+### Use of Provisioning Technology
+
+The Eclipse Foundation makes available provisioning software, examples of which include, but are not limited to, p2 and
+the Eclipse Update Manager ("Provisioning Technology") for the purpose of allowing users to install software,
+documentation, information and/or other materials (collectively "Installable Software"). This capability is provided
+with the intent of allowing such users to install, extend and update Eclipse-based products. Information about packaging
+Installable Software is available at
+[http://eclipse.org/equinox/p2/repository_packaging.html](http://eclipse.org/equinox/p2/repository_packaging.html)
+("Specification").
+
+You may use Provisioning Technology to allow other parties to install Installable Software. You shall be responsible for
+enabling the applicable license agreements relating to the Installable Software to be presented to, and accepted by, the
+users of the Provisioning Technology in accordance with the Specification. By using Provisioning Technology in such a
+manner and making it available in accordance with the Specification, you further acknowledge your agreement to, and the
+acquisition of all necessary rights to permit the following:
+
+1.  A series of actions may occur ("Provisioning Process") in which a user may execute the Provisioning Technology on a
+    machine ("Target Machine") with the intent of installing, extending or updating the functionality of an
+    Eclipse-based product.
+2.  During the Provisioning Process, the Provisioning Technology may cause third party Installable Software or a portion
+    thereof to be accessed and copied to the Target Machine.
+3.  Pursuant to the Specification, you will provide to the user the terms and conditions that govern the use of the
+    Installable Software ("Installable Software Agreement") and such Installable Software Agreement shall be accessed
+    from the Target Machine in accordance with the Specification. Such Installable Software Agreement must inform the
+    user of the terms and conditions that govern the Installable Software and must solicit acceptance by the end user in
+    the manner prescribed in such Installable Software Agreement. Upon such indication of agreement by the user, the
+    provisioning Technology will complete installation of the Installable Software.
+
+### Cryptography
+
+Content may contain encryption software. The country in which you are currently may have restrictions on the import,
+possession, and use, and/or re-export to another country, of encryption software. BEFORE using any encryption software,
+please check the country's laws, regulations and policies concerning the import, possession, or use, and re-export of
+encryption software, to see if this is permitted.
+
+<small>Java and all Java-based trademarks are trademarks of Oracle Corporation in the United States, other countries,
+or both.</small>
\ No newline at end of file


[8/9] lucene-solr git commit: LUCENE-7060: Spatial4j 0.6 upgrade. Package com.spatial4j.core -> org.locationtech.spatial4j (cherry picked from commit 569b6ca) (cherry picked from commit b90dbd4)

Posted by sh...@apache.org.
LUCENE-7060: Spatial4j 0.6 upgrade. Package com.spatial4j.core -> org.locationtech.spatial4j
(cherry picked from commit 569b6ca)
(cherry picked from commit b90dbd4)


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

Branch: refs/heads/branch_6_0
Commit: 891e659012c045f9202e2df65c15c1f43b692772
Parents: dc35cea
Author: David Smiley <ds...@apache.org>
Authored: Thu Mar 3 07:35:32 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Mar 3 20:49:58 2016 +0530

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 lucene/benchmark/conf/spatial.alg               |   2 +-
 lucene/benchmark/ivy.xml                        |   2 +-
 .../benchmark/byTask/feeds/SpatialDocMaker.java |   8 +-
 .../byTask/feeds/SpatialFileQueryMaker.java     |   4 +-
 lucene/ivy-versions.properties                  |   3 +-
 lucene/licenses/spatial4j-0.5-tests.jar.sha1    |   1 -
 lucene/licenses/spatial4j-0.5.jar.sha1          |   1 -
 lucene/licenses/spatial4j-0.6-tests.jar.sha1    |   1 +
 lucene/licenses/spatial4j-0.6.jar.sha1          |   1 +
 lucene/licenses/spatial4j-NOTICE.txt            | 136 ++++++++++++++++++-
 lucene/spatial-extras/ivy.xml                   |   4 +-
 .../apache/lucene/spatial/SpatialStrategy.java  |  12 +-
 .../bbox/BBoxOverlapRatioValueSource.java       |   2 +-
 .../spatial/bbox/BBoxSimilarityValueSource.java |   4 +-
 .../lucene/spatial/bbox/BBoxStrategy.java       |  10 +-
 .../lucene/spatial/bbox/BBoxValueSource.java    |   2 +-
 .../composite/CompositeSpatialStrategy.java     |   4 +-
 .../composite/IntersectsRPTVerifyQuery.java     |   4 +-
 .../spatial/prefix/AbstractPrefixTreeQuery.java |   2 +-
 .../prefix/AbstractVisitingPrefixTreeQuery.java |   4 +-
 .../spatial/prefix/ContainsPrefixTreeQuery.java |   4 +-
 .../spatial/prefix/HeatmapFacetCounter.java     |  10 +-
 .../prefix/IntersectsPrefixTreeQuery.java       |   4 +-
 .../prefix/NumberRangePrefixTreeStrategy.java   |   4 +-
 .../PointPrefixTreeFieldCacheProvider.java      |   2 +-
 .../spatial/prefix/PrefixTreeFacetCounter.java  |   6 +-
 .../spatial/prefix/PrefixTreeStrategy.java      |  10 +-
 .../prefix/RecursivePrefixTreeStrategy.java     |   4 +-
 .../prefix/TermQueryPrefixTreeStrategy.java     |   6 +-
 .../spatial/prefix/WithinPrefixTreeQuery.java   |  16 +--
 .../apache/lucene/spatial/prefix/tree/Cell.java |   6 +-
 .../prefix/tree/DateRangePrefixTree.java        |   2 +-
 .../spatial/prefix/tree/FilterCellIterator.java |   4 +-
 .../spatial/prefix/tree/GeohashPrefixTree.java  |  10 +-
 .../lucene/spatial/prefix/tree/LegacyCell.java  |   6 +-
 .../spatial/prefix/tree/LegacyPrefixTree.java   |   8 +-
 .../prefix/tree/NumberRangePrefixTree.java      |  16 +--
 .../prefix/tree/PackedQuadPrefixTree.java       |  12 +-
 .../spatial/prefix/tree/QuadPrefixTree.java     |  10 +-
 .../spatial/prefix/tree/SpatialPrefixTree.java  |   4 +-
 .../prefix/tree/SpatialPrefixTreeFactory.java   |   4 +-
 .../spatial/prefix/tree/TreeCellIterator.java   |   4 +-
 .../lucene/spatial/query/SpatialArgs.java       |  12 +-
 .../lucene/spatial/query/SpatialArgsParser.java |  10 +-
 .../lucene/spatial/query/SpatialOperation.java  |   8 +-
 .../serialized/SerializedDVStrategy.java        |  10 +-
 .../lucene/spatial/spatial4j/Geo3dShape.java    |  14 +-
 .../util/DistanceToShapeValueSource.java        |  10 +-
 .../spatial/util/ShapeAreaValueSource.java      |   6 +-
 .../lucene/spatial/util/ShapeFieldCache.java    |   2 +-
 .../ShapeFieldCacheDistanceValueSource.java     |   6 +-
 .../spatial/util/ShapeFieldCacheProvider.java   |   2 +-
 .../spatial/util/ShapePredicateValueSource.java |   2 +-
 .../spatial/vector/DistanceValueSource.java     |   4 +-
 .../spatial/vector/PointVectorStrategy.java     |  16 +--
 .../lucene/spatial/DistanceStrategyTest.java    |   6 +-
 .../apache/lucene/spatial/PortedSolr3Test.java  |   8 +-
 .../lucene/spatial/QueryEqualsHashCodeTest.java |   4 +-
 .../apache/lucene/spatial/SpatialArgsTest.java  |   4 +-
 .../apache/lucene/spatial/SpatialExample.java   |   8 +-
 .../apache/lucene/spatial/SpatialTestCase.java  |   8 +-
 .../apache/lucene/spatial/SpatialTestData.java  |   4 +-
 .../apache/lucene/spatial/SpatialTestQuery.java |   2 +-
 .../apache/lucene/spatial/StrategyTestCase.java |   4 +-
 .../lucene/spatial/TestTestFramework.java       |   4 +-
 .../lucene/spatial/bbox/TestBBoxStrategy.java   |  12 +-
 .../composite/CompositeStrategyTest.java        |  12 +-
 .../spatial/prefix/DateNRStrategyTest.java      |   2 +-
 .../spatial/prefix/HeatmapFacetCounterTest.java |  18 +--
 .../lucene/spatial/prefix/JtsPolygonTest.java   |   8 +-
 .../spatial/prefix/NumberRangeFacetsTest.java   |   2 +-
 .../RandomSpatialOpFuzzyPrefixTreeTest.java     |  24 ++--
 .../prefix/RandomSpatialOpStrategyTestCase.java |   2 +-
 .../prefix/TestRecursivePrefixTreeStrategy.java |   8 +-
 .../prefix/TestTermQueryPrefixGridStrategy.java |   4 +-
 .../prefix/tree/DateRangePrefixTreeTest.java    |   4 +-
 .../prefix/tree/SpatialPrefixTreeTest.java      |   8 +-
 .../spatial/query/SpatialArgsParserTest.java    |   4 +-
 .../serialized/SerializedStrategyTest.java      |   2 +-
 .../lucene/spatial/spatial4j/Geo3dRptTest.java  |  10 +-
 .../Geo3dShapeRectRelationTestCase.java         |  18 +--
 .../Geo3dShapeSphereModelRectRelationTest.java  |   2 +-
 .../spatial4j/RandomizedShapeTestCase.java      |  22 +--
 .../spatial/spatial4j/geo3d/GeoPointTest.java   |   2 +-
 .../spatial/vector/TestPointVectorStrategy.java |   6 +-
 solr/core/ivy.xml                               |   2 +-
 .../handler/component/SpatialHeatmapFacets.java |   4 +-
 .../solr/schema/AbstractSpatialFieldType.java   |  12 +-
 .../AbstractSpatialPrefixTreeFieldType.java     |   2 +-
 .../java/org/apache/solr/schema/BBoxField.java  |   2 +-
 .../org/apache/solr/schema/DateRangeField.java  |   2 +-
 .../org/apache/solr/schema/GeoHashField.java    |   8 +-
 .../java/org/apache/solr/schema/LatLonType.java |   8 +-
 .../java/org/apache/solr/schema/PointType.java  |   2 +-
 .../schema/RptWithGeometrySpatialField.java     |   6 +-
 .../apache/solr/search/ValueSourceParser.java   |   2 +-
 .../distance/GeoDistValueSourceParser.java      |   8 +-
 .../function/distance/GeohashFunction.java      |   2 +-
 .../distance/GeohashHaversineFunction.java      |  10 +-
 .../distance/HaversineConstFunction.java        |   4 +-
 .../function/distance/HaversineFunction.java    |   2 +-
 .../org/apache/solr/util/DistanceUnits.java     |   2 +-
 .../java/org/apache/solr/util/SpatialUtils.java |  18 +--
 .../apache/solr/search/TestSolr4Spatial.java    |   8 +-
 .../function/distance/DistanceFunctionTest.java |   4 +-
 .../org/apache/solr/util/DistanceUnitsTest.java |   2 +-
 solr/licenses/spatial4j-0.5.jar.sha1            |   1 -
 solr/licenses/spatial4j-0.6.jar.sha1            |   1 +
 solr/licenses/spatial4j-NOTICE.txt              | 136 ++++++++++++++++++-
 110 files changed, 595 insertions(+), 339 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 324437a..37473d5 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -110,6 +110,9 @@ API Changes
   comparator in ArrayUtil by Java 8's Comparator#naturalOrder().
   (Mike McCandless, Uwe Schindler, Robert Muir)
 
+* LUCENE-7060: Update Spatial4j to 0.6.  The package com.spatial4j.core
+  is now org.locationtech.spatial4j. (David Smiley)
+
 * LUCENE-7058: Add getters to various Query implementations (Guillaume Smet via
   Alan Woodward)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/benchmark/conf/spatial.alg
----------------------------------------------------------------------
diff --git a/lucene/benchmark/conf/spatial.alg b/lucene/benchmark/conf/spatial.alg
index 93f10ef..0ee637f 100644
--- a/lucene/benchmark/conf/spatial.alg
+++ b/lucene/benchmark/conf/spatial.alg
@@ -23,7 +23,7 @@
 ### Spatial Context, Grid, Strategy config
 doc.maker=org.apache.lucene.benchmark.byTask.feeds.SpatialDocMaker
 #  SpatialContext:    see SpatialContextFactory.makeSpatialContext
-#spatial.spatialContextFactory=com.spatial4j.core.context.jts.JtsSpatialContextFactory
+#spatial.spatialContextFactory=org.locationtech.spatial4j.context.jts.JtsSpatialContextFactory
 #spatial.geo=true
 #spatial.distCalculator=haversine
 #spatial.worldBounds=...

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/benchmark/ivy.xml
----------------------------------------------------------------------
diff --git a/lucene/benchmark/ivy.xml b/lucene/benchmark/ivy.xml
index ec02b6a..23c208c 100644
--- a/lucene/benchmark/ivy.xml
+++ b/lucene/benchmark/ivy.xml
@@ -26,7 +26,7 @@
     <dependency org="xerces" name="xercesImpl" rev="${/xerces/xercesImpl}" conf="compile"/>
     <dependency org="net.sourceforge.nekohtml" name="nekohtml" rev="${/net.sourceforge.nekohtml/nekohtml}" conf="compile"/>
     <dependency org="com.ibm.icu" name="icu4j" rev="${/com.ibm.icu/icu4j}" conf="compile"/>
-    <dependency org="com.spatial4j" name="spatial4j" rev="${/com.spatial4j/spatial4j}" conf="compile"/>
+    <dependency org="org.locationtech.spatial4j" name="spatial4j" rev="${/org.locationtech.spatial4j/spatial4j}" conf="compile"/>
     <exclude org="*" ext="*" matcher="regexp" type="${ivy.exclude.types}"/> 
   </dependencies>
 </ivy-module>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialDocMaker.java
----------------------------------------------------------------------
diff --git a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialDocMaker.java b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialDocMaker.java
index b7f04d1..b466e96 100644
--- a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialDocMaker.java
+++ b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialDocMaker.java
@@ -23,10 +23,10 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.context.SpatialContextFactory;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.context.SpatialContextFactory;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.benchmark.byTask.utils.Config;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialFileQueryMaker.java
----------------------------------------------------------------------
diff --git a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialFileQueryMaker.java b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialFileQueryMaker.java
index c99de3d..b6b8f50 100644
--- a/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialFileQueryMaker.java
+++ b/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/SpatialFileQueryMaker.java
@@ -21,7 +21,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.benchmark.byTask.utils.Config;
 import org.apache.lucene.queries.function.FunctionQuery;
 import org.apache.lucene.queries.function.ValueSource;
@@ -34,7 +34,7 @@ import org.apache.lucene.spatial.query.SpatialOperation;
 
 /**
  * Reads spatial data from the body field docs from an internally created {@link LineDocSource}.
- * It's parsed by {@link com.spatial4j.core.context.SpatialContext#readShapeFromWkt(String)} (String)} and then
+ * It's parsed by {@link org.locationtech.spatial4j.context.SpatialContext#readShapeFromWkt(String)} (String)} and then
  * further manipulated via a configurable {@link SpatialDocMaker.ShapeConverter}. When using point
  * data, it's likely you'll want to configure the shape converter so that the query shapes actually
  * cover a region. The queries are all created and cached in advance. This query maker works in

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/ivy-versions.properties
----------------------------------------------------------------------
diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index 2d630a2..d5ef256 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -41,7 +41,6 @@ com.google.inject.guice.version = 3.0
 /com.googlecode.mp4parser/isoparser = 1.0.2
 /com.ibm.icu/icu4j = 56.1
 /com.pff/java-libpst = 0.8.1
-/com.spatial4j/spatial4j = 0.5
 
 com.sun.jersey.version = 1.9
 /com.sun.jersey.contribs/jersey-guice = ${com.sun.jersey.version}
@@ -268,6 +267,8 @@ org.kitesdk.kite-morphlines.version = 1.1.0
 /org.kitesdk/kite-morphlines-tika-decompress = ${org.kitesdk.kite-morphlines.version}
 /org.kitesdk/kite-morphlines-twitter = ${org.kitesdk.kite-morphlines.version}
 
+/org.locationtech.spatial4j/spatial4j = 0.6
+
 /org.mockito/mockito-core = 1.9.5
 
 org.mortbay.jetty.version = 6.1.26

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/licenses/spatial4j-0.5-tests.jar.sha1
----------------------------------------------------------------------
diff --git a/lucene/licenses/spatial4j-0.5-tests.jar.sha1 b/lucene/licenses/spatial4j-0.5-tests.jar.sha1
deleted file mode 100644
index 0c514f8..0000000
--- a/lucene/licenses/spatial4j-0.5-tests.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-bdcdf20a723516a233b5bcc0ca7d4decaa88b6ed

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/licenses/spatial4j-0.5.jar.sha1
----------------------------------------------------------------------
diff --git a/lucene/licenses/spatial4j-0.5.jar.sha1 b/lucene/licenses/spatial4j-0.5.jar.sha1
deleted file mode 100644
index c81a76c..0000000
--- a/lucene/licenses/spatial4j-0.5.jar.sha1
+++ /dev/null
@@ -1 +0,0 @@
-6e16edaf6b1ba76db7f08c2f3723fce3b358ecc3

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/licenses/spatial4j-0.6-tests.jar.sha1
----------------------------------------------------------------------
diff --git a/lucene/licenses/spatial4j-0.6-tests.jar.sha1 b/lucene/licenses/spatial4j-0.6-tests.jar.sha1
new file mode 100644
index 0000000..0fd8404
--- /dev/null
+++ b/lucene/licenses/spatial4j-0.6-tests.jar.sha1
@@ -0,0 +1 @@
+0624ae8b9e43265822e0d79b481e34917fec1eba

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/licenses/spatial4j-0.6.jar.sha1
----------------------------------------------------------------------
diff --git a/lucene/licenses/spatial4j-0.6.jar.sha1 b/lucene/licenses/spatial4j-0.6.jar.sha1
new file mode 100644
index 0000000..56c02ad
--- /dev/null
+++ b/lucene/licenses/spatial4j-0.6.jar.sha1
@@ -0,0 +1 @@
+21b15310bddcfd8c72611c180f20cf23279809a3

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/licenses/spatial4j-NOTICE.txt
----------------------------------------------------------------------
diff --git a/lucene/licenses/spatial4j-NOTICE.txt b/lucene/licenses/spatial4j-NOTICE.txt
index d7d48d1..779b8df 100644
--- a/lucene/licenses/spatial4j-NOTICE.txt
+++ b/lucene/licenses/spatial4j-NOTICE.txt
@@ -1,5 +1,133 @@
-Spatial4j
-Copyright 2012-2014 The Apache Software Foundation
+# about.md file
 
-This product includes software developed by
-The Apache Software Foundation (http://www.apache.org/).
+## About This Content
+
+May 22, 2015
+
+### License
+
+The Eclipse Foundation makes available all content in this plug-in ("Content"). Unless otherwise indicated below, the
+Content is provided to you under the terms and conditions of the Apache License, Version 2.0.  A copy of the Apache
+License, Version 2.0 is available at
+[http://www.apache.org/licenses/LICENSE-2.0.txt](http://www.apache.org/licenses/LICENSE-2.0.txt)
+
+If you did not receive this Content directly from the Eclipse Foundation, the Content is being redistributed by another
+party ("Redistributor") and different terms and conditions may apply to your use of any object code in the Content.
+Check the Redistributor’s license that was provided with the Content. If no such license exists, contact the
+Redistributor. Unless otherwise indicated below, the terms and conditions of the Apache License, Version 2.0 still apply
+to any source code in the Content and such source code may be obtained at
+[http://www.eclipse.org](http://www.eclipse.org).
+
+# notice.md file
+
+Note: the below Eclipse user agreement is standard.  It says "Unless otherwise indicated, "... before referring to the
+EPL.  We indicate above that all content is licensed under the ASLv2 license.  -- David Smiley
+
+## Eclipse Foundation Software User Agreement
+
+April 9, 2014
+
+### Usage Of Content
+
+THE ECLIPSE FOUNDATION MAKES AVAILABLE SOFTWARE, DOCUMENTATION, INFORMATION AND/OR OTHER MATERIALS FOR OPEN SOURCE
+PROJECTS (COLLECTIVELY "CONTENT"). USE OF THE CONTENT IS GOVERNED BY THE TERMS AND CONDITIONS OF THIS AGREEMENT AND/OR
+THE TERMS AND CONDITIONS OF LICENSE AGREEMENTS OR NOTICES INDICATED OR REFERENCED BELOW. BY USING THE CONTENT, YOU AGREE
+THAT YOUR USE OF THE CONTENT IS GOVERNED BY THIS AGREEMENT AND/OR THE TERMS AND CONDITIONS OF ANY APPLICABLE LICENSE
+AGREEMENTS OR NOTICES INDICATED OR REFERENCED BELOW. IF YOU DO NOT AGREE TO THE TERMS AND CONDITIONS OF THIS AGREEMENT
+AND THE TERMS AND CONDITIONS OF ANY APPLICABLE LICENSE AGREEMENTS OR NOTICES INDICATED OR REFERENCED BELOW, THEN YOU MAY
+NOT USE THE CONTENT.
+
+### Applicable Licenses
+
+Unless otherwise indicated, all Content made available by the Eclipse Foundation is provided to you under the terms and
+conditions of the Eclipse Public License Version 1.0 ("EPL"). A copy of the EPL is provided with this Content and is
+also available at [http://www.eclipse.org/legal/epl-v10.html](http://www.eclipse.org/legal/epl-v10.html). For purposes
+of the EPL, "Program" will mean the Content.
+
+Content includes, but is not limited to, source code, object code, documentation and other files maintained in the
+Eclipse Foundation source code repository ("Repository") in software modules ("Modules") and made available as
+downloadable archives ("Downloads").
+
+*   Content may be structured and packaged into modules to facilitate delivering, extending, and upgrading the Content.
+    Typical modules may include plug-ins ("Plug-ins"), plug-in fragments ("Fragments"), and features ("Features").
+*   Each Plug-in or Fragment may be packaged as a sub-directory or JAR (Javaâ„¢ ARchive) in a directory named "plugins".
+*   A Feature is a bundle of one or more Plug-ins and/or Fragments and associated material. Each Feature may be packaged
+    as a sub-directory in a directory named "features". Within a Feature, files named "feature.xml" may contain a list
+    of the names and version numbers of the Plug-ins and/or Fragments associated with that Feature.
+*   Features may also include other Features ("Included Features"). Within a Feature, files named "feature.xml" may
+    contain a list of the names and version numbers of Included Features.
+
+The terms and conditions governing Plug-ins and Fragments should be contained in files named "about.html" ("Abouts").
+The terms and conditions governing Features and Included Features should be contained in files named "license.html"
+("Feature Licenses"). Abouts and Feature Licenses may be located in any directory of a Download or Module including, but
+not limited to the following locations:
+
+*   The top-level (root) directory
+*   Plug-in and Fragment directories
+*   Inside Plug-ins and Fragments packaged as JARs
+*   Sub-directories of the directory named "src" of certain Plug-ins
+*   Feature directories
+
+Note: if a Feature made available by the Eclipse Foundation is installed using the Provisioning Technology (as defined
+below), you must agree to a license ("Feature Update License") during the installation process. If the Feature contains
+Included Features, the Feature Update License should either provide you with the terms and conditions governing the
+Included Features or inform you where you can locate them. Feature Update Licenses may be found in the "license"
+property of files named "feature.properties" found within a Feature. Such Abouts, Feature Licenses, and Feature Update
+Licenses contain the terms and conditions (or references to such terms and conditions) that govern your use of the
+associated Content in that directory.
+
+THE ABOUTS, FEATURE LICENSES, AND FEATURE UPDATE LICENSES MAY REFER TO THE EPL OR OTHER LICENSE AGREEMENTS, NOTICES OR
+TERMS AND CONDITIONS. SOME OF THESE OTHER LICENSE AGREEMENTS MAY INCLUDE (BUT ARE NOT LIMITED TO):
+
+*   Eclipse Distribution License Version 1.0 (available at
+    [http://www.eclipse.org/licenses/edl-v1.0.html](http://www.eclipse.org/licenses/edl-v10.html))
+*   Common Public License Version 1.0 (available at
+    [http://www.eclipse.org/legal/cpl-v10.html](http://www.eclipse.org/legal/cpl-v10.html))
+*   Apache Software License 1.1 (available at
+    [http://www.apache.org/licenses/LICENSE](http://www.apache.org/licenses/LICENSE))
+*   Apache Software License 2.0 (available at
+    [http://www.apache.org/licenses/LICENSE-2.0](http://www.apache.org/licenses/LICENSE-2.0))
+*   Mozilla Public License Version 1.1 (available at
+    [http://www.mozilla.org/MPL/MPL-1.1.html](http://www.mozilla.org/MPL/MPL-1.1.html))
+
+IT IS YOUR OBLIGATION TO READ AND ACCEPT ALL SUCH TERMS AND CONDITIONS PRIOR TO USE OF THE CONTENT. If no About, Feature
+License, or Feature Update License is provided, please contact the Eclipse Foundation to determine what terms and
+conditions govern that particular Content.
+
+### Use of Provisioning Technology
+
+The Eclipse Foundation makes available provisioning software, examples of which include, but are not limited to, p2 and
+the Eclipse Update Manager ("Provisioning Technology") for the purpose of allowing users to install software,
+documentation, information and/or other materials (collectively "Installable Software"). This capability is provided
+with the intent of allowing such users to install, extend and update Eclipse-based products. Information about packaging
+Installable Software is available at
+[http://eclipse.org/equinox/p2/repository_packaging.html](http://eclipse.org/equinox/p2/repository_packaging.html)
+("Specification").
+
+You may use Provisioning Technology to allow other parties to install Installable Software. You shall be responsible for
+enabling the applicable license agreements relating to the Installable Software to be presented to, and accepted by, the
+users of the Provisioning Technology in accordance with the Specification. By using Provisioning Technology in such a
+manner and making it available in accordance with the Specification, you further acknowledge your agreement to, and the
+acquisition of all necessary rights to permit the following:
+
+1.  A series of actions may occur ("Provisioning Process") in which a user may execute the Provisioning Technology on a
+    machine ("Target Machine") with the intent of installing, extending or updating the functionality of an
+    Eclipse-based product.
+2.  During the Provisioning Process, the Provisioning Technology may cause third party Installable Software or a portion
+    thereof to be accessed and copied to the Target Machine.
+3.  Pursuant to the Specification, you will provide to the user the terms and conditions that govern the use of the
+    Installable Software ("Installable Software Agreement") and such Installable Software Agreement shall be accessed
+    from the Target Machine in accordance with the Specification. Such Installable Software Agreement must inform the
+    user of the terms and conditions that govern the Installable Software and must solicit acceptance by the end user in
+    the manner prescribed in such Installable Software Agreement. Upon such indication of agreement by the user, the
+    provisioning Technology will complete installation of the Installable Software.
+
+### Cryptography
+
+Content may contain encryption software. The country in which you are currently may have restrictions on the import,
+possession, and use, and/or re-export to another country, of encryption software. BEFORE using any encryption software,
+please check the country's laws, regulations and policies concerning the import, possession, or use, and re-export of
+encryption software, to see if this is permitted.
+
+<small>Java and all Java-based trademarks are trademarks of Oracle Corporation in the United States, other countries,
+or both.</small>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/ivy.xml
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/ivy.xml b/lucene/spatial-extras/ivy.xml
index 4fef30e..448776f 100644
--- a/lucene/spatial-extras/ivy.xml
+++ b/lucene/spatial-extras/ivy.xml
@@ -23,9 +23,9 @@
     <conf name="test" transitive="false"/>
   </configurations>
   <dependencies>
-    <dependency org="com.spatial4j" name="spatial4j" rev="${/com.spatial4j/spatial4j}" conf="compile"/>
+    <dependency org="org.locationtech.spatial4j" name="spatial4j" rev="${/org.locationtech.spatial4j/spatial4j}" conf="compile"/>
 
-    <dependency org="com.spatial4j" name="spatial4j" rev="${/com.spatial4j/spatial4j}" conf="test">
+    <dependency org="org.locationtech.spatial4j" name="spatial4j" rev="${/org.locationtech.spatial4j/spatial4j}" conf="test">
       <artifact name="spatial4j" type="test" ext="jar" maven:classifier="tests" />
     </dependency>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/SpatialStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/SpatialStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/SpatialStrategy.java
index f433c11..d980ba9 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/SpatialStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/SpatialStrategy.java
@@ -16,10 +16,10 @@
  */
 package org.apache.lucene.spatial;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.ReciprocalFloatFunction;
@@ -100,7 +100,7 @@ public abstract class SpatialStrategy {
   public abstract Field[] createIndexableFields(Shape shape);
 
   /**
-   * See {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point, double)} called with
+   * See {@link #makeDistanceValueSource(org.locationtech.spatial4j.shape.Point, double)} called with
    * a multiplier of 1.0 (i.e. units of degrees).
    */
   public ValueSource makeDistanceValueSource(Point queryPoint) {
@@ -127,7 +127,7 @@ public abstract class SpatialStrategy {
 
   /**
    * Returns a ValueSource with values ranging from 1 to 0, depending inversely
-   * on the distance from {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point,double)}.
+   * on the distance from {@link #makeDistanceValueSource(org.locationtech.spatial4j.shape.Point,double)}.
    * The formula is {@code c/(d + c)} where 'd' is the distance and 'c' is
    * one tenth the distance to the farthest edge from the center. Thus the
    * scores will be 1 for indexed points at the center of the query shape and as

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxOverlapRatioValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxOverlapRatioValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxOverlapRatioValueSource.java
index 9d0afe1..101f373 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxOverlapRatioValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxOverlapRatioValueSource.java
@@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.Explanation;
 
-import com.spatial4j.core.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Rectangle;
 
 /**
  * The algorithm is implemented as envelope on envelope (rect on rect) overlays rather than

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java
index 15cd646..1d8b4b0 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java
@@ -27,12 +27,12 @@ import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 
-import com.spatial4j.core.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Rectangle;
 
 /**
  * A base class for calculating a spatial relevance rank per document from a provided
  * {@link ValueSource} in which {@link FunctionValues#objectVal(int)} returns a {@link
- * com.spatial4j.core.shape.Rectangle}.
+ * org.locationtech.spatial4j.shape.Rectangle}.
  * <p>
  * Implementers: remember to implement equals and hashCode if you have
  * fields!

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
index 9565cbf..5cc6788 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
@@ -16,10 +16,10 @@
  */
 package org.apache.lucene.spatial.bbox;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.document.LegacyDoubleField;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
@@ -64,7 +64,7 @@ import org.apache.lucene.util.LegacyNumericUtils;
  * and a boolean to mark a dateline cross. Depending on the particular {@link
  * SpatialOperation}s, there are a variety of {@link org.apache.lucene.search.LegacyNumericRangeQuery}s to be
  * done.
- * The {@link #makeOverlapRatioValueSource(com.spatial4j.core.shape.Rectangle, double)}
+ * The {@link #makeOverlapRatioValueSource(org.locationtech.spatial4j.shape.Rectangle, double)}
  * works by calculating the query bbox overlap percentage against the indexed
  * shape overlap percentage. The indexed shape's coordinates are retrieved from
  * {@link org.apache.lucene.index.LeafReader#getNumericDocValues}.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxValueSource.java
index 5d95407..2bfbfd9 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxValueSource.java
@@ -16,7 +16,7 @@
  */
 package org.apache.lucene.spatial.bbox;
 
-import com.spatial4j.core.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Rectangle;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DocValues;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeSpatialStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeSpatialStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeSpatialStrategy.java
index 7dc2dfa..de5bb61 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeSpatialStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeSpatialStrategy.java
@@ -20,8 +20,8 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.Query;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
index a963b6e..f60bfee 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
@@ -19,8 +19,8 @@ package org.apache.lucene.spatial.composite;
 import java.io.IOException;
 import java.util.Map;
 
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java
index 127e689..bcf4867 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java
@@ -18,7 +18,7 @@ package org.apache.lucene.spatial.prefix;
 
 import java.io.IOException;
 
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.PostingsEnum;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeQuery.java
index 2237ca9..8ccee99 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeQuery.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeQuery.java
@@ -19,8 +19,8 @@ package org.apache.lucene.spatial.prefix;
 import java.io.IOException;
 import java.util.Iterator;
 
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.DocIdSet;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeQuery.java
index 0046378..b0864f6 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeQuery.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeQuery.java
@@ -19,8 +19,8 @@ package org.apache.lucene.spatial.prefix;
 import java.io.IOException;
 import java.util.Arrays;
 
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.TermsEnum;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/HeatmapFacetCounter.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/HeatmapFacetCounter.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/HeatmapFacetCounter.java
index c6700cd..adee2be 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/HeatmapFacetCounter.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/HeatmapFacetCounter.java
@@ -20,11 +20,11 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.spatial.prefix.tree.Cell;
 import org.apache.lucene.spatial.prefix.tree.CellIterator;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeQuery.java
index ccb0f89..89129a1 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeQuery.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeQuery.java
@@ -18,8 +18,8 @@ package org.apache.lucene.spatial.prefix;
 
 import java.io.IOException;
 
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.spatial.prefix.tree.Cell;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
index 8001c82..c727c0d 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/NumberRangePrefixTreeStrategy.java
@@ -23,8 +23,8 @@ import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.spatial.prefix.tree.Cell;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java
index 165c418..f44ca44 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PointPrefixTreeFieldCacheProvider.java
@@ -16,7 +16,7 @@
  */
 package org.apache.lucene.spatial.prefix;
 
-import com.spatial4j.core.shape.Point;
+import org.locationtech.spatial4j.shape.Point;
 import org.apache.lucene.spatial.prefix.tree.Cell;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
 import org.apache.lucene.spatial.util.ShapeFieldCacheProvider;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeFacetCounter.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeFacetCounter.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeFacetCounter.java
index 173c30e..b3b82db 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeFacetCounter.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeFacetCounter.java
@@ -18,7 +18,7 @@ package org.apache.lucene.spatial.prefix;
 
 import java.io.IOException;
 
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.PostingsEnum;
@@ -37,9 +37,9 @@ import org.apache.lucene.util.Bits;
  * of double-counting the document in the facet results.  Since each shape is independently turned into grid cells at
  * a resolution chosen by the shape's size, it's possible they will be indexed at different resolutions.  This means
  * the document could be present in BOTH the postings for a cell in both its prefix and leaf variants.  To avoid this,
- * use a single valued field with a {@link com.spatial4j.core.shape.ShapeCollection} (or WKT equivalent).  Or
+ * use a single valued field with a {@link org.locationtech.spatial4j.shape.ShapeCollection} (or WKT equivalent).  Or
  * calculate a suitable level/distErr to index both and call
- * {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy#createIndexableFields(com.spatial4j.core.shape.Shape, int)}
+ * {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy#createIndexableFields(org.locationtech.spatial4j.shape.Shape, int)}
  * with the same value for all shapes for a given document/field.
  *
  * @lucene.experimental

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
index 608879b..e9f43fd 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
@@ -21,8 +21,8 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.IndexOptions;
@@ -47,7 +47,7 @@ import org.apache.lucene.util.Bits;
  * <li>Can index any shape; however only {@link RecursivePrefixTreeStrategy}
  * can effectively search non-point shapes.</li>
  * <li>Can index a variable number of shapes per field value. This strategy
- * can do it via multiple calls to {@link #createIndexableFields(com.spatial4j.core.shape.Shape)}
+ * can do it via multiple calls to {@link #createIndexableFields(org.locationtech.spatial4j.shape.Shape)}
  * for a document or by giving it some sort of Shape aggregate (e.g. JTS
  * WKT MultiPoint).  The shape's boundary is approximated to a grid precision.
  * </li>
@@ -56,7 +56,7 @@ import org.apache.lucene.util.Bits;
  * <li>Only {@link org.apache.lucene.spatial.query.SpatialOperation#Intersects}
  * is supported.  If only points are indexed then this is effectively equivalent
  * to IsWithin.</li>
- * <li>The strategy supports {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point,double)}
+ * <li>The strategy supports {@link #makeDistanceValueSource(org.locationtech.spatial4j.shape.Point,double)}
  * even for multi-valued data, so long as the indexed data is all points; the
  * behavior is undefined otherwise.  However, <em>it will likely be removed in
  * the future</em> in lieu of using another strategy with a more scalable
@@ -93,7 +93,7 @@ public abstract class PrefixTreeStrategy extends SpatialStrategy {
   }
 
   /**
-   * A memory hint used by {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point)}
+   * A memory hint used by {@link #makeDistanceValueSource(org.locationtech.spatial4j.shape.Point)}
    * for how big the initial size of each Document's array should be. The
    * default is 2.  Set this to slightly more than the default expected number
    * of points per document.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
index 68b0449..d3d1626 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
@@ -20,8 +20,8 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.spatial.prefix.tree.Cell;
 import org.apache.lucene.spatial.prefix.tree.CellIterator;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
index a74786b..0273466 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
@@ -19,8 +19,8 @@ package org.apache.lucene.spatial.prefix;
 import java.util.ArrayList;
 import java.util.List;
 
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.queries.TermsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.spatial.prefix.tree.Cell;
@@ -35,7 +35,7 @@ import org.apache.lucene.util.BytesRefBuilder;
 /**
  * A basic implementation of {@link PrefixTreeStrategy} using a large
  * {@link TermsQuery} of all the cells from
- * {@link SpatialPrefixTree#getTreeCellIterator(com.spatial4j.core.shape.Shape, int)}.
+ * {@link SpatialPrefixTree#getTreeCellIterator(org.locationtech.spatial4j.shape.Shape, int)}.
  * It only supports the search of indexed Point shapes.
  * <p>
  * The precision of query shapes (distErrPct) is an important factor in using

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeQuery.java
index cf0d11b..f595f0e 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeQuery.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/WithinPrefixTreeQuery.java
@@ -18,13 +18,13 @@ package org.apache.lucene.spatial.prefix;
 
 import java.io.IOException;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Circle;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Circle;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.spatial.prefix.tree.Cell;
@@ -58,7 +58,7 @@ public class WithinPrefixTreeQuery extends AbstractVisitingPrefixTreeQuery {
   private final Shape bufferedQueryShape;//if null then the whole world
 
   /**
-   * See {@link AbstractVisitingPrefixTreeQuery#AbstractVisitingPrefixTreeQuery(com.spatial4j.core.shape.Shape, String, org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree, int, int)}.
+   * See {@link AbstractVisitingPrefixTreeQuery#AbstractVisitingPrefixTreeQuery(org.locationtech.spatial4j.shape.Shape, String, org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree, int, int)}.
    * {@code queryBuffer} is the (minimum) distance beyond the query shape edge
    * where non-matching documents are looked for so they can be excluded. If
    * -1 is used then the whole world is examined (a good default for correctness).

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java
index fe3846d..f4bc458 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/Cell.java
@@ -16,14 +16,14 @@
  */
 package org.apache.lucene.spatial.prefix.tree;
 
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
 import org.apache.lucene.util.BytesRef;
 
 /**
  * Represents a grid cell. Cell instances are generally very transient and may be re-used
  * internally.  To get an instance, you could start with {@link SpatialPrefixTree#getWorldCell()}.
- * And from there you could either traverse down the tree with {@link #getNextLevelCells(com.spatial4j.core.shape.Shape)},
+ * And from there you could either traverse down the tree with {@link #getNextLevelCells(org.locationtech.spatial4j.shape.Shape)},
  * or you could read an indexed term via {@link SpatialPrefixTree#readCell(org.apache.lucene.util.BytesRef,Cell)}.
  * When a cell is read from a term, it is comprised of just the base bytes plus optionally a leaf flag.
  *

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTree.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTree.java
index 13281f3..afdde71 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTree.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/DateRangePrefixTree.java
@@ -24,7 +24,7 @@ import java.util.GregorianCalendar;
 import java.util.Locale;
 import java.util.TimeZone;
 
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Shape;
 
 /**
  * A PrefixTree for date ranges in which the levels of the tree occur at natural periods of time (e.g. years,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/FilterCellIterator.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/FilterCellIterator.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/FilterCellIterator.java
index e4f50e0..ef17007 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/FilterCellIterator.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/FilterCellIterator.java
@@ -16,8 +16,8 @@
  */
 package org.apache.lucene.spatial.prefix.tree;
 
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
 
 import java.util.Iterator;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java
index fa4e987..237d26a 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java
@@ -20,11 +20,11 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.io.GeohashUtils;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.io.GeohashUtils;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.util.BytesRef;
 
 /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java
index 27c56a7..d978d3c 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java
@@ -18,9 +18,9 @@ package org.apache.lucene.spatial.prefix.tree;
 
 import java.util.Collection;
 
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.StringHelper;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java
index 672c2fe..1a3afcc 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java
@@ -18,10 +18,10 @@ package org.apache.lucene.spatial.prefix.tree;
 
 import java.util.Arrays;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.util.BytesRef;
 
 /** The base for the original two SPT's: Geohash and Quad. Don't subclass this for new SPTs.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/NumberRangePrefixTree.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/NumberRangePrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/NumberRangePrefixTree.java
index 40e80bc..72b689b 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/NumberRangePrefixTree.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/NumberRangePrefixTree.java
@@ -18,13 +18,13 @@ package org.apache.lucene.spatial.prefix.tree;
 
 import java.text.ParseException;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.context.SpatialContextFactory;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
-import com.spatial4j.core.shape.impl.RectangleImpl;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.context.SpatialContextFactory;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.impl.RectangleImpl;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.StringHelper;
 
@@ -47,7 +47,7 @@ import org.apache.lucene.util.StringHelper;
  * </ul>
  *
  * Unlike "normal" spatial components in this module, this special-purpose one only works with {@link Shape}s
- * created by the methods on this class, not from any {@link com.spatial4j.core.context.SpatialContext}.
+ * created by the methods on this class, not from any {@link org.locationtech.spatial4j.context.SpatialContext}.
  *
  * @see org.apache.lucene.spatial.prefix.NumberRangePrefixTreeStrategy
  * @see <a href="https://issues.apache.org/jira/browse/LUCENE-5648">LUCENE-5648</a>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java
index 6fe2bff..b86a6d1 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/PackedQuadPrefixTree.java
@@ -21,12 +21,12 @@ import java.util.Collection;
 import java.util.List;
 import java.util.NoSuchElementException;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
-import com.spatial4j.core.shape.impl.RectangleImpl;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.impl.RectangleImpl;
 import org.apache.lucene.util.BytesRef;
 
 /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
index 48dac87..3242e7e 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/QuadPrefixTree.java
@@ -24,11 +24,11 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Locale;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
 import org.apache.lucene.util.BytesRef;
 
 /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java
index 8ead954..ae2fe83 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java
@@ -16,8 +16,8 @@
  */
 package org.apache.lucene.spatial.prefix.tree;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.util.BytesRef;
 
 /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java
index b74dc93..67480d5 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java
@@ -18,8 +18,8 @@ package org.apache.lucene.spatial.prefix.tree;
 
 import java.util.Map;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
 
 /**
  * Abstract Factory for creating {@link SpatialPrefixTree} instances with useful

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/TreeCellIterator.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/TreeCellIterator.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/TreeCellIterator.java
index 3ec56ac..39c8068 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/TreeCellIterator.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/TreeCellIterator.java
@@ -16,8 +16,8 @@
  */
 package org.apache.lucene.spatial.prefix.tree;
 
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 
 /**
  * Navigates a {@link org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree} from a given cell (typically the world

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgs.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgs.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgs.java
index 0503072..37a5503 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgs.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgs.java
@@ -16,10 +16,10 @@
  */
 package org.apache.lucene.spatial.query;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
 
 /**
  * Principally holds the query {@link Shape} and the {@link SpatialOperation}.
@@ -120,8 +120,8 @@ public class SpatialArgs {
    * inflates the size of the shape but should not shrink it.
    *
    * @return 0 to 0.5
-   * @see #calcDistanceFromErrPct(com.spatial4j.core.shape.Shape, double,
-   *      com.spatial4j.core.context.SpatialContext)
+   * @see #calcDistanceFromErrPct(org.locationtech.spatial4j.shape.Shape, double,
+   *      org.locationtech.spatial4j.context.SpatialContext)
    */
   public Double getDistErrPct() {
     return distErrPct;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgsParser.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgsParser.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgsParser.java
index 81612ff..79ad7a9 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgsParser.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialArgsParser.java
@@ -16,9 +16,9 @@
  */
 package org.apache.lucene.spatial.query;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.exception.InvalidShapeException;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.exception.InvalidShapeException;
+import org.locationtech.spatial4j.shape.Shape;
 
 import java.text.ParseException;
 import java.util.HashMap;
@@ -29,8 +29,8 @@ import java.util.StringTokenizer;
 /**
  * Parses a string that usually looks like "OPERATION(SHAPE)" into a {@link SpatialArgs}
  * object. The set of operations supported are defined in {@link SpatialOperation}, such
- * as "Intersects" being a common one. The shape portion is defined by WKT {@link com.spatial4j.core.io.WktShapeParser},
- * but it can be overridden/customized via {@link #parseShape(String, com.spatial4j.core.context.SpatialContext)}.
+ * as "Intersects" being a common one. The shape portion is defined by WKT {@link org.locationtech.spatial4j.io.WktShapeParser},
+ * but it can be overridden/customized via {@link #parseShape(String, org.locationtech.spatial4j.context.SpatialContext)}.
  * There are some optional name-value pair parameters that follow the closing parenthesis.  Example:
  * <pre>
  *   Intersects(ENVELOPE(-10,-8,22,20)) distErrPct=0.025

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialOperation.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialOperation.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialOperation.java
index 7d750ac..1eeb4bc 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialOperation.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/query/SpatialOperation.java
@@ -16,9 +16,9 @@
  */
 package org.apache.lucene.spatial.query;
 
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
 
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -30,7 +30,7 @@ import java.util.Map;
 /**
  * A predicate that compares a stored geometry to a supplied geometry. It's enum-like. For more
  * explanation of each predicate, consider looking at the source implementation
- * of {@link #evaluate(com.spatial4j.core.shape.Shape, com.spatial4j.core.shape.Shape)}. It's important
+ * of {@link #evaluate(org.locationtech.spatial4j.shape.Shape, org.locationtech.spatial4j.shape.Shape)}. It's important
  * to be aware that Lucene-spatial makes no distinction of shape boundaries, unlike many standardized
  * definitions. Nor does it make dimensional distinctions (e.g. line vs polygon).
  * You can lookup a predicate by "Covers" or "Contains", for example, and you will get the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
index a6c575b..cf2c329 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
@@ -24,10 +24,10 @@ import java.io.FilterOutputStream;
 import java.io.IOException;
 import java.util.Map;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.io.BinaryCodec;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.io.BinaryCodec;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.BinaryDocValues;
@@ -55,7 +55,7 @@ import org.apache.lucene.util.BytesRefBuilder;
  * SpatialStrategy that is approximated (like {@link org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy})
  * to add precision or eventually make more specific / advanced calculations on the per-document
  * geometry.
- * The serialization uses Spatial4j's {@link com.spatial4j.core.io.BinaryCodec}.
+ * The serialization uses Spatial4j's {@link org.locationtech.spatial4j.io.BinaryCodec}.
  *
  * @lucene.experimental
  */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java
index 7a3078a..518fb32 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/spatial4j/Geo3dShape.java
@@ -16,13 +16,13 @@
  */
 package org.apache.lucene.spatial.spatial4j;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceUtils;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
-import com.spatial4j.core.shape.SpatialRelation;
-import com.spatial4j.core.shape.impl.RectangleImpl;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+import org.locationtech.spatial4j.shape.SpatialRelation;
+import org.locationtech.spatial4j.shape.impl.RectangleImpl;
 import org.apache.lucene.geo3d.LatLonBounds;
 import org.apache.lucene.geo3d.GeoArea;
 import org.apache.lucene.geo3d.GeoAreaFactory;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
index 57cad87..7be2433 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java
@@ -29,15 +29,15 @@ import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceCalculator;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceCalculator;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 
 /**
  * The distance from a provided Point to a Point retrieved from a ValueSource via
  * {@link org.apache.lucene.queries.function.FunctionValues#objectVal(int)}. The distance
- * is calculated via a {@link com.spatial4j.core.distance.DistanceCalculator}.
+ * is calculated via a {@link org.locationtech.spatial4j.distance.DistanceCalculator}.
  *
  * @lucene.experimental
  */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java
index dd391d1..257dc67 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeAreaValueSource.java
@@ -22,8 +22,8 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Shape;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
@@ -36,7 +36,7 @@ import org.apache.lucene.search.IndexSearcher;
  * The area of a Shape retrieved from a ValueSource via
  * {@link org.apache.lucene.queries.function.FunctionValues#objectVal(int)}.
  *
- * @see Shape#getArea(com.spatial4j.core.context.SpatialContext)
+ * @see Shape#getArea(org.locationtech.spatial4j.context.SpatialContext)
  *
  * @lucene.experimental
  */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCache.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCache.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCache.java
index 480369b..e24cd68 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCache.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCache.java
@@ -16,7 +16,7 @@
  */
 package org.apache.lucene.spatial.util;
 
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Shape;
 
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java
index e4cb146..1ac84e8 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java
@@ -16,9 +16,9 @@
  */
 package org.apache.lucene.spatial.util;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.distance.DistanceCalculator;
-import com.spatial4j.core.shape.Point;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceCalculator;
+import org.locationtech.spatial4j.shape.Point;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheProvider.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheProvider.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheProvider.java
index 04c52f7..bca73cc 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheProvider.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheProvider.java
@@ -16,7 +16,7 @@
  */
 package org.apache.lucene.spatial.util;
 
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.index.*;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.BytesRef;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java
index b1dfaaa..08c1e43 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java
@@ -16,7 +16,7 @@
  */
 package org.apache.lucene.spatial.util;
 
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.shape.Shape;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
index d31fd59..7cab3fe 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
@@ -16,8 +16,8 @@
  */
 package org.apache.lucene.spatial.vector;
 
-import com.spatial4j.core.distance.DistanceCalculator;
-import com.spatial4j.core.shape.Point;
+import org.locationtech.spatial4j.distance.DistanceCalculator;
+import org.locationtech.spatial4j.shape.Point;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DocValues;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
index f572f82..f5f5f34 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
@@ -16,11 +16,11 @@
  */
 package org.apache.lucene.spatial.vector;
 
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Circle;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Rectangle;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Circle;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.document.LegacyDoubleField;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
@@ -50,7 +50,7 @@ import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
  * org.apache.lucene.spatial.query.SpatialOperation#Intersects} and {@link
  * SpatialOperation#IsWithin} is supported.</li>
  * <li>Uses the FieldCache for
- * {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point)} and for
+ * {@link #makeDistanceValueSource(org.locationtech.spatial4j.shape.Point)} and for
  * searching with a Circle.</li>
  * </ul>
  *
@@ -60,7 +60,7 @@ import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
  * This is a simple Strategy.  Search works with {@link org.apache.lucene.search.LegacyNumericRangeQuery}s on
  * an x and y pair of fields.  A Circle query does the same bbox query but adds a
  * ValueSource filter on
- * {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point)}.
+ * {@link #makeDistanceValueSource(org.locationtech.spatial4j.shape.Point)}.
  * <p>
  * One performance shortcoming with this strategy is that a scenario involving
  * both a search using a Circle and sort will result in calculations for the
@@ -106,7 +106,7 @@ public class PointVectorStrategy extends SpatialStrategy {
     throw new UnsupportedOperationException("Can only index Point, not " + shape);
   }
 
-  /** @see #createIndexableFields(com.spatial4j.core.shape.Shape) */
+  /** @see #createIndexableFields(org.locationtech.spatial4j.shape.Shape) */
   public Field[] createIndexableFields(Point point) {
     FieldType doubleFieldType = new FieldType(LegacyDoubleField.TYPE_NOT_STORED);
     doubleFieldType.setNumericPrecisionStep(precisionStep);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/891e6590/lucene/spatial-extras/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java
index 9a29677..1602679 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java
@@ -22,9 +22,9 @@ import java.util.Arrays;
 import java.util.List;
 
 import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
-import com.spatial4j.core.context.SpatialContext;
-import com.spatial4j.core.shape.Point;
-import com.spatial4j.core.shape.Shape;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Shape;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.spatial.bbox.BBoxStrategy;


[6/9] lucene-solr git commit: LUCENE-7061: fix remaining api issues with XYZPoint classes

Posted by sh...@apache.org.
LUCENE-7061: fix remaining api issues with XYZPoint classes

Squashed commit of the following:

commit 0261e28dd29b1c2a1dcbd5e796966b2cdf2f4b82
Author: Robert Muir <rm...@apache.org>
Date:   Wed Mar 2 14:51:49 2016 -0500

    Add note about comparison order and test extreme values

commit e1f7bc244cd980e931d584c00ba73f8ac521c3fd
Author: Mike McCandless <mi...@apache.org>
Date:   Wed Mar 2 14:25:50 2016 -0500

    add explicit test to verify rect query is inclusive; make test fail fast by default

commit c9be9139ec2f9553ce05fb56b2667be77b8176b6
Author: Robert Muir <rm...@apache.org>
Date:   Wed Mar 2 14:03:47 2016 -0500

    oops

commit 351d0838bbc87dc7c6d83476bd9cb7ce6c38fc3c
Author: Robert Muir <rm...@apache.org>
Date:   Wed Mar 2 13:53:42 2016 -0500

    clean up pointrangequery: remove nulls and inclusives

commit 0796057a8041ddf43341611b477502fa2307f0b1
Merge: 742ee02 e3198ca
Author: Robert Muir <rm...@apache.org>
Date:   Wed Mar 2 13:26:48 2016 -0500

    Merge branch 'master' into unfuck_points

commit 742ee02aaf55439463daddbd3ea16c5e8df31f01
Author: Robert Muir <rm...@apache.org>
Date:   Wed Mar 2 13:16:08 2016 -0500

    Remove nulls and inclusives from points apis
(cherry picked from commit 8eada27)


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

Branch: refs/heads/branch_6_0
Commit: dc35cea65673ca1c340770f3d0999f50834ddea5
Parents: b38c2d6
Author: Robert Muir <rm...@apache.org>
Authored: Thu Mar 3 03:21:16 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Mar 3 20:49:11 2016 +0530

----------------------------------------------------------------------
 .../org/apache/lucene/document/BinaryPoint.java |  82 ++--
 .../org/apache/lucene/document/DoublePoint.java |  84 ++--
 .../org/apache/lucene/document/FloatPoint.java  |  84 ++--
 .../org/apache/lucene/document/IntPoint.java    |  84 ++--
 .../org/apache/lucene/document/LongPoint.java   |  84 ++--
 .../apache/lucene/search/PointInSetQuery.java   |  11 +-
 .../apache/lucene/search/PointRangeQuery.java   | 170 ++------
 .../index/TestDemoParallelLeafReader.java       |   2 +-
 .../apache/lucene/search/TestPointQueries.java  | 410 +++++++++----------
 .../TestUsageTrackingFilterCachingPolicy.java   |   2 +-
 .../demo/facet/DistanceFacetsExample.java       |   8 +-
 .../lucene/demo/facet/RangeFacetsExample.java   |   3 +-
 .../apache/lucene/facet/range/DoubleRange.java  |  33 +-
 .../facet/range/DoubleRangeFacetCounts.java     |   4 +-
 .../apache/lucene/facet/range/LongRange.java    |  25 +-
 .../lucene/facet/range/LongRangeCounter.java    |  14 +-
 .../facet/range/TestRangeFacetCounts.java       |  20 +-
 .../search/highlight/HighlighterTest.java       |   2 +-
 .../apache/lucene/document/BigIntegerPoint.java |  97 +++--
 .../lucene/document/InetAddressPoint.java       |  66 ++-
 .../org/apache/lucene/document/LatLonPoint.java |  11 +-
 .../lucene/document/TestBigIntegerPoint.java    |  12 +-
 .../lucene/document/TestInetAddressPoint.java   |   4 +-
 .../apache/lucene/document/TestLatLonPoint.java |   2 +-
 .../lucene/search/TestDocValuesRangeQuery.java  |  26 +-
 .../lucene/search/TestLatLonPointQueries.java   |   8 +-
 .../spatial/util/BaseGeoPointTestCase.java      |  74 +++-
 .../suggest/document/TestSuggestField.java      |   2 +-
 28 files changed, 636 insertions(+), 788 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java b/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
index b8b53d5..e139a87 100644
--- a/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/BinaryPoint.java
@@ -16,7 +16,6 @@
  */
 package org.apache.lucene.document;
 
-import java.io.IOException;
 import java.util.Arrays;
 import java.util.Comparator;
 
@@ -25,7 +24,6 @@ import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.StringHelper;
 
 /** 
@@ -37,10 +35,10 @@ import org.apache.lucene.util.StringHelper;
  * <p>
  * This field defines static factory methods for creating common queries:
  * <ul>
- *   <li>{@link #newExactQuery newExactQuery()} for matching an exact 1D point.
- *   <li>{@link #newRangeQuery newRangeQuery()} for matching a 1D range.
- *   <li>{@link #newMultiRangeQuery newMultiRangeQuery()} for matching points/ranges in n-dimensional space.
- *   <li>{@link #newSetQuery newSetQuery()} for matching a set of 1D values.
+ *   <li>{@link #newExactQuery(String, byte[])} for matching an exact 1D point.
+ *   <li>{@link #newSetQuery(String, byte[][]) newSetQuery(String, byte[]...)} for matching a set of 1D values.
+ *   <li>{@link #newRangeQuery(String, byte[], byte[])} for matching a 1D range.
+ *   <li>{@link #newRangeQuery(String, byte[][], byte[][])} for matching points/ranges in n-dimensional space.
  * </ul> 
  */
 public final class BinaryPoint extends Field {
@@ -133,7 +131,7 @@ public final class BinaryPoint extends Field {
    * Create a query for matching an exact binary value.
    * <p>
    * This is for simple one-dimension points, for multidimensional points use
-   * {@link #newMultiRangeQuery newMultiRangeQuery()} instead.
+   * {@link #newRangeQuery(String, byte[][], byte[][])} instead.
    *
    * @param field field name. must not be {@code null}.
    * @param value binary value
@@ -141,56 +139,39 @@ public final class BinaryPoint extends Field {
    * @return a query matching documents with this exact value
    */
   public static Query newExactQuery(String field, byte[] value) {
-    if (value == null) {
-      throw new IllegalArgumentException("value cannot be null");
-    }
-    return newRangeQuery(field, value, true, value, true);
+    return newRangeQuery(field, value, value);
   }
 
   /** 
    * Create a range query for binary values.
    * <p>
    * This is for simple one-dimension ranges, for multidimensional ranges use
-   * {@link #newMultiRangeQuery newMultiRangeQuery()} instead.
-   * <p>
-   * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting the {@code lowerValue} or {@code upperValue} to {@code null}. 
-   * <p>
-   * By setting inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
-   * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
+   * {@link #newRangeQuery(String, byte[][], byte[][])} instead.
    *
    * @param field field name. must not be {@code null}.
-   * @param lowerValue lower portion of the range. {@code null} means "open".
-   * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
-   * @param upperValue upper portion of the range. {@code null} means "open".
-   * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
-   * @throws IllegalArgumentException if {@code field} is null.
+   * @param lowerValue lower portion of the range (inclusive). must not be {@code null}
+   * @param upperValue upper portion of the range (inclusive). must not be {@code null}
+   * @throws IllegalArgumentException if {@code field} is null, if {@code lowerValue} is null,
+   *                                  or if {@code upperValue} is null
    * @return a query matching documents within this range.
    */
-  public static Query newRangeQuery(String field, byte[] lowerValue, boolean lowerInclusive, byte[] upperValue, boolean upperInclusive) {
-    return newMultiRangeQuery(field, new byte[][] {lowerValue}, new boolean[] {lowerInclusive}, new byte[][] {upperValue}, new boolean[] {upperInclusive});
+  public static Query newRangeQuery(String field, byte[] lowerValue, byte[] upperValue) {
+    PointRangeQuery.checkArgs(field, lowerValue, upperValue);
+    return newRangeQuery(field, new byte[][] {lowerValue}, new byte[][] {upperValue});
   }
   
   /** 
-   * Create a multidimensional range query for binary values.
-   * <p>
-   * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting a {@code lowerValue} element or {@code upperValue} element to {@code null}. 
-   * <p>
-   * By setting a dimension's inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
-   * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
+   * Create a range query for n-dimensional binary values.
    *
    * @param field field name. must not be {@code null}.
-   * @param lowerValue lower portion of the range. {@code null} values mean "open" for that dimension.
-   * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
-   * @param upperValue upper portion of the range. {@code null} values mean "open" for that dimension.
-   * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
-   * @throws IllegalArgumentException if {@code field} is null, or if {@code lowerValue.length != upperValue.length}
+   * @param lowerValue lower portion of the range (inclusive). must not be null.
+   * @param upperValue upper portion of the range (inclusive). must not be null.
+   * @throws IllegalArgumentException if {@code field} is null, if {@code lowerValue} is null, if {@code upperValue} is null, 
+   *                                  or if {@code lowerValue.length != upperValue.length}
    * @return a query matching documents within this range.
    */
-  public static Query newMultiRangeQuery(String field, byte[][] lowerValue, boolean[] lowerInclusive, byte[][] upperValue, boolean[] upperInclusive) {
-    PointRangeQuery.checkArgs(field, lowerValue, upperValue);
-    return new PointRangeQuery(field, lowerValue, lowerInclusive, upperValue, upperInclusive) {
+  public static Query newRangeQuery(String field, byte[][] lowerValue, byte[][] upperValue) {
+    return new PointRangeQuery(field, lowerValue, upperValue) {
       @Override
       protected String toString(int dimension, byte[] value) {
         assert value != null;
@@ -212,13 +193,13 @@ public final class BinaryPoint extends Field {
    * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
    * 
    * @param field field name. must not be {@code null}.
-   * @param valuesIn all values to match
+   * @param values all values to match
    */
-  public static Query newSetQuery(String field, byte[]... valuesIn) throws IOException {
+  public static Query newSetQuery(String field, byte[]... values) {
 
     // Make sure all byte[] have the same length
     int bytesPerDim = -1;
-    for(byte[] value : valuesIn) {
+    for(byte[] value : values) {
       if (bytesPerDim == -1) {
         bytesPerDim = value.length;
       } else if (value.length != bytesPerDim) {
@@ -232,9 +213,8 @@ public final class BinaryPoint extends Field {
     }
 
     // Don't unexpectedly change the user's incoming values array:
-    byte[][] values = valuesIn.clone();
-
-    Arrays.sort(values,
+    byte[][] sortedValues = values.clone();
+    Arrays.sort(sortedValues,
                 new Comparator<byte[]>() {
                   @Override
                   public int compare(byte[] a, byte[] b) {
@@ -242,21 +222,21 @@ public final class BinaryPoint extends Field {
                   }
                 });
 
-    final BytesRef value = new BytesRef(new byte[bytesPerDim]);
+    final BytesRef encoded = new BytesRef(new byte[bytesPerDim]);
     
     return new PointInSetQuery(field, 1, bytesPerDim,
-                               new BytesRefIterator() {
+                               new PointInSetQuery.Stream() {
 
                                  int upto;
 
                                  @Override
                                  public BytesRef next() {
-                                   if (upto == values.length) {
+                                   if (upto == sortedValues.length) {
                                      return null;
                                    } else {
-                                     value.bytes = values[upto];
+                                     encoded.bytes = sortedValues[upto];
                                      upto++;
-                                     return value;
+                                     return encoded;
                                    }
                                  }
                                }) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
index 58ec364..9dbd96e 100644
--- a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
@@ -16,14 +16,12 @@
  */
 package org.apache.lucene.document;
 
-import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.NumericUtils;
 
 /** 
@@ -35,10 +33,10 @@ import org.apache.lucene.util.NumericUtils;
  * <p>
  * This field defines static factory methods for creating common queries:
  * <ul>
- *   <li>{@link #newExactQuery newExactQuery()} for matching an exact 1D point.
- *   <li>{@link #newRangeQuery newRangeQuery()} for matching a 1D range.
- *   <li>{@link #newMultiRangeQuery newMultiRangeQuery()} for matching points/ranges in n-dimensional space.
- *   <li>{@link #newSetQuery newSetQuery()} for matching a set of 1D values.
+ *   <li>{@link #newExactQuery(String, double)} for matching an exact 1D point.
+ *   <li>{@link #newSetQuery(String, double...)} for matching a set of 1D values.
+ *   <li>{@link #newRangeQuery(String, double, double)} for matching a 1D range.
+ *   <li>{@link #newRangeQuery(String, double[], double[])} for matching points/ranges in n-dimensional space.
  * </ul> 
  */
 public final class DoublePoint extends Field {
@@ -126,13 +124,11 @@ public final class DoublePoint extends Field {
   }
   
   /** Encode n-dimensional double point into binary encoding */
-  private static byte[][] encode(Double value[]) {
+  private static byte[][] encode(double value[]) {
     byte[][] encoded = new byte[value.length][];
     for (int i = 0; i < value.length; i++) {
-      if (value[i] != null) {
-        encoded[i] = new byte[Double.BYTES];
-        encodeDimension(value[i], encoded[i], 0);
-      }
+      encoded[i] = new byte[Double.BYTES];
+      encodeDimension(value[i], encoded[i], 0);
     }
     return encoded;
   }
@@ -155,7 +151,7 @@ public final class DoublePoint extends Field {
    * Create a query for matching an exact double value.
    * <p>
    * This is for simple one-dimension points, for multidimensional points use
-   * {@link #newMultiRangeQuery newMultiRangeQuery()} instead.
+   * {@link #newRangeQuery(String, double[], double[])} instead.
    *
    * @param field field name. must not be {@code null}.
    * @param value double value
@@ -163,60 +159,51 @@ public final class DoublePoint extends Field {
    * @return a query matching documents with this exact value
    */
   public static Query newExactQuery(String field, double value) {
-    return newRangeQuery(field, value, true, value, true);
+    return newRangeQuery(field, value, value);
   }
   
   /** 
    * Create a range query for double values.
    * <p>
    * This is for simple one-dimension ranges, for multidimensional ranges use
-   * {@link #newMultiRangeQuery newMultiRangeQuery()} instead.
+   * {@link #newRangeQuery(String, double[], double[])} instead.
    * <p>
    * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting the {@code lowerValue} or {@code upperValue} to {@code null}. 
+   * by setting {@code lowerValue = Double.NEGATIVE_INFINITY} or {@code upperValue = Double.POSITIVE_INFINITY}. 
    * <p>
-   * By setting inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
-   * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
+   * Range comparisons are consistent with {@link Double#compareTo(Double)}.
    *
    * @param field field name. must not be {@code null}.
-   * @param lowerValue lower portion of the range. {@code null} means "open".
-   * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
-   * @param upperValue upper portion of the range. {@code null} means "open".
-   * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
+   * @param lowerValue lower portion of the range (inclusive).
+   * @param upperValue upper portion of the range (inclusive).
    * @throws IllegalArgumentException if {@code field} is null.
    * @return a query matching documents within this range.
    */
-  public static Query newRangeQuery(String field, Double lowerValue, boolean lowerInclusive, Double upperValue, boolean upperInclusive) {
-    return newMultiRangeQuery(field, 
-                              new Double[] { lowerValue },
-                              new boolean[] { lowerInclusive }, 
-                              new Double[] { upperValue },
-                              new boolean[] { upperInclusive });
+  public static Query newRangeQuery(String field, double lowerValue, double upperValue) {
+    return newRangeQuery(field, new double[] { lowerValue }, new double[] { upperValue });
   }
 
   /** 
-   * Create a multidimensional range query for double values.
+   * Create a range query for n-dimensional double values.
    * <p>
    * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting a {@code lowerValue} element or {@code upperValue} element to {@code null}. 
+   * by setting {@code lowerValue[i] = Double.NEGATIVE_INFINITY} or {@code upperValue[i] = Double.POSITIVE_INFINITY}. 
    * <p>
-   * By setting a dimension's inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
-   * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
+   * Range comparisons are consistent with {@link Double#compareTo(Double)}.
    *
    * @param field field name. must not be {@code null}.
-   * @param lowerValue lower portion of the range. {@code null} values mean "open" for that dimension.
-   * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
-   * @param upperValue upper portion of the range. {@code null} values mean "open" for that dimension.
-   * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
-   * @throws IllegalArgumentException if {@code field} is null, or if {@code lowerValue.length != upperValue.length}
+   * @param lowerValue lower portion of the range (inclusive). must not be {@code null}.
+   * @param upperValue upper portion of the range (inclusive). must not be {@code null}.
+   * @throws IllegalArgumentException if {@code field} is null, if {@code lowerValue} is null, if {@code upperValue} is null, 
+   *                                  or if {@code lowerValue.length != upperValue.length}
    * @return a query matching documents within this range.
    */
-  public static Query newMultiRangeQuery(String field, Double[] lowerValue, boolean lowerInclusive[], Double[] upperValue, boolean upperInclusive[]) {
+  public static Query newRangeQuery(String field, double[] lowerValue, double[] upperValue) {
     PointRangeQuery.checkArgs(field, lowerValue, upperValue);
-    return new PointRangeQuery(field, DoublePoint.encode(lowerValue), lowerInclusive, DoublePoint.encode(upperValue), upperInclusive) {
+    return new PointRangeQuery(field, encode(lowerValue), encode(upperValue)) {
       @Override
       protected String toString(int dimension, byte[] value) {
-        return Double.toString(DoublePoint.decodeDimension(value, 0));
+        return Double.toString(decodeDimension(value, 0));
       }
     };
   }
@@ -225,30 +212,29 @@ public final class DoublePoint extends Field {
    * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
    * 
    * @param field field name. must not be {@code null}.
-   * @param valuesIn all values to match
+   * @param values all values to match
    */
-  public static Query newSetQuery(String field, double... valuesIn) throws IOException {
+  public static Query newSetQuery(String field, double... values) {
 
     // Don't unexpectedly change the user's incoming values array:
-    double[] values = valuesIn.clone();
-
-    Arrays.sort(values);
+    double[] sortedValues = values.clone();
+    Arrays.sort(sortedValues);
 
-    final BytesRef value = new BytesRef(new byte[Double.BYTES]);
+    final BytesRef encoded = new BytesRef(new byte[Double.BYTES]);
 
     return new PointInSetQuery(field, 1, Double.BYTES,
-                               new BytesRefIterator() {
+                               new PointInSetQuery.Stream() {
 
                                  int upto;
 
                                  @Override
                                  public BytesRef next() {
-                                   if (upto == values.length) {
+                                   if (upto == sortedValues.length) {
                                      return null;
                                    } else {
-                                     encodeDimension(values[upto], value.bytes, 0);
+                                     encodeDimension(sortedValues[upto], encoded.bytes, 0);
                                      upto++;
-                                     return value;
+                                     return encoded;
                                    }
                                  }
                                }) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
index 7829f80..0b82abe 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
@@ -16,14 +16,12 @@
  */
 package org.apache.lucene.document;
 
-import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.NumericUtils;
 
 /** 
@@ -35,10 +33,10 @@ import org.apache.lucene.util.NumericUtils;
  * <p>
  * This field defines static factory methods for creating common queries:
  * <ul>
- *   <li>{@link #newExactQuery newExactQuery()} for matching an exact 1D point.
- *   <li>{@link #newRangeQuery newRangeQuery()} for matching a 1D range.
- *   <li>{@link #newMultiRangeQuery newMultiRangeQuery()} for matching points/ranges in n-dimensional space.
- *   <li>{@link #newSetQuery newSetQuery()} for matching a set of 1D values.
+ *   <li>{@link #newExactQuery(String, float)} for matching an exact 1D point.
+ *   <li>{@link #newSetQuery(String, float...)} for matching a set of 1D values.
+ *   <li>{@link #newRangeQuery(String, float, float)} for matching a 1D range.
+ *   <li>{@link #newRangeQuery(String, float[], float[])} for matching points/ranges in n-dimensional space.
  * </ul>
  */
 public final class FloatPoint extends Field {
@@ -126,13 +124,11 @@ public final class FloatPoint extends Field {
   }
   
   /** Encode n-dimensional float values into binary encoding */
-  private static byte[][] encode(Float value[]) {
+  private static byte[][] encode(float value[]) {
     byte[][] encoded = new byte[value.length][];
     for (int i = 0; i < value.length; i++) {
-      if (value[i] != null) {
-        encoded[i] = new byte[Float.BYTES];
-        encodeDimension(value[i], encoded[i], 0);
-      }
+      encoded[i] = new byte[Float.BYTES];
+      encodeDimension(value[i], encoded[i], 0);
     }
     return encoded;
   }
@@ -155,7 +151,7 @@ public final class FloatPoint extends Field {
    * Create a query for matching an exact float value.
    * <p>
    * This is for simple one-dimension points, for multidimensional points use
-   * {@link #newMultiRangeQuery newMultiRangeQuery()} instead.
+   * {@link #newRangeQuery(String, float[], float[])} instead.
    *
    * @param field field name. must not be {@code null}.
    * @param value float value
@@ -163,60 +159,51 @@ public final class FloatPoint extends Field {
    * @return a query matching documents with this exact value
    */
   public static Query newExactQuery(String field, float value) {
-    return newRangeQuery(field, value, true, value, true);
+    return newRangeQuery(field, value, value);
   }
   
   /** 
    * Create a range query for float values.
    * <p>
    * This is for simple one-dimension ranges, for multidimensional ranges use
-   * {@link #newMultiRangeQuery newMultiRangeQuery()} instead.
+   * {@link #newRangeQuery(String, float[], float[])} instead.
    * <p>
    * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting the {@code lowerValue} or {@code upperValue} to {@code null}. 
+   * by setting {@code lowerValue = Float.NEGATIVE_INFINITY} or {@code upperValue = Float.POSITIVE_INFINITY}. 
    * <p>
-   * By setting inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
-   * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
+   * Range comparisons are consistent with {@link Float#compareTo(Float)}.
    *
    * @param field field name. must not be {@code null}.
-   * @param lowerValue lower portion of the range. {@code null} means "open".
-   * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
-   * @param upperValue upper portion of the range. {@code null} means "open".
-   * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
+   * @param lowerValue lower portion of the range (inclusive).
+   * @param upperValue upper portion of the range (inclusive).
    * @throws IllegalArgumentException if {@code field} is null.
    * @return a query matching documents within this range.
    */
-  public static Query newRangeQuery(String field, Float lowerValue, boolean lowerInclusive, Float upperValue, boolean upperInclusive) {
-    return newMultiRangeQuery(field, 
-                              new Float[] { lowerValue },
-                              new boolean[] { lowerInclusive }, 
-                              new Float[] { upperValue },
-                              new boolean[] { upperInclusive });
+  public static Query newRangeQuery(String field, float lowerValue, float upperValue) {
+    return newRangeQuery(field, new float[] { lowerValue }, new float[] { upperValue });
   }
 
   /** 
-   * Create a multidimensional range query for float values.
+   * Create a range query for n-dimensional float values.
    * <p>
    * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting a {@code lowerValue} element or {@code upperValue} element to {@code null}. 
+   * by setting {@code lowerValue[i] = Float.NEGATIVE_INFINITY} or {@code upperValue[i] = Float.POSITIVE_INFINITY}. 
    * <p>
-   * By setting a dimension's inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
-   * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
+   * Range comparisons are consistent with {@link Float#compareTo(Float)}.
    *
    * @param field field name. must not be {@code null}.
-   * @param lowerValue lower portion of the range. {@code null} values mean "open" for that dimension.
-   * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
-   * @param upperValue upper portion of the range. {@code null} values mean "open" for that dimension.
-   * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
-   * @throws IllegalArgumentException if {@code field} is null, or if {@code lowerValue.length != upperValue.length}
+   * @param lowerValue lower portion of the range (inclusive). must not be {@code null}.
+   * @param upperValue upper portion of the range (inclusive). must not be {@code null}.
+   * @throws IllegalArgumentException if {@code field} is null, if {@code lowerValue} is null, if {@code upperValue} is null, 
+   *                                  or if {@code lowerValue.length != upperValue.length}
    * @return a query matching documents within this range.
    */
-  public static Query newMultiRangeQuery(String field, Float[] lowerValue, boolean lowerInclusive[], Float[] upperValue, boolean upperInclusive[]) {
+  public static Query newRangeQuery(String field, float[] lowerValue, float[] upperValue) {
     PointRangeQuery.checkArgs(field, lowerValue, upperValue);
-    return new PointRangeQuery(field, FloatPoint.encode(lowerValue), lowerInclusive, FloatPoint.encode(upperValue), upperInclusive) {
+    return new PointRangeQuery(field, encode(lowerValue), encode(upperValue)) {
       @Override
       protected String toString(int dimension, byte[] value) {
-        return Float.toString(FloatPoint.decodeDimension(value, 0));
+        return Float.toString(decodeDimension(value, 0));
       }
     };
   }
@@ -225,30 +212,29 @@ public final class FloatPoint extends Field {
    * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
    * 
    * @param field field name. must not be {@code null}.
-   * @param valuesIn all values to match
+   * @param values all values to match
    */
-  public static Query newSetQuery(String field, float... valuesIn) throws IOException {
+  public static Query newSetQuery(String field, float... values) {
 
     // Don't unexpectedly change the user's incoming values array:
-    float[] values = valuesIn.clone();
-
-    Arrays.sort(values);
+    float[] sortedValues = values.clone();
+    Arrays.sort(sortedValues);
 
-    final BytesRef value = new BytesRef(new byte[Float.BYTES]);
+    final BytesRef encoded = new BytesRef(new byte[Float.BYTES]);
 
     return new PointInSetQuery(field, 1, Float.BYTES,
-                               new BytesRefIterator() {
+                               new PointInSetQuery.Stream() {
 
                                  int upto;
 
                                  @Override
                                  public BytesRef next() {
-                                   if (upto == values.length) {
+                                   if (upto == sortedValues.length) {
                                      return null;
                                    } else {
-                                     encodeDimension(values[upto], value.bytes, 0);
+                                     encodeDimension(sortedValues[upto], encoded.bytes, 0);
                                      upto++;
-                                     return value;
+                                     return encoded;
                                    }
                                  }
                                }) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
index f27df75..effcb62 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
@@ -16,14 +16,12 @@
  */
 package org.apache.lucene.document;
 
-import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.NumericUtils;
 
 /** 
@@ -35,10 +33,10 @@ import org.apache.lucene.util.NumericUtils;
  * <p>
  * This field defines static factory methods for creating common queries:
  * <ul>
- *   <li>{@link #newExactQuery newExactQuery()} for matching an exact 1D point.
- *   <li>{@link #newRangeQuery newRangeQuery()} for matching a 1D range.
- *   <li>{@link #newMultiRangeQuery newMultiRangeQuery()} for matching points/ranges in n-dimensional space.
- *   <li>{@link #newSetQuery newSetQuery()} for matching a set of 1D values.
+ *   <li>{@link #newExactQuery(String, int)} for matching an exact 1D point.
+ *   <li>{@link #newSetQuery(String, int...)} for matching a set of 1D values.
+ *   <li>{@link #newRangeQuery(String, int, int)} for matching a 1D range.
+ *   <li>{@link #newRangeQuery(String, int[], int[])} for matching points/ranges in n-dimensional space.
  * </ul>
  */
 public final class IntPoint extends Field {
@@ -126,13 +124,11 @@ public final class IntPoint extends Field {
   }
 
   /** Encode n-dimensional integer values into binary encoding */
-  private static byte[][] encode(Integer value[]) {
+  private static byte[][] encode(int value[]) {
     byte[][] encoded = new byte[value.length][];
     for (int i = 0; i < value.length; i++) {
-      if (value[i] != null) {
-        encoded[i] = new byte[Integer.BYTES];
-        encodeDimension(value[i], encoded[i], 0);
-      }
+      encoded[i] = new byte[Integer.BYTES];
+      encodeDimension(value[i], encoded[i], 0);
     }
     return encoded;
   }
@@ -155,7 +151,7 @@ public final class IntPoint extends Field {
    * Create a query for matching an exact integer value.
    * <p>
    * This is for simple one-dimension points, for multidimensional points use
-   * {@link #newMultiRangeQuery newMultiRangeQuery()} instead.
+   * {@link #newRangeQuery(String, int[], int[])} instead.
    *
    * @param field field name. must not be {@code null}.
    * @param value exact value
@@ -163,60 +159,51 @@ public final class IntPoint extends Field {
    * @return a query matching documents with this exact value
    */
   public static Query newExactQuery(String field, int value) {
-    return newRangeQuery(field, value, true, value, true);
+    return newRangeQuery(field, value, value);
   }
 
   /** 
    * Create a range query for integer values.
    * <p>
    * This is for simple one-dimension ranges, for multidimensional ranges use
-   * {@link #newMultiRangeQuery newMultiRangeQuery()} instead.
+   * {@link #newRangeQuery(String, int[], int[])} instead.
    * <p>
    * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting the {@code lowerValue} or {@code upperValue} to {@code null}. 
+   * by setting {@code lowerValue = Integer.MIN_VALUE} or {@code upperValue = Integer.MAX_VALUE}. 
    * <p>
-   * By setting inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
-   * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
+   * Ranges are inclusive. For exclusive ranges, pass {@code lowerValue + 1} or {@code upperValue - 1}
    *
    * @param field field name. must not be {@code null}.
-   * @param lowerValue lower portion of the range. {@code null} means "open".
-   * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
-   * @param upperValue upper portion of the range. {@code null} means "open".
-   * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
+   * @param lowerValue lower portion of the range (inclusive).
+   * @param upperValue upper portion of the range (inclusive).
    * @throws IllegalArgumentException if {@code field} is null.
    * @return a query matching documents within this range.
    */
-  public static Query newRangeQuery(String field, Integer lowerValue, boolean lowerInclusive, Integer upperValue, boolean upperInclusive) {
-    return newMultiRangeQuery(field, 
-                              new Integer[] { lowerValue },
-                              new boolean[] { lowerInclusive }, 
-                              new Integer[] { upperValue },
-                              new boolean[] { upperInclusive });
+  public static Query newRangeQuery(String field, int lowerValue, int upperValue) {
+    return newRangeQuery(field, new int[] { lowerValue }, new int[] { upperValue });
   }
 
   /** 
-   * Create a multidimensional range query for integer values.
+   * Create a range query for n-dimensional integer values.
    * <p>
    * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting a {@code lowerValue} element or {@code upperValue} element to {@code null}. 
+   * by setting {@code lowerValue[i] = Integer.MIN_VALUE} or {@code upperValue[i] = Integer.MAX_VALUE}. 
    * <p>
-   * By setting a dimension's inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
-   * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
+   * Ranges are inclusive. For exclusive ranges, pass {@code lowerValue[i] + 1} or {@code upperValue[i] - 1}
    *
    * @param field field name. must not be {@code null}.
-   * @param lowerValue lower portion of the range. {@code null} values mean "open" for that dimension.
-   * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
-   * @param upperValue upper portion of the range. {@code null} values mean "open" for that dimension.
-   * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
-   * @throws IllegalArgumentException if {@code field} is null, or if {@code lowerValue.length != upperValue.length}
+   * @param lowerValue lower portion of the range (inclusive). must not be {@code null}.
+   * @param upperValue upper portion of the range (inclusive). must not be {@code null}.
+   * @throws IllegalArgumentException if {@code field} is null, if {@code lowerValue} is null, if {@code upperValue} is null, 
+   *                                  or if {@code lowerValue.length != upperValue.length}
    * @return a query matching documents within this range.
    */
-  public static Query newMultiRangeQuery(String field, Integer[] lowerValue, boolean lowerInclusive[], Integer[] upperValue, boolean upperInclusive[]) {
+  public static Query newRangeQuery(String field, int[] lowerValue, int[] upperValue) {
     PointRangeQuery.checkArgs(field, lowerValue, upperValue);
-    return new PointRangeQuery(field, IntPoint.encode(lowerValue), lowerInclusive, IntPoint.encode(upperValue), upperInclusive) {
+    return new PointRangeQuery(field, encode(lowerValue), encode(upperValue)) {
       @Override
       protected String toString(int dimension, byte[] value) {
-        return Integer.toString(IntPoint.decodeDimension(value, 0));
+        return Integer.toString(decodeDimension(value, 0));
       }
     };
   }
@@ -225,30 +212,29 @@ public final class IntPoint extends Field {
    * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
    * 
    * @param field field name. must not be {@code null}.
-   * @param valuesIn all values to match
+   * @param values all values to match
    */
-  public static Query newSetQuery(String field, int... valuesIn) throws IOException {
+  public static Query newSetQuery(String field, int... values) {
 
     // Don't unexpectedly change the user's incoming values array:
-    int[] values = valuesIn.clone();
-
-    Arrays.sort(values);
+    int[] sortedValues = values.clone();
+    Arrays.sort(sortedValues);
 
-    final BytesRef value = new BytesRef(new byte[Integer.BYTES]);
+    final BytesRef encoded = new BytesRef(new byte[Integer.BYTES]);
 
     return new PointInSetQuery(field, 1, Integer.BYTES,
-                               new BytesRefIterator() {
+                               new PointInSetQuery.Stream() {
 
                                  int upto;
 
                                  @Override
                                  public BytesRef next() {
-                                   if (upto == values.length) {
+                                   if (upto == sortedValues.length) {
                                      return null;
                                    } else {
-                                     encodeDimension(values[upto], value.bytes, 0);
+                                     encodeDimension(sortedValues[upto], encoded.bytes, 0);
                                      upto++;
-                                     return value;
+                                     return encoded;
                                    }
                                  }
                                }) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
index 5b37089..a1d05d1 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
@@ -16,14 +16,12 @@
  */
 package org.apache.lucene.document;
 
-import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.NumericUtils;
 
 /** 
@@ -35,10 +33,10 @@ import org.apache.lucene.util.NumericUtils;
  * <p>
  * This field defines static factory methods for creating common queries:
  * <ul>
- *   <li>{@link #newExactQuery newExactQuery()} for matching an exact 1D point.
- *   <li>{@link #newRangeQuery newRangeQuery()} for matching a 1D range.
- *   <li>{@link #newMultiRangeQuery newMultiRangeQuery()} for matching points/ranges in n-dimensional space.
- *   <li>{@link #newSetQuery newSetQuery()} for matching a set of 1D values.
+ *   <li>{@link #newExactQuery(String, long)} for matching an exact 1D point.
+ *   <li>{@link #newSetQuery(String, long...)} for matching a set of 1D values.
+ *   <li>{@link #newRangeQuery(String, long, long)} for matching a 1D range.
+ *   <li>{@link #newRangeQuery(String, long[], long[])} for matching points/ranges in n-dimensional space.
  * </ul>
  */
 public final class LongPoint extends Field {
@@ -126,13 +124,11 @@ public final class LongPoint extends Field {
   }
   
   /** Encode n-dimensional long values into binary encoding */
-  private static byte[][] encode(Long value[]) {
+  private static byte[][] encode(long value[]) {
     byte[][] encoded = new byte[value.length][];
     for (int i = 0; i < value.length; i++) {
-      if (value[i] != null) {
-        encoded[i] = new byte[Long.BYTES];
-        encodeDimension(value[i], encoded[i], 0);
-      }
+      encoded[i] = new byte[Long.BYTES];
+      encodeDimension(value[i], encoded[i], 0);
     }
     return encoded;
   }
@@ -155,7 +151,7 @@ public final class LongPoint extends Field {
    * Create a query for matching an exact long value.
    * <p>
    * This is for simple one-dimension points, for multidimensional points use
-   * {@link #newMultiRangeQuery newMultiRangeQuery()} instead.
+   * {@link #newRangeQuery(String, long[], long[])} instead.
    *
    * @param field field name. must not be {@code null}.
    * @param value exact value
@@ -163,60 +159,51 @@ public final class LongPoint extends Field {
    * @return a query matching documents with this exact value
    */
   public static Query newExactQuery(String field, long value) {
-    return newRangeQuery(field, value, true, value, true);
+    return newRangeQuery(field, value, value);
   }
 
   /** 
    * Create a range query for long values.
    * <p>
    * This is for simple one-dimension ranges, for multidimensional ranges use
-   * {@link #newMultiRangeQuery newMultiRangeQuery()} instead.
+   * {@link #newRangeQuery(String, long[], long[])} instead.
    * <p>
    * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting the {@code lowerValue} or {@code upperValue} to {@code null}. 
+   * by setting {@code lowerValue = Long.MIN_VALUE} or {@code upperValue = Long.MAX_VALUE}. 
    * <p>
-   * By setting inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
-   * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
+   * Ranges are inclusive. For exclusive ranges, pass {@code lowerValue + 1} or {@code upperValue - 1}
    *
    * @param field field name. must not be {@code null}.
-   * @param lowerValue lower portion of the range. {@code null} means "open".
-   * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
-   * @param upperValue upper portion of the range. {@code null} means "open".
-   * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
+   * @param lowerValue lower portion of the range (inclusive).
+   * @param upperValue upper portion of the range (inclusive).
    * @throws IllegalArgumentException if {@code field} is null.
    * @return a query matching documents within this range.
    */
-  public static Query newRangeQuery(String field, Long lowerValue, boolean lowerInclusive, Long upperValue, boolean upperInclusive) {
-    return newMultiRangeQuery(field, 
-                              new Long[] { lowerValue },
-                              new boolean[] { lowerInclusive }, 
-                              new Long[] { upperValue },
-                              new boolean[] { upperInclusive });
+  public static Query newRangeQuery(String field, long lowerValue, long upperValue) {
+    return newRangeQuery(field, new long[] { lowerValue }, new long[] { upperValue });
   }
 
   /** 
-   * Create a multidimensional range query for long values.
+   * Create a range query for n-dimensional long values.
    * <p>
    * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting a {@code lowerValue} element or {@code upperValue} element to {@code null}. 
+   * by setting {@code lowerValue[i] = Long.MIN_VALUE} or {@code upperValue[i] = Long.MAX_VALUE}. 
    * <p>
-   * By setting a dimension's inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
-   * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
+   * Ranges are inclusive. For exclusive ranges, pass {@code lowerValue[i] + 1} or {@code upperValue[i] - 1}
    *
    * @param field field name. must not be {@code null}.
-   * @param lowerValue lower portion of the range. {@code null} values mean "open" for that dimension.
-   * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
-   * @param upperValue upper portion of the range. {@code null} values mean "open" for that dimension.
-   * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
-   * @throws IllegalArgumentException if {@code field} is null, or if {@code lowerValue.length != upperValue.length}
+   * @param lowerValue lower portion of the range (inclusive). must not be {@code null}.
+   * @param upperValue upper portion of the range (inclusive). must not be {@code null}.
+   * @throws IllegalArgumentException if {@code field} is null, if {@code lowerValue} is null, if {@code upperValue} is null, 
+   *                                  or if {@code lowerValue.length != upperValue.length}
    * @return a query matching documents within this range.
    */
-  public static Query newMultiRangeQuery(String field, Long[] lowerValue, boolean lowerInclusive[], Long[] upperValue, boolean upperInclusive[]) {
+  public static Query newRangeQuery(String field, long[] lowerValue, long[] upperValue) {
     PointRangeQuery.checkArgs(field, lowerValue, upperValue);
-    return new PointRangeQuery(field, LongPoint.encode(lowerValue), lowerInclusive, LongPoint.encode(upperValue), upperInclusive) {
+    return new PointRangeQuery(field, encode(lowerValue), encode(upperValue)) {
       @Override
       protected String toString(int dimension, byte[] value) {
-        return Long.toString(LongPoint.decodeDimension(value, 0));
+        return Long.toString(decodeDimension(value, 0));
       }
     };
   }
@@ -225,30 +212,29 @@ public final class LongPoint extends Field {
    * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
    * 
    * @param field field name. must not be {@code null}.
-   * @param valuesIn all values to match
+   * @param values all values to match
    */
-  public static Query newSetQuery(String field, long... valuesIn) throws IOException {
+  public static Query newSetQuery(String field, long... values) {
 
     // Don't unexpectedly change the user's incoming values array:
-    long[] values = valuesIn.clone();
-
-    Arrays.sort(values);
+    long[] sortedValues = values.clone();
+    Arrays.sort(sortedValues);
 
-    final BytesRef value = new BytesRef(new byte[Long.BYTES]);
+    final BytesRef encoded = new BytesRef(new byte[Long.BYTES]);
 
     return new PointInSetQuery(field, 1, Long.BYTES,
-                               new BytesRefIterator() {
+                               new PointInSetQuery.Stream() {
 
                                  int upto;
 
                                  @Override
                                  public BytesRef next() {
-                                   if (upto == values.length) {
+                                   if (upto == sortedValues.length) {
                                      return null;
                                    } else {
-                                     encodeDimension(values[upto], value.bytes, 0);
+                                     encodeDimension(sortedValues[upto], encoded.bytes, 0);
                                      upto++;
-                                     return value;
+                                     return encoded;
                                    }
                                  }
                                }) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index 3d6086c..f5ba12d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -64,9 +64,18 @@ public abstract class PointInSetQuery extends Query {
   final String field;
   final int numDims;
   final int bytesPerDim;
+  
+  /** 
+   * Iterator of encoded point values.
+   */
+  // TODO: if we want to stream, maybe we should use jdk stream class?
+  public static abstract class Stream implements BytesRefIterator {
+    @Override
+    public abstract BytesRef next();
+  };
 
   /** The {@code packedPoints} iterator must be in sorted order. */
-  protected PointInSetQuery(String field, int numDims, int bytesPerDim, BytesRefIterator packedPoints) throws IOException {
+  protected PointInSetQuery(String field, int numDims, int bytesPerDim, Stream packedPoints) {
     this.field = field;
     if (bytesPerDim < 1 || bytesPerDim > PointValues.MAX_NUM_BYTES) {
       throw new IllegalArgumentException("bytesPerDim must be > 0 and <= " + PointValues.MAX_NUM_BYTES + "; got " + bytesPerDim);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
index 189ba43..85c486e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
@@ -16,7 +16,6 @@
  */
 package org.apache.lucene.search;
 
-
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Objects;
@@ -33,7 +32,6 @@ import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.util.DocIdSetBuilder;
-import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.StringHelper;
 
 /** 
@@ -57,72 +55,49 @@ import org.apache.lucene.util.StringHelper;
 public abstract class PointRangeQuery extends Query {
   final String field;
   final int numDims;
+  final int bytesPerDim;
   final byte[][] lowerPoint;
-  final boolean[] lowerInclusive;
   final byte[][] upperPoint;
-  final boolean[] upperInclusive;
-  // This is null only in the "fully open range" case
-  final Integer bytesPerDim;
 
   /** 
    * Expert: create a multidimensional range query for point values.
-   * <p>
-   * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting a {@code lowerValue} element or {@code upperValue} element to {@code null}. 
-   * <p>
-   * By setting a dimension's inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
-   * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
    *
    * @param field field name. must not be {@code null}.
-   * @param lowerPoint lower portion of the range. {@code null} values mean "open" for that dimension.
-   * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
-   * @param upperPoint upper portion of the range. {@code null} values mean "open" for that dimension.
-   * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
+   * @param lowerPoint lower portion of the range (inclusive).
+   * @param upperPoint upper portion of the range (inclusive).
    * @throws IllegalArgumentException if {@code field} is null, or if {@code lowerValue.length != upperValue.length}
    */
-  protected PointRangeQuery(String field,
-                         byte[][] lowerPoint, boolean[] lowerInclusive,
-                         byte[][] upperPoint, boolean[] upperInclusive) {
+  protected PointRangeQuery(String field, byte[][] lowerPoint, byte[][] upperPoint) {
     checkArgs(field, lowerPoint, upperPoint);
     this.field = field;
-    numDims = lowerPoint.length;
+    if (lowerPoint.length == 0) {
+      throw new IllegalArgumentException("lowerPoint has length of zero");
+    }
+    this.numDims = lowerPoint.length;
+
     if (upperPoint.length != numDims) {
       throw new IllegalArgumentException("lowerPoint has length=" + numDims + " but upperPoint has different length=" + upperPoint.length);
     }
-    if (lowerInclusive.length != numDims) {
-      throw new IllegalArgumentException("lowerInclusive has length=" + lowerInclusive.length + " but expected=" + numDims);
-    }
-    if (upperInclusive.length != numDims) {
-      throw new IllegalArgumentException("upperInclusive has length=" + upperInclusive.length + " but expected=" + numDims);
-    }
     this.lowerPoint = lowerPoint;
-    this.lowerInclusive = lowerInclusive;
     this.upperPoint = upperPoint;
-    this.upperInclusive = upperInclusive;
-
-    int bytesPerDim = -1;
-    for(byte[] value : lowerPoint) {
-      if (value != null) {
-        if (bytesPerDim == -1) {
-          bytesPerDim = value.length;
-        } else if (value.length != bytesPerDim) {
-          throw new IllegalArgumentException("all dimensions must have same bytes length, but saw " + bytesPerDim + " and " + value.length);
-        }
-      }
+
+    if (lowerPoint[0] == null) {
+      throw new IllegalArgumentException("lowerPoint[0] is null");
     }
-    for(byte[] value : upperPoint) {
-      if (value != null) {
-        if (bytesPerDim == -1) {
-          bytesPerDim = value.length;
-        } else if (value.length != bytesPerDim) {
-          throw new IllegalArgumentException("all dimensions must have same bytes length, but saw " + bytesPerDim + " and " + value.length);
-        }
+    this.bytesPerDim = lowerPoint[0].length;
+    for (int i = 0; i < numDims; i++) {
+      if (lowerPoint[i] == null) {
+        throw new IllegalArgumentException("lowerPoint[" + i + "] is null");
+      }
+      if (upperPoint[i] == null) {
+        throw new IllegalArgumentException("upperPoint[" + i + "] is null");
+      }
+      if (lowerPoint[i].length != bytesPerDim) {
+        throw new IllegalArgumentException("all dimensions must have same bytes length, but saw " + bytesPerDim + " and " + lowerPoint[i].length);
+      }
+      if (upperPoint[i].length != bytesPerDim) {
+        throw new IllegalArgumentException("all dimensions must have same bytes length, but saw " + bytesPerDim + " and " + upperPoint[i].length);
       }
-    }
-    if (bytesPerDim == -1) {
-      this.bytesPerDim = null;
-    } else {
-      this.bytesPerDim = bytesPerDim;
     }
   }
 
@@ -166,55 +141,18 @@ public abstract class PointRangeQuery extends Query {
         if (fieldInfo.getPointDimensionCount() != numDims) {
           throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims=" + fieldInfo.getPointDimensionCount() + " but this query has numDims=" + numDims);
         }
-        if (bytesPerDim != null && bytesPerDim.intValue() != fieldInfo.getPointNumBytes()) {
+        if (bytesPerDim != fieldInfo.getPointNumBytes()) {
           throw new IllegalArgumentException("field=\"" + field + "\" was indexed with bytesPerDim=" + fieldInfo.getPointNumBytes() + " but this query has bytesPerDim=" + bytesPerDim);
         }
         int bytesPerDim = fieldInfo.getPointNumBytes();
 
-        byte[] packedLowerIncl = new byte[numDims * bytesPerDim];
-        byte[] packedUpperIncl = new byte[numDims * bytesPerDim];
-
-        byte[] minValue = new byte[bytesPerDim];
-        byte[] maxValue = new byte[bytesPerDim];
-        Arrays.fill(maxValue, (byte) 0xff);
-
-        byte[] one = new byte[bytesPerDim];
-        one[bytesPerDim-1] = 1;
+        byte[] packedLower = new byte[numDims * bytesPerDim];
+        byte[] packedUpper = new byte[numDims * bytesPerDim];
 
-        // Carefully pack lower and upper bounds, taking care of per-dim inclusive:
+        // Carefully pack lower and upper bounds
         for(int dim=0;dim<numDims;dim++) {
-          if (lowerPoint[dim] != null) {
-            if (lowerInclusive[dim] == false) {
-              if (Arrays.equals(lowerPoint[dim], maxValue)) {
-                return null;
-              } else {
-                byte[] value = new byte[bytesPerDim];
-                NumericUtils.add(bytesPerDim, 0, lowerPoint[dim], one, value);
-                System.arraycopy(value, 0, packedLowerIncl, dim*bytesPerDim, bytesPerDim);
-              }
-            } else {
-              System.arraycopy(lowerPoint[dim], 0, packedLowerIncl, dim*bytesPerDim, bytesPerDim);
-            }
-          } else {
-            // Open-ended range: we just leave 0s in this packed dim for the lower value
-          }
-
-          if (upperPoint[dim] != null) {
-            if (upperInclusive[dim] == false) {
-              if (Arrays.equals(upperPoint[dim], minValue)) {
-                return null;
-              } else {
-                byte[] value = new byte[bytesPerDim];
-                NumericUtils.subtract(bytesPerDim, 0, upperPoint[dim], one, value);
-                System.arraycopy(value, 0, packedUpperIncl, dim*bytesPerDim, bytesPerDim);
-              }
-            } else {
-              System.arraycopy(upperPoint[dim], 0, packedUpperIncl, dim*bytesPerDim, bytesPerDim);
-            }
-          } else {
-            // Open-ended range: fill with max point for this dim:
-            System.arraycopy(maxValue, 0, packedUpperIncl, dim*bytesPerDim, bytesPerDim);
-          }
+          System.arraycopy(lowerPoint[dim], 0, packedLower, dim*bytesPerDim, bytesPerDim);
+          System.arraycopy(upperPoint[dim], 0, packedUpper, dim*bytesPerDim, bytesPerDim);
         }
 
         // Now packedLowerIncl and packedUpperIncl are inclusive, and non-empty space:
@@ -238,11 +176,11 @@ public abstract class PointRangeQuery extends Query {
                            public void visit(int docID, byte[] packedValue) {
                              for(int dim=0;dim<numDims;dim++) {
                                int offset = dim*bytesPerDim;
-                               if (StringHelper.compare(bytesPerDim, packedValue, offset, packedLowerIncl, offset) < 0) {
+                               if (StringHelper.compare(bytesPerDim, packedValue, offset, packedLower, offset) < 0) {
                                  // Doc's value is too low, in this dimension
                                  return;
                                }
-                               if (StringHelper.compare(bytesPerDim, packedValue, offset, packedUpperIncl, offset) > 0) {
+                               if (StringHelper.compare(bytesPerDim, packedValue, offset, packedUpper, offset) > 0) {
                                  // Doc's value is too high, in this dimension
                                  return;
                                }
@@ -260,13 +198,13 @@ public abstract class PointRangeQuery extends Query {
                              for(int dim=0;dim<numDims;dim++) {
                                int offset = dim*bytesPerDim;
 
-                               if (StringHelper.compare(bytesPerDim, minPackedValue, offset, packedUpperIncl, offset) > 0 ||
-                                   StringHelper.compare(bytesPerDim, maxPackedValue, offset, packedLowerIncl, offset) < 0) {
+                               if (StringHelper.compare(bytesPerDim, minPackedValue, offset, packedUpper, offset) > 0 ||
+                                   StringHelper.compare(bytesPerDim, maxPackedValue, offset, packedLower, offset) < 0) {
                                  return Relation.CELL_OUTSIDE_QUERY;
                                }
 
-                               crosses |= StringHelper.compare(bytesPerDim, minPackedValue, offset, packedLowerIncl, offset) < 0 ||
-                                 StringHelper.compare(bytesPerDim, maxPackedValue, offset, packedUpperIncl, offset) > 0;
+                               crosses |= StringHelper.compare(bytesPerDim, minPackedValue, offset, packedLower, offset) < 0 ||
+                                 StringHelper.compare(bytesPerDim, maxPackedValue, offset, packedUpper, offset) > 0;
                              }
 
                              if (crosses) {
@@ -287,8 +225,6 @@ public abstract class PointRangeQuery extends Query {
     int hash = super.hashCode();
     hash = 31 * hash + Arrays.hashCode(lowerPoint);
     hash = 31 * hash + Arrays.hashCode(upperPoint);
-    hash = 31 * hash + Arrays.hashCode(lowerInclusive);
-    hash = 31 * hash + Arrays.hashCode(upperInclusive);
     hash = 31 * hash + numDims;
     hash = 31 * hash + Objects.hashCode(bytesPerDim);
     return hash;
@@ -301,9 +237,7 @@ public abstract class PointRangeQuery extends Query {
       return q.numDims == numDims &&
         q.bytesPerDim == bytesPerDim &&
         Arrays.equals(lowerPoint, q.lowerPoint) &&
-        Arrays.equals(lowerInclusive, q.lowerInclusive) &&
-        Arrays.equals(upperPoint, q.upperPoint) &&
-        Arrays.equals(upperInclusive, q.upperInclusive);
+        Arrays.equals(upperPoint, q.upperPoint);
     }
 
     return false;
@@ -323,31 +257,11 @@ public abstract class PointRangeQuery extends Query {
         sb.append(',');
       }
 
-      if (lowerInclusive[i]) {
-        sb.append('[');
-      } else {
-        sb.append('{');
-      }
-
-      if (lowerPoint[i] == null) {
-        sb.append('*');
-      } else {
-        sb.append(toString(i, lowerPoint[i]));
-      }
-
+      sb.append('[');
+      sb.append(toString(i, lowerPoint[i]));
       sb.append(" TO ");
-
-      if (upperPoint[i] == null) {
-        sb.append('*');
-      } else {
-        sb.append(toString(i, upperPoint[i]));
-      }
-
-      if (upperInclusive[i]) {
-        sb.append(']');
-      } else {
-        sb.append('}');
-      }
+      sb.append(toString(i, upperPoint[i]));
+      sb.append(']');
     }
 
     return sb.toString();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
index 0f26551..0034cee 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
@@ -1351,7 +1351,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
         max = x;
       }
 
-      TopDocs hits = s.search(LongPoint.newRangeQuery("number", min, true, max, true), 100);
+      TopDocs hits = s.search(LongPoint.newRangeQuery("number", min, max), 100);
       for(ScoreDoc scoreDoc : hits.scoreDocs) {
         long value = Long.parseLong(s.doc(scoreDoc.doc).get("text").split(" ")[1]);
         assertTrue(value >= min);


[9/9] lucene-solr git commit: SOLR-8764: test schema-latest.xml spatial dist units should be kilometers (no test uses yet?) (cherry picked from commit deb6a49) (cherry picked from commit 6dcb01c)

Posted by sh...@apache.org.
SOLR-8764: test schema-latest.xml spatial dist units should be kilometers
(no test uses yet?)
(cherry picked from commit deb6a49)
(cherry picked from commit 6dcb01c)


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

Branch: refs/heads/branch_6_0
Commit: 84d2bb604207e6f07914fd37da5337b867c0cee9
Parents: 891e659
Author: David Smiley <ds...@apache.org>
Authored: Thu Mar 3 09:33:44 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Mar 3 20:50:02 2016 +0530

----------------------------------------------------------------------
 solr/core/src/test-files/solr/collection1/conf/schema_latest.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84d2bb60/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
index 803d45e..ef82aa1 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml
@@ -718,7 +718,7 @@
       http://wiki.apache.org/solr/SolrAdaptersForLuceneSpatial4
     -->
     <fieldType name="location_rpt" class="solr.SpatialRecursivePrefixTreeFieldType"
-        geo="true" distErrPct="0.025" maxDistErr="0.000009" />
+        geo="true" distErrPct="0.025" maxDistErr="0.001" distanceUnits="kilometers" />
 
    <!-- Money/currency field type. See http://wiki.apache.org/solr/MoneyFieldType
         Parameters:


[4/9] lucene-solr git commit: LUCENE-7059: remove MultiPointValues (cherry picked from commit 25cc48b)

Posted by sh...@apache.org.
LUCENE-7059: remove MultiPointValues
(cherry picked from commit 25cc48b)


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

Branch: refs/heads/branch_6_0
Commit: b38c2d698a70e56b2be7006f30f628452c85cfdc
Parents: b219278
Author: Mike McCandless <mi...@apache.org>
Authored: Thu Mar 3 03:49:24 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Mar 3 20:48:30 2016 +0530

----------------------------------------------------------------------
 .../apache/lucene/index/MultiPointValues.java   | 172 -------------------
 1 file changed, 172 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b38c2d69/lucene/core/src/java/org/apache/lucene/index/MultiPointValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MultiPointValues.java b/lucene/core/src/java/org/apache/lucene/index/MultiPointValues.java
deleted file mode 100644
index dcc33da..0000000
--- a/lucene/core/src/java/org/apache/lucene/index/MultiPointValues.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.index;
-
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.lucene.util.StringHelper;
-
-/** Merges multiple {@link PointValues} into a single one. */
-public class MultiPointValues extends PointValues {
-
-  private final List<PointValues> subs;
-  private final List<Integer> docBases;
-
-  private MultiPointValues(List<PointValues> subs, List<Integer> docBases) {
-    this.subs = subs;
-    this.docBases = docBases;
-  }
-
-  /** Returns a {@link PointValues} merging all point values from the provided reader. */
-  public static PointValues get(IndexReader r) {
-    final List<LeafReaderContext> leaves = r.leaves();
-    final int size = leaves.size();
-    if (size == 0) {
-      return null;
-    } else if (size == 1) {
-      return leaves.get(0).reader().getPointValues();
-    }
-
-    List<PointValues> values = new ArrayList<>();
-    List<Integer> docBases = new ArrayList<>();
-    for (int i = 0; i < size; i++) {
-      LeafReaderContext context = leaves.get(i);
-      PointValues v = context.reader().getPointValues();
-      if (v != null) {
-        values.add(v);
-        docBases.add(context.docBase);
-      }
-    }
-
-    if (values.isEmpty()) {
-      return null;
-    }
-
-    return new MultiPointValues(values, docBases);
-  }
-
-  /** Finds all documents and points matching the provided visitor */
-  public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
-    for(int i=0;i<subs.size();i++) {
-      int docBase = docBases.get(i);
-      subs.get(i).intersect(fieldName,
-                        new IntersectVisitor() {
-                          @Override
-                          public void visit(int docID) throws IOException {
-                            visitor.visit(docBase+docID);
-                          }
-                          @Override
-                          public void visit(int docID, byte[] packedValue) throws IOException {
-                            visitor.visit(docBase+docID, packedValue);
-                          }
-                          @Override
-                          public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-                            return visitor.compare(minPackedValue, maxPackedValue);
-                          }
-                        });
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder b = new StringBuilder();
-    b.append("MultiPointValues(");
-    for(int i=0;i<subs.size();i++) {
-      if (i > 0) {
-        b.append(", ");
-      }
-      b.append("docBase=");
-      b.append(docBases.get(i));
-      b.append(" sub=" + subs.get(i));
-    }
-    b.append(')');
-    return b.toString();
-  }
-
-  @Override
-  public byte[] getMinPackedValue(String fieldName) throws IOException {
-    byte[] result = null;
-    for(int i=0;i<subs.size();i++) {
-      byte[] minPackedValue = subs.get(i).getMinPackedValue(fieldName);
-      if (result == null) {
-        if (minPackedValue != null) {
-          result = minPackedValue.clone();
-        }
-      } else {
-        int numDims = subs.get(0).getNumDimensions(fieldName);
-        int bytesPerDim = subs.get(0).getBytesPerDimension(fieldName);
-        for(int dim=0;dim<numDims;dim++) {
-          int offset = dim*bytesPerDim;
-          if (StringHelper.compare(bytesPerDim, minPackedValue, offset, result, offset) < 0) {
-            System.arraycopy(minPackedValue, offset, result, offset, bytesPerDim);
-          }
-        }
-      }
-    }
-
-    return result;
-  }
-
-  @Override
-  public byte[] getMaxPackedValue(String fieldName) throws IOException {
-    byte[] result = null;
-    for(int i=0;i<subs.size();i++) {
-      byte[] maxPackedValue = subs.get(i).getMaxPackedValue(fieldName);
-      if (result == null) {
-        if (maxPackedValue != null) {
-          result = maxPackedValue.clone();
-        }
-      } else {
-        int numDims = subs.get(0).getNumDimensions(fieldName);
-        int bytesPerDim = subs.get(0).getBytesPerDimension(fieldName);
-        for(int dim=0;dim<numDims;dim++) {
-          int offset = dim*bytesPerDim;
-          if (StringHelper.compare(bytesPerDim, maxPackedValue, offset, result, offset) > 0) {
-            System.arraycopy(maxPackedValue, offset, result, offset, bytesPerDim);
-          }
-        }
-      }
-    }
-
-    return result;
-  }
-
-  @Override
-  public int getNumDimensions(String fieldName) throws IOException {
-    for(int i=0;i<subs.size();i++) {
-      int result = subs.get(i).getNumDimensions(fieldName);
-      if (result != 0) {
-        return result;
-      }
-    }
-    return 0;
-  }
-
-  @Override
-  public int getBytesPerDimension(String fieldName) throws IOException {
-    for(int i=0;i<subs.size();i++) {
-      int result = subs.get(i).getBytesPerDimension(fieldName);
-      if (result != 0) {
-        return result;
-      }
-    }
-    return 0;
-  }
-}


[5/9] lucene-solr git commit: LUCENE-7061: fix remaining api issues with XYZPoint classes

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
index fb31792..19096c7 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -63,7 +63,6 @@ import org.apache.lucene.index.SlowCompositeReaderWrapper;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.NumericUtils;
@@ -118,8 +117,8 @@ public class TestPointQueries extends LuceneTestCase {
 
     DirectoryReader r = DirectoryReader.open(w);
     IndexSearcher s = new IndexSearcher(r);
-    assertEquals(2, s.count(IntPoint.newRangeQuery("point", -8, false, 1, false)));
-    assertEquals(3, s.count(IntPoint.newRangeQuery("point", -7, true, 3, true)));
+    assertEquals(2, s.count(IntPoint.newRangeQuery("point", -8, 1)));
+    assertEquals(3, s.count(IntPoint.newRangeQuery("point", -7, 3)));
     assertEquals(1, s.count(IntPoint.newExactQuery("point", -7)));
     assertEquals(0, s.count(IntPoint.newExactQuery("point", -6)));
     w.close();
@@ -145,8 +144,8 @@ public class TestPointQueries extends LuceneTestCase {
 
     DirectoryReader r = DirectoryReader.open(w);
     IndexSearcher s = new IndexSearcher(r);
-    assertEquals(2, s.count(FloatPoint.newRangeQuery("point", -8.0f, false, 1.0f, false)));
-    assertEquals(3, s.count(FloatPoint.newRangeQuery("point", -7.0f, true, 3.0f, true)));
+    assertEquals(2, s.count(FloatPoint.newRangeQuery("point", -8.0f, 1.0f)));
+    assertEquals(3, s.count(FloatPoint.newRangeQuery("point", -7.0f, 3.0f)));
     assertEquals(1, s.count(FloatPoint.newExactQuery("point", -7.0f)));
     assertEquals(0, s.count(FloatPoint.newExactQuery("point", -6.0f)));
     w.close();
@@ -172,8 +171,8 @@ public class TestPointQueries extends LuceneTestCase {
 
     DirectoryReader r = DirectoryReader.open(w);
     IndexSearcher s = new IndexSearcher(r);
-    assertEquals(2, s.count(LongPoint.newRangeQuery("point", -8L, false, 1L, false)));
-    assertEquals(3, s.count(LongPoint.newRangeQuery("point", -7L, true, 3L, true)));
+    assertEquals(2, s.count(LongPoint.newRangeQuery("point", -8L, 1L)));
+    assertEquals(3, s.count(LongPoint.newRangeQuery("point", -7L, 3L)));
     assertEquals(1, s.count(LongPoint.newExactQuery("point", -7L)));
     assertEquals(0, s.count(LongPoint.newExactQuery("point", -6L)));
     w.close();
@@ -199,18 +198,140 @@ public class TestPointQueries extends LuceneTestCase {
 
     DirectoryReader r = DirectoryReader.open(w);
     IndexSearcher s = new IndexSearcher(r);
-    assertEquals(2, s.count(DoublePoint.newRangeQuery("point", -8.0, false, 1.0, false)));
-    assertEquals(3, s.count(DoublePoint.newRangeQuery("point", -7.0, true, 3.0, true)));
+    assertEquals(2, s.count(DoublePoint.newRangeQuery("point", -8.0, 1.0)));
+    assertEquals(3, s.count(DoublePoint.newRangeQuery("point", -7.0, 3.0)));
     assertEquals(1, s.count(DoublePoint.newExactQuery("point", -7.0)));
     assertEquals(0, s.count(DoublePoint.newExactQuery("point", -6.0)));
     w.close();
     r.close();
     dir.close();
   }
+  
+  public void testCrazyDoubles() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())));
+
+    Document doc = new Document();
+    doc.add(new DoublePoint("point", Double.NEGATIVE_INFINITY));
+    w.addDocument(doc);
+    
+    doc = new Document();
+    doc.add(new DoublePoint("point", -0.0D));
+    w.addDocument(doc);
+    
+    doc = new Document();
+    doc.add(new DoublePoint("point", +0.0D));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new DoublePoint("point", Double.MIN_VALUE));
+    w.addDocument(doc);
+    
+    doc = new Document();
+    doc.add(new DoublePoint("point", Double.MAX_VALUE));
+    w.addDocument(doc);
+    
+    doc = new Document();
+    doc.add(new DoublePoint("point", Double.POSITIVE_INFINITY));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new DoublePoint("point", Double.NaN));
+    w.addDocument(doc);
+
+    DirectoryReader r = DirectoryReader.open(w);
+    IndexSearcher s = new IndexSearcher(r);
+    
+    // exact queries
+    assertEquals(1, s.count(DoublePoint.newExactQuery("point", Double.NEGATIVE_INFINITY)));
+    assertEquals(1, s.count(DoublePoint.newExactQuery("point", -0.0D)));
+    assertEquals(1, s.count(DoublePoint.newExactQuery("point", +0.0D)));
+    assertEquals(1, s.count(DoublePoint.newExactQuery("point", Double.MIN_VALUE)));
+    assertEquals(1, s.count(DoublePoint.newExactQuery("point", Double.MAX_VALUE)));
+    assertEquals(1, s.count(DoublePoint.newExactQuery("point", Double.POSITIVE_INFINITY)));
+    assertEquals(1, s.count(DoublePoint.newExactQuery("point", Double.NaN)));
+    
+    // set query
+    double set[] = new double[] { Double.MAX_VALUE, Double.NaN, +0.0D, Double.NEGATIVE_INFINITY, Double.MIN_VALUE, -0.0D, Double.POSITIVE_INFINITY };
+    assertEquals(7, s.count(DoublePoint.newSetQuery("point", set)));
+
+    // ranges
+    assertEquals(2, s.count(DoublePoint.newRangeQuery("point", Double.NEGATIVE_INFINITY, -0.0D)));
+    assertEquals(2, s.count(DoublePoint.newRangeQuery("point", -0.0D, 0.0D)));
+    assertEquals(2, s.count(DoublePoint.newRangeQuery("point", 0.0D, Double.MIN_VALUE)));
+    assertEquals(2, s.count(DoublePoint.newRangeQuery("point", Double.MIN_VALUE, Double.MAX_VALUE)));
+    assertEquals(2, s.count(DoublePoint.newRangeQuery("point", Double.MAX_VALUE, Double.POSITIVE_INFINITY)));
+    assertEquals(2, s.count(DoublePoint.newRangeQuery("point", Double.POSITIVE_INFINITY, Double.NaN)));
+
+    w.close();
+    r.close();
+    dir.close();
+  }
+  
+  public void testCrazyFloats() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())));
+
+    Document doc = new Document();
+    doc.add(new FloatPoint("point", Float.NEGATIVE_INFINITY));
+    w.addDocument(doc);
+    
+    doc = new Document();
+    doc.add(new FloatPoint("point", -0.0F));
+    w.addDocument(doc);
+    
+    doc = new Document();
+    doc.add(new FloatPoint("point", +0.0F));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new FloatPoint("point", Float.MIN_VALUE));
+    w.addDocument(doc);
+    
+    doc = new Document();
+    doc.add(new FloatPoint("point", Float.MAX_VALUE));
+    w.addDocument(doc);
+    
+    doc = new Document();
+    doc.add(new FloatPoint("point", Float.POSITIVE_INFINITY));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new FloatPoint("point", Float.NaN));
+    w.addDocument(doc);
+
+    DirectoryReader r = DirectoryReader.open(w);
+    IndexSearcher s = new IndexSearcher(r);
+    
+    // exact queries
+    assertEquals(1, s.count(FloatPoint.newExactQuery("point", Float.NEGATIVE_INFINITY)));
+    assertEquals(1, s.count(FloatPoint.newExactQuery("point", -0.0F)));
+    assertEquals(1, s.count(FloatPoint.newExactQuery("point", +0.0F)));
+    assertEquals(1, s.count(FloatPoint.newExactQuery("point", Float.MIN_VALUE)));
+    assertEquals(1, s.count(FloatPoint.newExactQuery("point", Float.MAX_VALUE)));
+    assertEquals(1, s.count(FloatPoint.newExactQuery("point", Float.POSITIVE_INFINITY)));
+    assertEquals(1, s.count(FloatPoint.newExactQuery("point", Float.NaN)));
+    
+    // set query
+    float set[] = new float[] { Float.MAX_VALUE, Float.NaN, +0.0F, Float.NEGATIVE_INFINITY, Float.MIN_VALUE, -0.0F, Float.POSITIVE_INFINITY };
+    assertEquals(7, s.count(FloatPoint.newSetQuery("point", set)));
+
+    // ranges
+    assertEquals(2, s.count(FloatPoint.newRangeQuery("point", Float.NEGATIVE_INFINITY, -0.0F)));
+    assertEquals(2, s.count(FloatPoint.newRangeQuery("point", -0.0F, 0.0F)));
+    assertEquals(2, s.count(FloatPoint.newRangeQuery("point", 0.0F, Float.MIN_VALUE)));
+    assertEquals(2, s.count(FloatPoint.newRangeQuery("point", Float.MIN_VALUE, Float.MAX_VALUE)));
+    assertEquals(2, s.count(FloatPoint.newRangeQuery("point", Float.MAX_VALUE, Float.POSITIVE_INFINITY)));
+    assertEquals(2, s.count(FloatPoint.newRangeQuery("point", Float.POSITIVE_INFINITY, Float.NaN)));
+
+    w.close();
+    r.close();
+    dir.close();
+  }
 
   public void testAllEqual() throws Exception {
     int numValues = atLeast(10000);
-    long value = randomValue(false);
+    long value = randomValue();
     long[] values = new long[numValues];
 
     if (VERBOSE) {
@@ -256,7 +377,7 @@ public class TestPointQueries extends LuceneTestCase {
         // Identical to old value
         values[ord] = values[random().nextInt(ord)];
       } else {
-        values[ord] = randomValue(false);
+        values[ord] = randomValue();
       }
 
       ids[ord] = id;
@@ -393,21 +514,19 @@ public class TestPointQueries extends LuceneTestCase {
 
             NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
             for (int iter=0;iter<iters && failed.get() == false;iter++) {
-              Long lower = randomValue(true);
-              Long upper = randomValue(true);
+              Long lower = randomValue();
+              Long upper = randomValue();
 
-              if (lower != null && upper != null && upper < lower) {
+              if (upper < lower) {
                 long x = lower;
                 lower = upper;
                 upper = x;
               }
 
-              boolean includeLower = random().nextBoolean();
-              boolean includeUpper = random().nextBoolean();
               Query query;
 
               if (VERBOSE) {
-                System.out.println("\n" + Thread.currentThread().getName() + ": TEST: iter=" + iter + " value=" + lower + " (inclusive?=" + includeLower + ") TO " + upper + " (inclusive?=" + includeUpper + ")");
+                System.out.println("\n" + Thread.currentThread().getName() + ": TEST: iter=" + iter + " value=" + lower + " TO " + upper);
                 byte[] tmp = new byte[8];
                 if (lower != null) {
                   NumericUtils.longToBytes(lower, tmp, 0);
@@ -420,23 +539,13 @@ public class TestPointQueries extends LuceneTestCase {
               }
 
               if (random().nextBoolean()) {
-                query = LongPoint.newRangeQuery("sn_value", lower, includeLower, upper, includeUpper);
+                query = LongPoint.newRangeQuery("sn_value", lower, upper);
               } else {
-                byte[] lowerBytes;
-                if (lower == null) {
-                  lowerBytes = null;
-                } else {
-                  lowerBytes = new byte[8];
-                  NumericUtils.longToBytes(lower, lowerBytes, 0);
-                }
-                byte[] upperBytes;
-                if (upper == null) {
-                  upperBytes = null;
-                } else {
-                  upperBytes = new byte[8];
-                  NumericUtils.longToBytes(upper, upperBytes, 0);
-                }
-                query = BinaryPoint.newRangeQuery("ss_value", lowerBytes, includeLower, upperBytes, includeUpper);
+                byte[] lowerBytes = new byte[8];
+                NumericUtils.longToBytes(lower, lowerBytes, 0);
+                byte[] upperBytes = new byte[8];
+                NumericUtils.longToBytes(upper, upperBytes, 0);
+                query = BinaryPoint.newRangeQuery("ss_value", lowerBytes, upperBytes);
               }
 
               if (VERBOSE) {
@@ -470,7 +579,7 @@ public class TestPointQueries extends LuceneTestCase {
       
               for(int docID=0;docID<r.maxDoc();docID++) {
                 int id = (int) docIDToID.get(docID);
-                boolean expected = missing.get(id) == false && deleted.get(id) == false && matches(lower, includeLower, upper, includeUpper, values[id]);
+                boolean expected = missing.get(id) == false && deleted.get(id) == false && values[id] >= lower && values[id] <= upper;
                 if (hits.get(docID) != expected) {
                   // We do exact quantized comparison so the bbox query should never disagree:
                   fail(Thread.currentThread().getName() + ": iter=" + iter + " id=" + id + " docID=" + docID + " value=" + values[id] + " (range: " + lower + " TO " + upper + ") expected " + expected + " but got: " + hits.get(docID) + " deleted?=" + deleted.get(id) + " query=" + query);
@@ -662,32 +771,20 @@ public class TestPointQueries extends LuceneTestCase {
 
             for (int iter=0;iter<iters && failed.get() == false;iter++) {
 
-              boolean[] includeUpper = new boolean[numDims];
-              boolean[] includeLower = new boolean[numDims];
               byte[][] lower = new byte[numDims][];
               byte[][] upper = new byte[numDims][];
               for(int dim=0;dim<numDims;dim++) {
-                if (random().nextInt(5) != 1) {
-                  lower[dim] = new byte[bytesPerDim];
-                  random().nextBytes(lower[dim]);
-                } else {
-                  // open-ended on the lower bound
-                }
-                if (random().nextInt(5) != 1) {
-                  upper[dim] = new byte[bytesPerDim];
-                  random().nextBytes(upper[dim]);
-                } else {
-                  // open-ended on the upper bound
-                }
+                lower[dim] = new byte[bytesPerDim];
+                random().nextBytes(lower[dim]);
+
+                upper[dim] = new byte[bytesPerDim];
+                random().nextBytes(upper[dim]);
 
-                if (lower[dim] != null && upper[dim] != null && StringHelper.compare(bytesPerDim, lower[dim], 0, upper[dim], 0) > 0) {
+                if (StringHelper.compare(bytesPerDim, lower[dim], 0, upper[dim], 0) > 0) {
                   byte[] x = lower[dim];
                   lower[dim] = upper[dim];
                   upper[dim] = x;
                 }
-
-                includeLower[dim] = random().nextBoolean();
-                includeUpper[dim] = random().nextBoolean();
               }
 
               if (VERBOSE) {
@@ -695,13 +792,12 @@ public class TestPointQueries extends LuceneTestCase {
                 for(int dim=0;dim<numDims;dim++) {
                   System.out.println("  dim=" + dim + " " +
                                      bytesToString(lower[dim]) +
-                                     " (inclusive?=" + includeLower[dim] + ") TO " +
-                                     bytesToString(upper[dim]) +
-                                     " (inclusive?=" + includeUpper[dim] + ")");
+                                     " TO " +
+                                     bytesToString(upper[dim]));
                 }
               }
 
-              Query query = BinaryPoint.newMultiRangeQuery("value", lower, includeLower, upper, includeUpper);
+              Query query = BinaryPoint.newRangeQuery("value", lower, upper);
 
               if (VERBOSE) {
                 System.out.println(Thread.currentThread().getName() + ":  using query: " + query);
@@ -735,7 +831,7 @@ public class TestPointQueries extends LuceneTestCase {
               BitSet expected = new BitSet();
               for(int ord=0;ord<numValues;ord++) {
                 int id = ids[ord];
-                if (missing.get(id) == false && deleted.get(id) == false && matches(bytesPerDim, lower, includeLower, upper, includeUpper, docValues[ord])) {
+                if (missing.get(id) == false && deleted.get(id) == false && matches(bytesPerDim, lower, upper, docValues[ord])) {
                   expected.set(id);
                 }
               }
@@ -746,7 +842,7 @@ public class TestPointQueries extends LuceneTestCase {
                 if (hits.get(docID) != expected.get(id)) {
                   System.out.println("FAIL: iter=" + iter + " id=" + id + " docID=" + docID + " expected=" + expected.get(id) + " but got " + hits.get(docID) + " deleted?=" + deleted.get(id) + " missing?=" + missing.get(id));
                   for(int dim=0;dim<numDims;dim++) {
-                    System.out.println("  dim=" + dim + " range: " + bytesToString(lower[dim]) + " (inclusive?=" + includeLower[dim] + ") TO " + bytesToString(upper[dim]) + " (inclusive?=" + includeUpper[dim] + ")");
+                    System.out.println("  dim=" + dim + " range: " + bytesToString(lower[dim]) + " TO " + bytesToString(upper[dim]));
                     failCount++;
                   }
                 }
@@ -768,23 +864,6 @@ public class TestPointQueries extends LuceneTestCase {
     IOUtils.close(r, dir);
   }
 
-  private static boolean matches(Long lower, boolean includeLower, Long upper, boolean includeUpper, long value) {
-    if (includeLower == false && lower != null) {
-      if (lower == Long.MAX_VALUE) {
-        return false;
-      }
-      lower++;
-    }
-    if (includeUpper == false && upper != null) {
-      if (upper == Long.MIN_VALUE) {
-        return false;
-      }
-      upper--;
-    }
-
-    return (lower == null || value >= lower) && (upper == null || value <= upper);
-  }
-
   static String bytesToString(byte[] bytes) {
     if (bytes == null) {
       return "null";
@@ -792,28 +871,16 @@ public class TestPointQueries extends LuceneTestCase {
     return new BytesRef(bytes).toString();
   }
 
-  private static boolean matches(int bytesPerDim, byte[][] lower, boolean[] includeLower, byte[][] upper, boolean[] includeUpper, byte[][] value) {
+  private static boolean matches(int bytesPerDim, byte[][] lower, byte[][] upper, byte[][] value) {
     int numDims = lower.length;
     for(int dim=0;dim<numDims;dim++) {
-      int cmp;
-      if (lower[dim] == null) {
-        cmp = 1;
-      } else {
-        cmp = StringHelper.compare(bytesPerDim, value[dim], 0, lower[dim], 0);
-      }
 
-      if (cmp < 0 || (cmp == 0 && includeLower[dim] == false)) {
+      if (StringHelper.compare(bytesPerDim, value[dim], 0, lower[dim], 0) < 0) {
         // Value is below the lower bound, on this dim
         return false;
       }
 
-      if (upper[dim] == null) {
-        cmp = -1;
-      } else {
-        cmp = StringHelper.compare(bytesPerDim, value[dim], 0, upper[dim], 0);
-      }
-
-      if (cmp > 0 || (cmp == 0 && includeUpper[dim] == false)) {
+      if (StringHelper.compare(bytesPerDim, value[dim], 0, upper[dim], 0) > 0) {
         // Value is above the upper bound, on this dim
         return false;
       }
@@ -822,13 +889,9 @@ public class TestPointQueries extends LuceneTestCase {
     return true;
   }
 
-  private static Long randomValue(boolean allowNull) {
+  private static long randomValue() {
     if (valueRange == 0) {
-      if (allowNull && random().nextInt(10) == 1) {
-        return null;
-      } else {
-        return random().nextLong();
-      }
+      return random().nextLong();
     } else {
       return valueMid + TestUtil.nextInt(random(), -valueRange, valueRange);
     }
@@ -851,9 +914,9 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexSearcher s = newSearcher(r, false);
 
-    assertEquals(1, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, true, 0L, true)));
-    assertEquals(1, s.count(LongPoint.newRangeQuery("value", 0L, true, Long.MAX_VALUE, true)));
-    assertEquals(2, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
+    assertEquals(1, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, 0L)));
+    assertEquals(1, s.count(LongPoint.newRangeQuery("value", 0L, Long.MAX_VALUE)));
+    assertEquals(2, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, Long.MAX_VALUE)));
 
     IOUtils.close(r, w, dir);
   }
@@ -889,51 +952,15 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexSearcher s = newSearcher(r, false);
 
-    assertEquals(1, s.count(BinaryPoint.newRangeQuery("value",
-        toUTF8("aaa"),
-        true,
-        toUTF8("bbb"),
-        true)));
-    assertEquals(1, s.count(BinaryPoint.newRangeQuery("value",
-        toUTF8("c", 3),
-        true,
-        toUTF8("e", 3),
-        true)));
-    assertEquals(2, s.count(BinaryPoint.newRangeQuery("value",
-        toUTF8("a", 3),
-        true,
-        toUTF8("z", 3),
-        true)));
-    assertEquals(1, s.count(BinaryPoint.newRangeQuery("value",
-        null,
-        true,
-        toUTF8("abc"),
-        true)));
-    assertEquals(1, s.count(BinaryPoint.newRangeQuery("value",
-        toUTF8("a", 3),
-        true,
-        toUTF8("abc"),
-        true)));
-    assertEquals(0, s.count(BinaryPoint.newRangeQuery("value",
-        toUTF8("a", 3),
-        true,
-        toUTF8("abc"),
-        false)));
-    assertEquals(1, s.count(BinaryPoint.newRangeQuery("value",
-        toUTF8("def"),
-        true,
-        null,
-        false)));
-    assertEquals(1, s.count(BinaryPoint.newRangeQuery("value",
-        toUTF8(("def")),
-        true,
-        toUTF8("z", 3),
-        true)));
-    assertEquals(0, s.count(BinaryPoint.newRangeQuery("value",
-        toUTF8("def"),
-        false,
-        toUTF8("z", 3),
-        true)));
+    assertEquals(1, s.count(BinaryPoint.newRangeQuery("value", toUTF8("aaa"), toUTF8("bbb"))));
+    assertEquals(1, s.count(BinaryPoint.newRangeQuery("value", toUTF8("c", 3), toUTF8("e", 3))));
+    assertEquals(2, s.count(BinaryPoint.newRangeQuery("value", toUTF8("a", 3), toUTF8("z", 3))));
+    assertEquals(1, s.count(BinaryPoint.newRangeQuery("value", toUTF8("", 3), toUTF8("abc"))));
+    assertEquals(1, s.count(BinaryPoint.newRangeQuery("value", toUTF8("a", 3), toUTF8("abc"))));
+    assertEquals(0, s.count(BinaryPoint.newRangeQuery("value", toUTF8("a", 3), toUTF8("abb"))));
+    assertEquals(1, s.count(BinaryPoint.newRangeQuery("value", toUTF8("def"), toUTF8("zzz"))));
+    assertEquals(1, s.count(BinaryPoint.newRangeQuery("value", toUTF8(("def")), toUTF8("z", 3))));
+    assertEquals(0, s.count(BinaryPoint.newRangeQuery("value", toUTF8("deg"), toUTF8("z", 3))));
 
     IOUtils.close(r, w, dir);
   }
@@ -954,12 +981,10 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexSearcher s = newSearcher(r, false);
 
-    assertEquals(2, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
-    assertEquals(1, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, true, Long.MAX_VALUE, false)));
-    assertEquals(1, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, false, Long.MAX_VALUE, true)));
-    assertEquals(0, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, false, Long.MAX_VALUE, false)));
-
-    assertEquals(2, s.count(BinaryPoint.newRangeQuery("value", (byte[]) null, true, null, true)));
+    assertEquals(2, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, Long.MAX_VALUE)));
+    assertEquals(1, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, Long.MAX_VALUE-1)));
+    assertEquals(1, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE+1, Long.MAX_VALUE)));
+    assertEquals(0, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE+1, Long.MAX_VALUE-1)));
 
     IOUtils.close(r, w, dir);
   }
@@ -980,12 +1005,10 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexSearcher s = newSearcher(r, false);
 
-    assertEquals(2, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
-    assertEquals(1, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, true, Long.MAX_VALUE, false)));
-    assertEquals(1, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, false, Long.MAX_VALUE, true)));
-    assertEquals(0, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, false, Long.MAX_VALUE, false)));
-
-    assertEquals(2, s.count(LongPoint.newRangeQuery("value", (Long) null, true, null, true)));
+    assertEquals(2, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, Long.MAX_VALUE)));
+    assertEquals(1, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE, Long.MAX_VALUE-1)));
+    assertEquals(1, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE+1, Long.MAX_VALUE)));
+    assertEquals(0, s.count(LongPoint.newRangeQuery("value", Long.MIN_VALUE+1, Long.MAX_VALUE-1)));
 
     IOUtils.close(r, w, dir);
   }
@@ -1004,10 +1027,8 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = w.getReader();
 
-    IndexSearcher s = newSearcher(r, false);
-    assertEquals(0, s.count(BinaryPoint.newRangeQuery("value", toUTF8("m"), true, toUTF8("n"), false)));
-
-    assertEquals(2, s.count(BinaryPoint.newRangeQuery("value", (byte[]) null, true, null, true)));
+    IndexSearcher s = newSearcher(r,false);
+    assertEquals(0, s.count(BinaryPoint.newRangeQuery("value", toUTF8("m"), toUTF8("m"))));
 
     IOUtils.close(r, w, dir);
   }
@@ -1027,7 +1048,7 @@ public class TestPointQueries extends LuceneTestCase {
     IndexReader r = w.getReader();
 
     IndexSearcher s = new IndexSearcher(r);
-    assertEquals(0, s.count(LongPoint.newRangeQuery("value", 17L, true, 13L, false)));
+    assertEquals(0, s.count(LongPoint.newRangeQuery("value", 17L, 13L)));
 
     IOUtils.close(r, w, dir);
   }
@@ -1042,7 +1063,7 @@ public class TestPointQueries extends LuceneTestCase {
     IndexReader r = w.getReader();
 
     IndexSearcher s = newSearcher(r, false);
-    assertEquals(0, s.count(LongPoint.newRangeQuery("value", 17L, true, 13L, false)));
+    assertEquals(0, s.count(LongPoint.newRangeQuery("value", 17L, 13L)));
 
     IOUtils.close(r, w, dir);
   }
@@ -1061,41 +1082,16 @@ public class TestPointQueries extends LuceneTestCase {
     // no wrapping, else the exc might happen in executor thread:
     IndexSearcher s = new IndexSearcher(r);
     byte[][] point = new byte[2][];
+    point[0] = new byte[8];
+    point[1] = new byte[8];
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
-      s.count(BinaryPoint.newMultiRangeQuery("value", point, new boolean[] {true, true}, point, new boolean[] {true, true}));
+      s.count(BinaryPoint.newRangeQuery("value", point, point));
     });
     assertEquals("field=\"value\" was indexed with numDims=1 but this query has numDims=2", expected.getMessage());
 
     IOUtils.close(r, w, dir);
   }
 
-  /** ensure good exception when boolean[]s for inclusive have wrong length */
-  public void testWrongNumBooleans() throws Exception {
-    Directory dir = newDirectory();
-    IndexWriterConfig iwc = newIndexWriterConfig();
-    iwc.setCodec(getCodec());
-    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
-    Document doc = new Document();
-    doc.add(new LongPoint("value", 1L, 2L));
-    w.addDocument(doc);
-
-    IndexReader r = w.getReader();
-
-    // no wrapping, else the exc might happen in executor thread:
-    IndexSearcher s = new IndexSearcher(r);
-    IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
-      s.count(LongPoint.newMultiRangeQuery("value", new Long[] { 1L, 2L }, new boolean[] {true}, new Long[] { 1L, 2L }, new boolean[] {true, true}));
-    });
-    assertEquals("lowerInclusive has length=1 but expected=2", expected.getMessage());
-
-    expected = expectThrows(IllegalArgumentException.class, () -> {
-      s.count(LongPoint.newMultiRangeQuery("value", new Long[] { 1L, 2L }, new boolean[] {true, true}, new Long[] { 1L, 2L }, new boolean[] {true}));
-    });
-    assertEquals("upperInclusive has length=1 but expected=2", expected.getMessage());
-
-    IOUtils.close(r, w, dir);
-  }
-
   public void testWrongNumBytes() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
@@ -1112,7 +1108,7 @@ public class TestPointQueries extends LuceneTestCase {
     byte[][] point = new byte[1][];
     point[0] = new byte[10];
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
-      s.count(BinaryPoint.newMultiRangeQuery("value", point, new boolean[] {true}, point, new boolean[] {true}));
+      s.count(BinaryPoint.newRangeQuery("value", point, point));
     });
     assertEquals("field=\"value\" was indexed with bytesPerDim=8 but this query has bytesPerDim=10", expected.getMessage());
 
@@ -1228,31 +1224,25 @@ public class TestPointQueries extends LuceneTestCase {
   public void testToString() throws Exception {
     
     // ints
-    assertEquals("field:[1 TO 2}", IntPoint.newRangeQuery("field", 1, true, 2, false).toString());
-    assertEquals("field:{-2 TO 1]", IntPoint.newRangeQuery("field", -2, false, 1, true).toString());
-    assertEquals("field:[* TO 2}", IntPoint.newRangeQuery("field", null, true, 2, false).toString());
+    assertEquals("field:[1 TO 2]", IntPoint.newRangeQuery("field", 1, 2).toString());
+    assertEquals("field:[-2 TO 1]", IntPoint.newRangeQuery("field", -2, 1).toString());
 
     // longs
-    assertEquals("field:[1099511627776 TO 2199023255552}", LongPoint.newRangeQuery("field", 1L<<40, true, 1L<<41, false).toString());
-    assertEquals("field:{-5 TO 6]", LongPoint.newRangeQuery("field", -5L, false, 6L, true).toString());
-    assertEquals("field:[* TO 2}", LongPoint.newRangeQuery("field", null, true, 2L, false).toString());
+    assertEquals("field:[1099511627776 TO 2199023255552]", LongPoint.newRangeQuery("field", 1L<<40, 1L<<41).toString());
+    assertEquals("field:[-5 TO 6]", LongPoint.newRangeQuery("field", -5L, 6L).toString());
     
     // floats
-    assertEquals("field:[1.3 TO 2.5}", FloatPoint.newRangeQuery("field", 1.3F, true, 2.5F, false).toString());
-    assertEquals("field:{-2.9 TO 1.0]", FloatPoint.newRangeQuery("field", -2.9F, false, 1.0F, true).toString());
-    assertEquals("field:{-2.9 TO *]", FloatPoint.newRangeQuery("field", -2.9F, false, null, true).toString());
+    assertEquals("field:[1.3 TO 2.5]", FloatPoint.newRangeQuery("field", 1.3F, 2.5F).toString());
+    assertEquals("field:[-2.9 TO 1.0]", FloatPoint.newRangeQuery("field", -2.9F, 1.0F).toString());
     
     // doubles
-    assertEquals("field:[1.3 TO 2.5}", DoublePoint.newRangeQuery("field", 1.3, true, 2.5, false).toString());
-    assertEquals("field:{-2.9 TO 1.0]", DoublePoint.newRangeQuery("field", -2.9, false, 1.0, true).toString());
-    assertEquals("field:{-2.9 TO *]", DoublePoint.newRangeQuery("field", -2.9, false, null, true).toString());
+    assertEquals("field:[1.3 TO 2.5]", DoublePoint.newRangeQuery("field", 1.3, 2.5).toString());
+    assertEquals("field:[-2.9 TO 1.0]", DoublePoint.newRangeQuery("field", -2.9, 1.0).toString());
     
     // n-dimensional double
-    assertEquals("field:[1.3 TO 2.5},{-2.9 TO 1.0]", DoublePoint.newMultiRangeQuery("field", 
-                                                                      new Double[] { 1.3, -2.9 }, 
-                                                                      new boolean[] { true, false }, 
-                                                                      new Double[] { 2.5, 1.0 },
-                                                                      new boolean[] { false, true }).toString());
+    assertEquals("field:[1.3 TO 2.5],[-2.9 TO 1.0]", DoublePoint.newRangeQuery("field", 
+                                                                      new double[] { 1.3, -2.9 }, 
+                                                                      new double[] { 2.5, 1.0 }).toString());
 
   }
 
@@ -1430,7 +1420,7 @@ public class TestPointQueries extends LuceneTestCase {
     return new PointInSetQuery(field,
                                numDims,
                                Integer.BYTES,
-                               new BytesRefIterator() {
+                               new PointInSetQuery.Stream() {
                                  int upto;
                                  @Override
                                  public BytesRef next() {
@@ -1818,7 +1808,7 @@ public class TestPointQueries extends LuceneTestCase {
     IllegalArgumentException expected = expectThrows(IllegalArgumentException.class,
                                                      () -> {
                                                        new PointInSetQuery("foo", 3, 4,
-                                                                           new BytesRefIterator() {
+                                                                           new PointInSetQuery.Stream() {
                                                                              @Override
                                                                              public BytesRef next() {
                                                                                return new BytesRef(new byte[3]);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java b/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
index 2edebb3..9c7ada8 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
@@ -26,7 +26,7 @@ public class TestUsageTrackingFilterCachingPolicy extends LuceneTestCase {
 
   public void testCostlyFilter() {
     assertTrue(UsageTrackingQueryCachingPolicy.isCostly(new PrefixQuery(new Term("field", "prefix"))));
-    assertTrue(UsageTrackingQueryCachingPolicy.isCostly(IntPoint.newRangeQuery("intField", 1, true, 1000, true)));
+    assertTrue(UsageTrackingQueryCachingPolicy.isCostly(IntPoint.newRangeQuery("intField", 1, 1000)));
     assertFalse(UsageTrackingQueryCachingPolicy.isCostly(new TermQuery(new Term("field", "value"))));
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
----------------------------------------------------------------------
diff --git a/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java b/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
index d9e9441..fff3bff 100644
--- a/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
+++ b/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
@@ -179,7 +179,7 @@ public class DistanceFacetsExample implements Closeable {
     BooleanQuery.Builder f = new BooleanQuery.Builder();
 
     // Add latitude range filter:
-    f.add(DoublePoint.newRangeQuery("latitude", Math.toDegrees(minLat), true, Math.toDegrees(maxLat), true),
+    f.add(DoublePoint.newRangeQuery("latitude", Math.toDegrees(minLat), Math.toDegrees(maxLat)),
           BooleanClause.Occur.FILTER);
 
     // Add longitude range filter:
@@ -187,13 +187,13 @@ public class DistanceFacetsExample implements Closeable {
       // The bounding box crosses the international date
       // line:
       BooleanQuery.Builder lonF = new BooleanQuery.Builder();
-      lonF.add(DoublePoint.newRangeQuery("longitude", Math.toDegrees(minLng), true, null, true),
+      lonF.add(DoublePoint.newRangeQuery("longitude", Math.toDegrees(minLng), Double.POSITIVE_INFINITY),
                BooleanClause.Occur.SHOULD);
-      lonF.add(DoublePoint.newRangeQuery("longitude", null, true, Math.toDegrees(maxLng), true),
+      lonF.add(DoublePoint.newRangeQuery("longitude", Double.NEGATIVE_INFINITY, Math.toDegrees(maxLng)),
                BooleanClause.Occur.SHOULD);
       f.add(lonF.build(), BooleanClause.Occur.MUST);
     } else {
-      f.add(DoublePoint.newRangeQuery("longitude", Math.toDegrees(minLng), true, Math.toDegrees(maxLng), true),
+      f.add(DoublePoint.newRangeQuery("longitude", Math.toDegrees(minLng), Math.toDegrees(maxLng)),
             BooleanClause.Occur.FILTER);
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/demo/src/java/org/apache/lucene/demo/facet/RangeFacetsExample.java
----------------------------------------------------------------------
diff --git a/lucene/demo/src/java/org/apache/lucene/demo/facet/RangeFacetsExample.java b/lucene/demo/src/java/org/apache/lucene/demo/facet/RangeFacetsExample.java
index 96d7c17..8fc7949 100644
--- a/lucene/demo/src/java/org/apache/lucene/demo/facet/RangeFacetsExample.java
+++ b/lucene/demo/src/java/org/apache/lucene/demo/facet/RangeFacetsExample.java
@@ -105,8 +105,7 @@ public class RangeFacetsExample implements Closeable {
     // documents ("browse only"):
     DrillDownQuery q = new DrillDownQuery(getConfig());
 
-    q.add("timestamp", LongPoint.newRangeQuery("timestamp", range.min, range.minInclusive, range.max, range.maxInclusive));
-
+    q.add("timestamp", LongPoint.newRangeQuery("timestamp", range.min, range.max));
     return searcher.search(q, 10);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
index 362dd7b..c246d74 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
@@ -38,42 +38,29 @@ import org.apache.lucene.util.LegacyNumericUtils;
  *
  * @lucene.experimental */
 public final class DoubleRange extends Range {
-  final double minIncl;
-  final double maxIncl;
-
-  /** Minimum. */
+  /** Minimum (inclusive). */
   public final double min;
 
-  /** Maximum. */
+  /** Maximum (inclusive. */
   public final double max;
 
-  /** True if the minimum value is inclusive. */
-  public final boolean minInclusive;
-
-  /** True if the maximum value is inclusive. */
-  public final boolean maxInclusive;
-
   /** Create a DoubleRange. */
   public DoubleRange(String label, double minIn, boolean minInclusive, double maxIn, boolean maxInclusive) {
     super(label);
-    this.min = minIn;
-    this.max = maxIn;
-    this.minInclusive = minInclusive;
-    this.maxInclusive = maxInclusive;
 
     // TODO: if DoubleDocValuesField used
     // LegacyNumericUtils.doubleToSortableLong format (instead of
     // Double.doubleToRawLongBits) we could do comparisons
     // in long space 
 
-    if (Double.isNaN(min)) {
+    if (Double.isNaN(minIn)) {
       throw new IllegalArgumentException("min cannot be NaN");
     }
     if (!minInclusive) {
       minIn = Math.nextUp(minIn);
     }
 
-    if (Double.isNaN(max)) {
+    if (Double.isNaN(maxIn)) {
       throw new IllegalArgumentException("max cannot be NaN");
     }
     if (!maxInclusive) {
@@ -85,24 +72,24 @@ public final class DoubleRange extends Range {
       failNoMatch();
     }
 
-    this.minIncl = minIn;
-    this.maxIncl = maxIn;
+    this.min = minIn;
+    this.max = maxIn;
   }
 
   /** True if this range accepts the provided value. */
   public boolean accept(double value) {
-    return value >= minIncl && value <= maxIncl;
+    return value >= min && value <= max;
   }
 
   LongRange toLongRange() {
     return new LongRange(label,
-                         LegacyNumericUtils.doubleToSortableLong(minIncl), true,
-                         LegacyNumericUtils.doubleToSortableLong(maxIncl), true);
+                         LegacyNumericUtils.doubleToSortableLong(min), true,
+                         LegacyNumericUtils.doubleToSortableLong(max), true);
   }
 
   @Override
   public String toString() {
-    return "DoubleRange(" + minIncl + " to " + maxIncl + ")";
+    return "DoubleRange(" + min + " to " + max + ")";
   }
 
   private static class ValueSourceQuery extends Query {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
index 8892725..a39ea7e 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
@@ -88,8 +88,8 @@ public class DoubleRangeFacetCounts extends RangeFacetCounts {
     for(int i=0;i<ranges.length;i++) {
       DoubleRange range = ranges[i];
       longRanges[i] =  new LongRange(range.label,
-                                     LegacyNumericUtils.doubleToSortableLong(range.minIncl), true,
-                                     LegacyNumericUtils.doubleToSortableLong(range.maxIncl), true);
+                                     LegacyNumericUtils.doubleToSortableLong(range.min), true,
+                                     LegacyNumericUtils.doubleToSortableLong(range.max), true);
     }
 
     LongRangeCounter counter = new LongRangeCounter(longRanges);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
index 73c519d..ef789c5 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
@@ -37,31 +37,18 @@ import org.apache.lucene.search.Weight;
  *
  * @lucene.experimental */
 public final class LongRange extends Range {
-  final long minIncl;
-  final long maxIncl;
-
-  /** Minimum. */
+  /** Minimum (inclusive). */
   public final long min;
 
-  /** Maximum. */
+  /** Maximum (inclusive). */
   public final long max;
 
-  /** True if the minimum value is inclusive. */
-  public final boolean minInclusive;
-
-  /** True if the maximum value is inclusive. */
-  public final boolean maxInclusive;
-
   // TODO: can we require fewer args? (same for
   // Double/FloatRange too)
 
   /** Create a LongRange. */
   public LongRange(String label, long minIn, boolean minInclusive, long maxIn, boolean maxInclusive) {
     super(label);
-    this.min = minIn;
-    this.max = maxIn;
-    this.minInclusive = minInclusive;
-    this.maxInclusive = maxInclusive;
 
     if (!minInclusive) {
       if (minIn != Long.MAX_VALUE) {
@@ -83,18 +70,18 @@ public final class LongRange extends Range {
       failNoMatch();
     }
 
-    this.minIncl = minIn;
-    this.maxIncl = maxIn;
+    this.min = minIn;
+    this.max = maxIn;
   }
 
   /** True if this range accepts the provided value. */
   public boolean accept(long value) {
-    return value >= minIncl && value <= maxIncl;
+    return value >= min && value <= max;
   }
 
   @Override
   public String toString() {
-    return "LongRange(" + minIncl + " to " + maxIncl + ")";
+    return "LongRange(" + min + " to " + max + ")";
   }
 
   private static class ValueSourceQuery extends Query {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java
index 8c0b123..5c625f0 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeCounter.java
@@ -49,17 +49,17 @@ final class LongRangeCounter {
     endsMap.put(Long.MAX_VALUE, 2);
 
     for(LongRange range : ranges) {
-      Integer cur = endsMap.get(range.minIncl);
+      Integer cur = endsMap.get(range.min);
       if (cur == null) {
-        endsMap.put(range.minIncl, 1);
+        endsMap.put(range.min, 1);
       } else {
-        endsMap.put(range.minIncl, cur.intValue() | 1);
+        endsMap.put(range.min, cur.intValue() | 1);
       }
-      cur = endsMap.get(range.maxIncl);
+      cur = endsMap.get(range.max);
       if (cur == null) {
-        endsMap.put(range.maxIncl, 2);
+        endsMap.put(range.max, 2);
       } else {
-        endsMap.put(range.maxIncl, cur.intValue() | 2);
+        endsMap.put(range.max, cur.intValue() | 2);
       }
     }
 
@@ -276,7 +276,7 @@ final class LongRangeCounter {
 
     /** Recursively assigns range outputs to each node. */
     void addOutputs(int index, LongRange range) {
-      if (start >= range.minIncl && end <= range.maxIncl) {
+      if (start >= range.min && end <= range.max) {
         // Our range is fully included in the incoming
         // range; add to our output list:
         if (outputs == null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
index c4233c0..e7e5d57 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
@@ -280,7 +280,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
 
     // Third search, drill down on "less than or equal to 10":
     ddq = new DrillDownQuery(config);
-    ddq.add("field", LongPoint.newRangeQuery("field", 0L, true, 10L, true));
+    ddq.add("field", LongPoint.newRangeQuery("field", 0L, 10L));
     dsr = ds.search(null, ddq, 10);
 
     assertEquals(11, dsr.hits.totalHits);
@@ -460,9 +460,9 @@ public class TestRangeFacetCounts extends FacetTestCase {
       Query fastMatchQuery;
       if (random().nextBoolean()) {
         if (random().nextBoolean()) {
-          fastMatchQuery = LongPoint.newRangeQuery("field", minValue, true, maxValue, true);
+          fastMatchQuery = LongPoint.newRangeQuery("field", minValue, maxValue);
         } else {
-          fastMatchQuery = LongPoint.newRangeQuery("field", minAcceptedValue, true, maxAcceptedValue, true);
+          fastMatchQuery = LongPoint.newRangeQuery("field", minAcceptedValue, maxAcceptedValue);
         }
       } else {
         fastMatchQuery = null;
@@ -484,7 +484,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
         // Test drill-down:
         DrillDownQuery ddq = new DrillDownQuery(config);
         if (random().nextBoolean()) {
-          ddq.add("field", LongPoint.newRangeQuery("field", range.min, range.minInclusive, range.max, range.maxInclusive));
+          ddq.add("field", LongPoint.newRangeQuery("field", range.min, range.max));
         } else {
           ddq.add("field", range.getQuery(fastMatchQuery, vs));
         }
@@ -615,9 +615,9 @@ public class TestRangeFacetCounts extends FacetTestCase {
       Query fastMatchQuery;
       if (random().nextBoolean()) {
         if (random().nextBoolean()) {
-          fastMatchQuery = FloatPoint.newRangeQuery("field", minValue, true, maxValue, true);
+          fastMatchQuery = FloatPoint.newRangeQuery("field", minValue, maxValue);
         } else {
-          fastMatchQuery = FloatPoint.newRangeQuery("field", minAcceptedValue, true, maxAcceptedValue, true);
+          fastMatchQuery = FloatPoint.newRangeQuery("field", minAcceptedValue, maxAcceptedValue);
         }
       } else {
         fastMatchQuery = null;
@@ -639,7 +639,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
         // Test drill-down:
         DrillDownQuery ddq = new DrillDownQuery(config);
         if (random().nextBoolean()) {
-          ddq.add("field", FloatPoint.newRangeQuery("field", (float) range.min, range.minInclusive, (float) range.max, range.maxInclusive));
+          ddq.add("field", FloatPoint.newRangeQuery("field", (float) range.min, (float) range.max));
         } else {
           ddq.add("field", range.getQuery(fastMatchQuery, vs));
         }
@@ -754,9 +754,9 @@ public class TestRangeFacetCounts extends FacetTestCase {
       Query fastMatchFilter;
       if (random().nextBoolean()) {
         if (random().nextBoolean()) {
-          fastMatchFilter = DoublePoint.newRangeQuery("field", minValue, true, maxValue, true);
+          fastMatchFilter = DoublePoint.newRangeQuery("field", minValue, maxValue);
         } else {
-          fastMatchFilter = DoublePoint.newRangeQuery("field", minAcceptedValue, true, maxAcceptedValue, true);
+          fastMatchFilter = DoublePoint.newRangeQuery("field", minAcceptedValue, maxAcceptedValue);
         }
       } else {
         fastMatchFilter = null;
@@ -778,7 +778,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
         // Test drill-down:
         DrillDownQuery ddq = new DrillDownQuery(config);
         if (random().nextBoolean()) {
-          ddq.add("field", DoublePoint.newRangeQuery("field", range.min, range.minInclusive, range.max, range.maxInclusive));
+          ddq.add("field", DoublePoint.newRangeQuery("field", range.min, range.max));
         } else {
           ddq.add("field", range.getQuery(fastMatchFilter, vs));
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
index edc91f6..187f4a5 100644
--- a/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
@@ -583,7 +583,7 @@ public class HighlighterTest extends BaseTokenStreamTestCase implements Formatte
   
   public void testDimensionalRangeQuery() throws Exception {
     // doesn't currently highlight, but make sure it doesn't cause exception either
-    query = IntPoint.newRangeQuery(NUMERIC_FIELD_NAME, 2, true, 6, true);
+    query = IntPoint.newRangeQuery(NUMERIC_FIELD_NAME, 2, 6);
     searcher = newSearcher(reader);
     hits = searcher.search(query, 100);
     int maxNumFragmentsRequired = 2;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
index cb9f5c1..9f765ab 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/BigIntegerPoint.java
@@ -16,7 +16,6 @@
  */
 package org.apache.lucene.document;
 
-import java.io.IOException;
 import java.math.BigInteger;
 import java.util.Arrays;
 
@@ -24,7 +23,6 @@ import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.NumericUtils;
 
 /** 
@@ -36,16 +34,22 @@ import org.apache.lucene.util.NumericUtils;
  * <p>
  * This field defines static factory methods for creating common queries:
  * <ul>
- *   <li>{@link #newExactQuery newExactQuery()} for matching an exact 1D point.
- *   <li>{@link #newRangeQuery newRangeQuery()} for matching a 1D range.
- *   <li>{@link #newMultiRangeQuery newMultiRangeQuery()} for matching points/ranges in n-dimensional space.
- *   <li>{@link #newSetQuery newSetQuery()} for matching a set of 1D values.
+ *   <li>{@link #newExactQuery(String, BigInteger)} for matching an exact 1D point.
+ *   <li>{@link #newSetQuery(String, BigInteger...)} for matching a set of 1D values.
+ *   <li>{@link #newRangeQuery(String, BigInteger, BigInteger)} for matching a 1D range.
+ *   <li>{@link #newRangeQuery(String, BigInteger[], BigInteger[])} for matching points/ranges in n-dimensional space.
  * </ul>
  */
 public class BigIntegerPoint extends Field {
 
   /** The number of bytes per dimension: 128 bits. */
   public static final int BYTES = 16;
+  
+  /** A constant holding the minimum value a BigIntegerPoint can have, -2<sup>127</sup>. */
+  public static final BigInteger MIN_VALUE = BigInteger.ONE.shiftLeft(BYTES * 8 - 1).negate();
+
+  /** A constant holding the maximum value a BigIntegerPoint can have, 2<sup>127</sup>-1. */
+  public static final BigInteger MAX_VALUE = BigInteger.ONE.shiftLeft(BYTES * 8 - 1).subtract(BigInteger.ONE);
 
   private static FieldType getType(int numDims) {
     FieldType type = new FieldType();
@@ -128,10 +132,8 @@ public class BigIntegerPoint extends Field {
   private static byte[][] encode(BigInteger value[]) {
     byte[][] encoded = new byte[value.length][];
     for (int i = 0; i < value.length; i++) {
-      if (value[i] != null) {
-        encoded[i] = new byte[BYTES];
-        encodeDimension(value[i], encoded[i], 0);
-      }
+      encoded[i] = new byte[BYTES];
+      encodeDimension(value[i], encoded[i], 0);
     }
     return encoded;
   }
@@ -154,65 +156,61 @@ public class BigIntegerPoint extends Field {
    * Create a query for matching an exact big integer value.
    * <p>
    * This is for simple one-dimension points, for multidimensional points use
-   * {@link #newMultiRangeQuery newMultiRangeQuery()} instead.
+   * {@link #newRangeQuery(String, BigInteger[], BigInteger[])} instead.
    *
    * @param field field name. must not be {@code null}.
-   * @param value exact value
-   * @throws IllegalArgumentException if {@code field} is null.
+   * @param value exact value. must not be {@code null}.
+   * @throws IllegalArgumentException if {@code field} is null or {@code value} is null.
    * @return a query matching documents with this exact value
    */
   public static Query newExactQuery(String field, BigInteger value) {
-    return newRangeQuery(field, value, true, value, true);
+    return newRangeQuery(field, value, value);
   }
 
   /** 
    * Create a range query for big integer values.
    * <p>
    * This is for simple one-dimension ranges, for multidimensional ranges use
-   * {@link #newMultiRangeQuery newMultiRangeQuery()} instead.
+   * {@link #newRangeQuery(String, BigInteger[], BigInteger[])} instead.
    * <p>
    * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting the {@code lowerValue} or {@code upperValue} to {@code null}. 
+   * by setting {@code lowerValue = BigIntegerPoint.MIN_VALUE} 
+   * or {@code upperValue = BigIntegerPoint.MAX_VALUE}. 
    * <p>
-   * By setting inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
-   * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
+   * Ranges are inclusive. For exclusive ranges, pass {@code lowerValue.add(BigInteger.ONE)} 
+   * or {@code upperValue.subtract(BigInteger.ONE)}
    *
    * @param field field name. must not be {@code null}.
-   * @param lowerValue lower portion of the range. {@code null} means "open".
-   * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
-   * @param upperValue upper portion of the range. {@code null} means "open".
-   * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
-   * @throws IllegalArgumentException if {@code field} is null.
+   * @param lowerValue lower portion of the range (inclusive). must not be {@code null}.
+   * @param upperValue upper portion of the range (inclusive). must not be {@code null}.
+   * @throws IllegalArgumentException if {@code field} is null, {@code lowerValue} is null, or {@code upperValue} is null.
    * @return a query matching documents within this range.
    */
-  public static Query newRangeQuery(String field, BigInteger lowerValue, boolean lowerInclusive, BigInteger upperValue, boolean upperInclusive) {
-    return newMultiRangeQuery(field, 
-                              new BigInteger[] { lowerValue },
-                              new boolean[] { lowerInclusive }, 
-                              new BigInteger[] { upperValue },
-                              new boolean[] { upperInclusive });
+  public static Query newRangeQuery(String field, BigInteger lowerValue, BigInteger upperValue) {
+    PointRangeQuery.checkArgs(field, lowerValue, upperValue);
+    return newRangeQuery(field, new BigInteger[] { lowerValue }, new BigInteger[] { upperValue });
   }
 
   /** 
-   * Create a multidimensional range query for big integer values.
+   * Create a range query for n-dimensional big integer values.
    * <p>
    * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting a {@code lowerValue} element or {@code upperValue} element to {@code null}. 
+   * by setting {@code lowerValue[i] = BigIntegerPoint.MIN_VALUE} 
+   * or {@code upperValue[i] = BigIntegerPoint.MAX_VALUE}. 
    * <p>
-   * By setting a dimension's inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
-   * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
+   * Ranges are inclusive. For exclusive ranges, pass {@code lowerValue[i].add(BigInteger.ONE)} 
+   * or {@code upperValue[i].subtract(BigInteger.ONE)}
    *
    * @param field field name. must not be {@code null}.
-   * @param lowerValue lower portion of the range. {@code null} values mean "open" for that dimension.
-   * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
-   * @param upperValue upper portion of the range. {@code null} values mean "open" for that dimension.
-   * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
-   * @throws IllegalArgumentException if {@code field} is null, or if {@code lowerValue.length != upperValue.length}
+   * @param lowerValue lower portion of the range (inclusive). must not be {@code null}.
+   * @param upperValue upper portion of the range (inclusive). must not be {@code null}.
+   * @throws IllegalArgumentException if {@code field} is null, if {@code lowerValue} is null, if {@code upperValue} is null, 
+   *                                  or if {@code lowerValue.length != upperValue.length}
    * @return a query matching documents within this range.
    */
-  public static Query newMultiRangeQuery(String field, BigInteger[] lowerValue, boolean lowerInclusive[], BigInteger[] upperValue, boolean upperInclusive[]) {
+  public static Query newRangeQuery(String field, BigInteger[] lowerValue, BigInteger[] upperValue) {
     PointRangeQuery.checkArgs(field, lowerValue, upperValue);
-    return new PointRangeQuery(field, BigIntegerPoint.encode(lowerValue), lowerInclusive, BigIntegerPoint.encode(upperValue), upperInclusive) {
+    return new PointRangeQuery(field, BigIntegerPoint.encode(lowerValue), BigIntegerPoint.encode(upperValue)) {
       @Override
       protected String toString(int dimension, byte[] value) {
         return BigIntegerPoint.decodeDimension(value, 0).toString();
@@ -224,30 +222,29 @@ public class BigIntegerPoint extends Field {
    * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
    * 
    * @param field field name. must not be {@code null}.
-   * @param valuesIn all values to match
+   * @param values all values to match
    */
-  public static Query newSetQuery(String field, BigInteger... valuesIn) throws IOException {
+  public static Query newSetQuery(String field, BigInteger... values) {
 
     // Don't unexpectedly change the user's incoming values array:
-    BigInteger[] values = valuesIn.clone();
-
-    Arrays.sort(values);
+    BigInteger[] sortedValues = values.clone();
+    Arrays.sort(sortedValues);
 
-    final BytesRef value = new BytesRef(new byte[BYTES]);
+    final BytesRef encoded = new BytesRef(new byte[BYTES]);
 
     return new PointInSetQuery(field, 1, BYTES,
-                               new BytesRefIterator() {
+                               new PointInSetQuery.Stream() {
 
                                  int upto;
 
                                  @Override
                                  public BytesRef next() {
-                                   if (upto == values.length) {
+                                   if (upto == sortedValues.length) {
                                      return null;
                                    } else {
-                                     encodeDimension(values[upto], value.bytes, 0);
+                                     encodeDimension(sortedValues[upto], encoded.bytes, 0);
                                      upto++;
-                                     return value;
+                                     return encoded;
                                    }
                                  }
                                }) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
index 1a73dc1..a0623b3 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
@@ -16,7 +16,6 @@
  */
 package org.apache.lucene.document;
 
-import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Arrays;
@@ -25,7 +24,6 @@ import org.apache.lucene.search.PointInSetQuery;
 import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefIterator;
 
 /** 
  * An indexed 128-bit {@code InetAddress} field.
@@ -36,10 +34,10 @@ import org.apache.lucene.util.BytesRefIterator;
  * <p>
  * This field defines static factory methods for creating common queries:
  * <ul>
- *   <li>{@link #newExactQuery newExactQuery()} for matching an exact network address.
- *   <li>{@link #newPrefixQuery newPrefixQuery()} for matching a network based on CIDR prefix.
- *   <li>{@link #newRangeQuery newRangeQuery()} for matching arbitrary network address ranges.
- *   <li>{@link #newSetQuery newSetQuery()} for matching a set of 1D values.
+ *   <li>{@link #newExactQuery(String, InetAddress)} for matching an exact network address.
+ *   <li>{@link #newPrefixQuery(String, InetAddress, int)} for matching a network based on CIDR prefix.
+ *   <li>{@link #newRangeQuery(String, InetAddress, InetAddress)} for matching arbitrary network address ranges.
+ *   <li>{@link #newSetQuery(String, InetAddress...)} for matching a set of 1D values.
  * </ul>
  * <p>
  * This field supports both IPv4 and IPv6 addresses: IPv4 addresses are converted
@@ -149,7 +147,7 @@ public class InetAddressPoint extends Field {
    * @return a query matching documents with this exact value
    */
   public static Query newExactQuery(String field, InetAddress value) {
-    return newRangeQuery(field, value, true, value, true);
+    return newRangeQuery(field, value, value);
   }
   
   /** 
@@ -162,6 +160,9 @@ public class InetAddressPoint extends Field {
    * @return a query matching documents with addresses contained within this network
    */
   public static Query newPrefixQuery(String field, InetAddress value, int prefixLength) {
+    if (value == null) {
+      throw new IllegalArgumentException("InetAddress cannot be null");
+    }
     if (prefixLength < 0 || prefixLength > 8 * value.getAddress().length) {
       throw new IllegalArgumentException("illegal prefixLength '" + prefixLength + "'. Must be 0-32 for IPv4 ranges, 0-128 for IPv6 ranges");
     }
@@ -173,7 +174,7 @@ public class InetAddressPoint extends Field {
       upper[i >> 3] |= 1 << (i & 7);
     }
     try {
-      return newRangeQuery(field, InetAddress.getByAddress(lower), true, InetAddress.getByAddress(upper), true);
+      return newRangeQuery(field, InetAddress.getByAddress(lower), InetAddress.getByAddress(upper));
     } catch (UnknownHostException e) {
       throw new AssertionError(e); // values are coming from InetAddress
     }
@@ -181,31 +182,21 @@ public class InetAddressPoint extends Field {
 
   /** 
    * Create a range query for network addresses.
-   * <p>
-   * You can have half-open ranges (which are in fact &lt;/&le; or &gt;/&ge; queries)
-   * by setting the {@code lowerValue} or {@code upperValue} to {@code null}. 
-   * <p>
-   * By setting inclusive ({@code lowerInclusive} or {@code upperInclusive}) to false, it will
-   * match all documents excluding the bounds, with inclusive on, the boundaries are hits, too.
    *
    * @param field field name. must not be {@code null}.
-   * @param lowerValue lower portion of the range. {@code null} means "open".
-   * @param lowerInclusive {@code true} if the lower portion of the range is inclusive, {@code false} if it should be excluded.
-   * @param upperValue upper portion of the range. {@code null} means "open".
-   * @param upperInclusive {@code true} if the upper portion of the range is inclusive, {@code false} if it should be excluded.
-   * @throws IllegalArgumentException if {@code field} is null.
+   * @param lowerValue lower portion of the range (inclusive). must not be null.
+   * @param upperValue upper portion of the range (inclusive). must not be null.
+   * @throws IllegalArgumentException if {@code field} is null, {@code lowerValue} is null, 
+   *                                  or {@code upperValue} is null
    * @return a query matching documents within this range.
    */
-  public static Query newRangeQuery(String field, InetAddress lowerValue, boolean lowerInclusive, InetAddress upperValue, boolean upperInclusive) {
+  public static Query newRangeQuery(String field, InetAddress lowerValue, InetAddress upperValue) {
+    PointRangeQuery.checkArgs(field, lowerValue, upperValue);
     byte[][] lowerBytes = new byte[1][];
-    if (lowerValue != null) {
-      lowerBytes[0] = encode(lowerValue);
-    }
+    lowerBytes[0] = encode(lowerValue);
     byte[][] upperBytes = new byte[1][];
-    if (upperValue != null) {
-      upperBytes[0] = encode(upperValue);
-    }
-    return new PointRangeQuery(field, lowerBytes, new boolean[] { lowerInclusive }, upperBytes, new boolean[] { upperInclusive }) {
+    upperBytes[0] = encode(upperValue);
+    return new PointRangeQuery(field, lowerBytes, upperBytes) {
       @Override
       protected String toString(int dimension, byte[] value) {
         return decode(value).getHostAddress(); // for ranges, the range itself is already bracketed
@@ -217,31 +208,30 @@ public class InetAddressPoint extends Field {
    * Create a query matching any of the specified 1D values.  This is the points equivalent of {@code TermsQuery}.
    * 
    * @param field field name. must not be {@code null}.
-   * @param valuesIn all values to match
+   * @param values all values to match
    */
-  public static Query newSetQuery(String field, InetAddress... valuesIn) throws IOException {
+  public static Query newSetQuery(String field, InetAddress... values) {
 
     // Don't unexpectedly change the user's incoming values array:
-    InetAddress[] values = valuesIn.clone();
-
-    Arrays.sort(values);
+    InetAddress[] sortedValues = values.clone();
+    Arrays.sort(sortedValues);
 
-    final BytesRef value = new BytesRef(new byte[BYTES]);
+    final BytesRef encoded = new BytesRef(new byte[BYTES]);
 
     return new PointInSetQuery(field, 1, BYTES,
-                               new BytesRefIterator() {
+                               new PointInSetQuery.Stream() {
 
                                  int upto;
 
                                  @Override
                                  public BytesRef next() {
-                                   if (upto == values.length) {
+                                   if (upto == sortedValues.length) {
                                      return null;
                                    } else {
-                                     value.bytes = encode(values[upto]);
-                                     assert value.bytes.length == value.length;
+                                     encoded.bytes = encode(sortedValues[upto]);
+                                     assert encoded.bytes.length == encoded.length;
                                      upto++;
-                                     return value;
+                                     return encoded;
                                    }
                                  }
                                }) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
index b0902f5..aeb0a0f 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonPoint.java
@@ -233,12 +233,17 @@ public class LatLonPoint extends Field {
       // E.g.: maxLon = -179, minLon = 179
       byte[][] leftOpen = new byte[2][];
       leftOpen[0] = lower[0];
-      // leave longitude open (null)
+      // leave longitude open
+      leftOpen[1] = new byte[Integer.BYTES];
+      NumericUtils.intToBytes(Integer.MIN_VALUE, leftOpen[1], 0);
       Query left = newBoxInternal(field, leftOpen, upper);
       q.add(new BooleanClause(left, BooleanClause.Occur.SHOULD));
+
       byte[][] rightOpen = new byte[2][];
       rightOpen[0] = upper[0];
-      // leave longitude open (null)
+      // leave longitude open
+      rightOpen[1] = new byte[Integer.BYTES];
+      NumericUtils.intToBytes(Integer.MAX_VALUE, rightOpen[1], 0);
       Query right = newBoxInternal(field, lower, rightOpen);
       q.add(new BooleanClause(right, BooleanClause.Occur.SHOULD));
       return new ConstantScoreQuery(q.build());
@@ -248,7 +253,7 @@ public class LatLonPoint extends Field {
   }
   
   private static Query newBoxInternal(String field, byte[][] min, byte[][] max) {
-    return new PointRangeQuery(field, min, new boolean[] { true, true }, max, new boolean[] { false, false }) {
+    return new PointRangeQuery(field, min, max) {
       @Override
       protected String toString(int dimension, byte[] value) {
         if (dimension == 0) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
index f6d407d..500c2a3 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestBigIntegerPoint.java
@@ -42,7 +42,7 @@ public class TestBigIntegerPoint extends LuceneTestCase {
     IndexReader reader = writer.getReader();
     IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(BigIntegerPoint.newExactQuery("field", large)));
-    assertEquals(1, searcher.count(BigIntegerPoint.newRangeQuery("field", large.subtract(BigInteger.ONE), false, large.add(BigInteger.ONE), false)));
+    assertEquals(1, searcher.count(BigIntegerPoint.newRangeQuery("field", large.subtract(BigInteger.ONE), large.add(BigInteger.ONE))));
     assertEquals(1, searcher.count(BigIntegerPoint.newSetQuery("field", large)));
     assertEquals(0, searcher.count(BigIntegerPoint.newSetQuery("field", large.subtract(BigInteger.ONE))));
     assertEquals(0, searcher.count(BigIntegerPoint.newSetQuery("field")));
@@ -67,7 +67,7 @@ public class TestBigIntegerPoint extends LuceneTestCase {
     IndexReader reader = writer.getReader();
     IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(BigIntegerPoint.newExactQuery("field", negative)));
-    assertEquals(1, searcher.count(BigIntegerPoint.newRangeQuery("field", negative.subtract(BigInteger.ONE), false, negative.add(BigInteger.ONE), false)));
+    assertEquals(1, searcher.count(BigIntegerPoint.newRangeQuery("field", negative.subtract(BigInteger.ONE), negative.add(BigInteger.ONE))));
 
     reader.close();
     writer.close();
@@ -87,12 +87,10 @@ public class TestBigIntegerPoint extends LuceneTestCase {
     assertEquals("BigIntegerPoint <field:1>", new BigIntegerPoint("field", BigInteger.ONE).toString());
     assertEquals("BigIntegerPoint <field:1,-2>", new BigIntegerPoint("field", BigInteger.ONE, BigInteger.valueOf(-2)).toString());
     assertEquals("field:[1 TO 1]", BigIntegerPoint.newExactQuery("field", BigInteger.ONE).toString());
-    assertEquals("field:{1 TO 17]", BigIntegerPoint.newRangeQuery("field", BigInteger.ONE, false, BigInteger.valueOf(17), true).toString());
-    assertEquals("field:{1 TO 17],[0 TO 42}", BigIntegerPoint.newMultiRangeQuery("field",
+    assertEquals("field:[1 TO 17]", BigIntegerPoint.newRangeQuery("field", BigInteger.ONE, BigInteger.valueOf(17)).toString());
+    assertEquals("field:[1 TO 17],[0 TO 42]", BigIntegerPoint.newRangeQuery("field",
                                                                                  new BigInteger[] {BigInteger.ONE, BigInteger.ZERO},
-                                                                                 new boolean[] {false, true},
-                                                                                 new BigInteger[] {BigInteger.valueOf(17), BigInteger.valueOf(42)},
-                                                                                 new boolean[] {true, false}).toString());
+                                                                                 new BigInteger[] {BigInteger.valueOf(17), BigInteger.valueOf(42)}).toString());
     assertEquals("field:{1}", BigIntegerPoint.newSetQuery("field", BigInteger.ONE).toString());
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
index 9854001..d4ddb3a 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
@@ -43,7 +43,7 @@ public class TestInetAddressPoint extends LuceneTestCase {
     IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(InetAddressPoint.newExactQuery("field", address)));
     assertEquals(1, searcher.count(InetAddressPoint.newPrefixQuery("field", address, 24)));
-    assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("1.2.3.3"), false, InetAddress.getByName("1.2.3.5"), false)));
+    assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"))));
     assertEquals(1, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.4"))));
     assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.3"))));
     assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field")));
@@ -69,7 +69,7 @@ public class TestInetAddressPoint extends LuceneTestCase {
     IndexSearcher searcher = newSearcher(reader, false);
     assertEquals(1, searcher.count(InetAddressPoint.newExactQuery("field", address)));
     assertEquals(1, searcher.count(InetAddressPoint.newPrefixQuery("field", address, 64)));
-    assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("fec0::f66c"), false, InetAddress.getByName("fec0::f66e"), false)));
+    assertEquals(1, searcher.count(InetAddressPoint.newRangeQuery("field", InetAddress.getByName("fec0::f66c"), InetAddress.getByName("fec0::f66e"))));
 
     reader.close();
     writer.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
index 1d3bfac..0ef948d 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestLatLonPoint.java
@@ -51,7 +51,7 @@ public class TestLatLonPoint extends LuceneTestCase {
     assertEquals("LatLonPoint <field:18.313693958334625,-65.22744392976165>",(new LatLonPoint("field", 18.313694, -65.227444)).toString());
     
     // looks crazy due to lossiness
-    assertEquals("field:[17.99999997485429 TO 18.999999999068677},[-65.9999999217689 TO -64.99999998137355}", LatLonPoint.newBoxQuery("field", 18, 19, -66, -65).toString());
+    assertEquals("field:[17.99999997485429 TO 18.999999999068677],[-65.9999999217689 TO -64.99999998137355]", LatLonPoint.newBoxQuery("field", 18, 19, -66, -65).toString());
     
     // distance query does not quantize inputs
     assertEquals("field:18.0,19.0 +/- 25.0 meters", LatLonPoint.newDistanceQuery("field", 18, 19, 25).toString());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
index 47cd740..f2ec17e 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
@@ -56,7 +56,7 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
         iw.addDocument(doc);
       }
       if (random().nextBoolean()) {
-        iw.deleteDocuments(LongPoint.newRangeQuery("idx", 0L, true, 10L, true));
+        iw.deleteDocuments(LongPoint.newRangeQuery("idx", 0L, 10L));
       }
       iw.commit();
       final IndexReader reader = iw.getReader();
@@ -64,12 +64,10 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
       iw.close();
 
       for (int i = 0; i < 100; ++i) {
-        final Long min = random().nextBoolean() ? null : TestUtil.nextLong(random(), -100, 1000);
-        final Long max = random().nextBoolean() ? null : TestUtil.nextLong(random(), -100, 1000);
-        final boolean minInclusive = random().nextBoolean();
-        final boolean maxInclusive = random().nextBoolean();
-        final Query q1 = LongPoint.newRangeQuery("idx", min, minInclusive, max, maxInclusive);
-        final Query q2 = DocValuesRangeQuery.newLongRange("dv", min, max, minInclusive, maxInclusive);
+        final Long min = TestUtil.nextLong(random(), -100, 1000);
+        final Long max = TestUtil.nextLong(random(), -100, 1000);
+        final Query q1 = LongPoint.newRangeQuery("idx", min, max);
+        final Query q2 = DocValuesRangeQuery.newLongRange("dv", min, max, true, true);
         assertSameMatches(searcher, q1, q2, false);
       }
 
@@ -184,7 +182,7 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
       iw.addDocument(doc);
     }
     if (random().nextBoolean()) {
-      iw.deleteDocuments(LongPoint.newRangeQuery("idx", 0L, true, 10L, true));
+      iw.deleteDocuments(LongPoint.newRangeQuery("idx", 0L, 10L));
     }
     iw.commit();
     final IndexReader reader = iw.getReader();
@@ -192,23 +190,21 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
     iw.close();
 
     for (int i = 0; i < 100; ++i) {
-      final Long min = random().nextBoolean() ? null : TestUtil.nextLong(random(), -100, 1000);
-      final Long max = random().nextBoolean() ? null : TestUtil.nextLong(random(), -100, 1000);
-      final boolean minInclusive = random().nextBoolean();
-      final boolean maxInclusive = random().nextBoolean();
+      final Long min = TestUtil.nextLong(random(), -100, 1000);
+      final Long max = TestUtil.nextLong(random(), -100, 1000);
 
       BooleanQuery.Builder ref = new BooleanQuery.Builder();
-      ref.add(LongPoint.newRangeQuery("idx", min, minInclusive, max, maxInclusive), Occur.FILTER);
+      ref.add(LongPoint.newRangeQuery("idx", min, max), Occur.FILTER);
       ref.add(new TermQuery(new Term("f", "a")), Occur.MUST);
 
       BooleanQuery.Builder bq1 = new BooleanQuery.Builder();
-      bq1.add(DocValuesRangeQuery.newLongRange("dv1", min, max, minInclusive, maxInclusive), Occur.FILTER);
+      bq1.add(DocValuesRangeQuery.newLongRange("dv1", min, max, true, true), Occur.FILTER);
       bq1.add(new TermQuery(new Term("f", "a")), Occur.MUST);
 
       assertSameMatches(searcher, ref.build(), bq1.build(), true);
 
       BooleanQuery.Builder bq2 = new BooleanQuery.Builder();
-      bq2.add(DocValuesRangeQuery.newBytesRefRange("dv2", toSortableBytes(min), toSortableBytes(max), minInclusive, maxInclusive), Occur.FILTER);
+      bq2.add(DocValuesRangeQuery.newBytesRefRange("dv2", toSortableBytes(min), toSortableBytes(max), true, true), Occur.FILTER);
       bq2.add(new TermQuery(new Term("f", "a")), Occur.MUST);
 
       assertSameMatches(searcher, ref.build(), bq2.build(), true);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/sandbox/src/test/org/apache/lucene/search/TestLatLonPointQueries.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestLatLonPointQueries.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestLatLonPointQueries.java
index 8eb1be0..9f9bba2 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestLatLonPointQueries.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestLatLonPointQueries.java
@@ -66,15 +66,15 @@ public class TestLatLonPointQueries extends BaseGeoPointTestCase {
 
     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);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java b/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
index 8313616..ac636ec 100644
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
+++ b/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
@@ -519,6 +519,9 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
 
       boolean fail = false;
 
+      // Change to false to see all wrong hits:
+      boolean failFast = true;
+
       for(int docID=0;docID<maxDoc;docID++) {
         int id = (int) docIDToID.get(docID);
         Boolean expected;
@@ -532,18 +535,26 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
 
         // null means it's a borderline case which is allowed to be wrong:
         if (expected != null && hits.get(docID) != expected) {
-          if (expected) {
-            System.out.println(Thread.currentThread().getName() + ": id=" + id + " should match but did not");
-          } else {
-            System.out.println(Thread.currentThread().getName() + ": id=" + id + " should not match but did");
-          }
-          System.out.println("  small=" + small + " query=" + query +
-                             " docID=" + docID + "\n  lat=" + lats[id] + " lon=" + lons[id] +
-                             "\n  deleted?=" + deleted.contains(id));
-          if (Double.isNaN(lats[id]) == false) {
-            describe(docID, lats[id], lons[id]);
+
+          // Print only one failed hit; add a true || in here to see all failures:
+          if (failFast == false || failed.getAndSet(true) == false) {
+            if (expected) {
+              System.out.println(Thread.currentThread().getName() + ": id=" + id + " should match but did not");
+            } else {
+              System.out.println(Thread.currentThread().getName() + ": id=" + id + " should not match but did");
+            }
+            System.out.println("  small=" + small + " query=" + query +
+                               " docID=" + docID + "\n  lat=" + lats[id] + " lon=" + lons[id] +
+                               "\n  deleted?=" + deleted.contains(id));
+            if (Double.isNaN(lats[id]) == false) {
+              describe(docID, lats[id], lons[id]);
+            }
+            if (failFast) {
+              fail("wrong hit (first of possibly more)");
+            } else {
+              fail = true;
+            }
           }
-          fail = true;
         }
       }
 
@@ -768,4 +779,45 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
     IOUtils.close(r, dir);
     assertFalse(failed.get());
   }
+
+  public void testRectBoundariesAreInclusive() throws Exception {
+    GeoRect rect = randomRect(random().nextBoolean(), false);
+    Query query = newRectQuery(FIELD_NAME, rect);
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+    for(int x=0;x<3;x++) {
+      double lat;
+      if (x == 0) {
+        lat = rect.minLat;
+      } else if (x == 1) {
+        lat = quantizeLat((rect.minLat+rect.maxLat)/2.0);
+      } else {
+        lat = rect.maxLat;
+      }
+      for(int y=0;y<3;y++) {
+        double lon;
+        if (y == 0) {
+          lon = rect.minLon;
+        } else if (y == 1) {
+          if (x == 1) {
+            continue;
+          }
+          lon = quantizeLon((rect.minLon+rect.maxLon)/2.0);
+        } else {
+          lon = rect.maxLon;
+        }
+
+        Document doc = new Document();
+        addPointToDoc(FIELD_NAME, doc, lat, lon);
+        w.addDocument(doc);
+      }
+    }
+    IndexReader r = w.getReader();
+    IndexSearcher s = newSearcher(r, false);
+    assertEquals(8, s.count(newRectQuery(FIELD_NAME, rect)));
+    r.close();
+    w.close();
+    dir.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dc35cea6/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
index 384b98b..62ed08b 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
@@ -301,7 +301,7 @@ public class TestSuggestField extends LuceneTestCase {
       }
     }
 
-    iw.deleteDocuments(IntPoint.newRangeQuery("weight_fld", 2, true, null, false));
+    iw.deleteDocuments(IntPoint.newRangeQuery("weight_fld", 2, Integer.MAX_VALUE));
 
     DirectoryReader reader = DirectoryReader.open(iw);
     SuggestIndexSearcher indexSearcher = new SuggestIndexSearcher(reader);


[3/9] lucene-solr git commit: SOLR-8145: mention fix in solr/CHANGES.txt (cherry picked from commit ddd019f)

Posted by sh...@apache.org.
SOLR-8145: mention fix in solr/CHANGES.txt
(cherry picked from commit ddd019f)


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

Branch: refs/heads/branch_6_0
Commit: b21927818c30ea2d3defab3d94c19225dc2452c8
Parents: cc8af0d
Author: thelabdude <th...@gmail.com>
Authored: Thu Mar 3 00:07:23 2016 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Mar 3 20:48:26 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b2192781/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index e2c3269..4fbc7ae 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -233,6 +233,9 @@ Bug Fixes
 
 * SOLR-8771: Multi-threaded core shutdown creates executor per core. (Mike Drob via Mark Miller)
 
+* SOLR-8145: Fix position of OOM killer script when starting Solr in the background (Jurian Broertjes via
+  Timothy Potter)
+
 Optimizations
 ----------------------
 * SOLR-7876: Speed up queries and operations that use many terms when timeAllowed has not been