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:17 UTC

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

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);