You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2016/02/22 21:10:22 UTC

lucene-solr git commit: LUCENE-7039: Improve PointRangeQuery & co

Repository: lucene-solr
Updated Branches:
  refs/heads/master 120326464 -> 54a544b30


LUCENE-7039: Improve PointRangeQuery & co


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

Branch: refs/heads/master
Commit: 54a544b3075fe4a5a775d8774d0b462c5324b287
Parents: 1203264
Author: Robert Muir <rm...@apache.org>
Authored: Mon Feb 22 15:09:40 2016 -0500
Committer: Robert Muir <rm...@apache.org>
Committed: Mon Feb 22 15:10:15 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/document/DoublePoint.java |  26 +
 .../org/apache/lucene/document/FloatPoint.java  |  26 +
 .../org/apache/lucene/document/IntPoint.java    |  26 +
 .../org/apache/lucene/document/LongPoint.java   |  26 +
 .../apache/lucene/search/ExactPointQuery.java   | 153 ------
 .../apache/lucene/search/PointRangeQuery.java   | 491 ++++++++++++++++---
 .../index/TestDemoParallelLeafReader.java       |   2 +-
 .../apache/lucene/search/TestPointQueries.java  | 106 ++--
 .../TestUsageTrackingFilterCachingPolicy.java   |   2 +-
 .../demo/facet/DistanceFacetsExample.java       |   8 +-
 .../lucene/demo/facet/RangeFacetsExample.java   |   2 +-
 .../facet/range/TestRangeFacetCounts.java       |  20 +-
 .../search/highlight/HighlighterTest.java       |   2 +-
 .../lucene/search/TestDocValuesRangeQuery.java  |   8 +-
 .../suggest/document/TestSuggestField.java      |   2 +-
 .../lucene/index/BasePointFormatTestCase.java   |   6 +-
 16 files changed, 626 insertions(+), 280 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 b3534fc..1619857 100644
--- a/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DoublePoint.java
@@ -89,4 +89,30 @@ public final class DoublePoint extends Field {
   public DoublePoint(String name, double... point) {
     super(name, pack(point), getType(point.length));
   }
+  
+  // public helper methods (e.g. for queries)
+  // TODO: try to rectify with pack() above, which works on a single concatenated array...
+
+  /** Encode n-dimensional double point into binary encoding */
+  public 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] = encodeDimension(value[i]);
+      }
+    }
+    return encoded;
+  }
+  
+  /** Encode single double dimension */
+  public static byte[] encodeDimension(Double value) {
+    byte encoded[] = new byte[Long.BYTES];
+    NumericUtils.longToBytesDirect(NumericUtils.doubleToSortableLong(value), encoded, 0);
+    return encoded;
+  }
+  
+  /** Decode single double value */
+  public static Double decodeDimension(byte value[]) {
+    return NumericUtils.sortableLongToDouble(NumericUtils.bytesToLongDirect(value, 0));
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 92ea9d6..cfd6c7c 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatPoint.java
@@ -89,4 +89,30 @@ public final class FloatPoint extends Field {
   public FloatPoint(String name, float... point) {
     super(name, pack(point), getType(point.length));
   }
+  
+  // public helper methods (e.g. for queries)
+  // TODO: try to rectify with pack() above, which works on a single concatenated array...
+
+  /** Encode n-dimensional float values into binary encoding */
+  public 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] = encodeDimension(value[i]);
+      }
+    }
+    return encoded;
+  }
+  
+  /** Encode single float dimension */
+  public static byte[] encodeDimension(Float value) {
+    byte encoded[] = new byte[Integer.BYTES];
+    NumericUtils.intToBytesDirect(NumericUtils.floatToSortableInt(value), encoded, 0);
+    return encoded;
+  }
+  
+  /** Decode single float dimension */
+  public static Float decodeDimension(byte value[]) {
+    return NumericUtils.sortableIntToFloat(NumericUtils.bytesToIntDirect(value, 0));
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 e4a88e4..40ddf07 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntPoint.java
@@ -89,4 +89,30 @@ public final class IntPoint extends Field {
   public IntPoint(String name, int... point) {
     super(name, pack(point), getType(point.length));
   }
+
+  // public helper methods (e.g. for queries)
+  // TODO: try to rectify with pack() above, which works on a single concatenated array...
+
+  /** Encode n-dimensional integer values into binary encoding */
+  public static byte[][] encode(Integer value[]) {
+    byte[][] encoded = new byte[value.length][];
+    for (int i = 0; i < value.length; i++) {
+      if (value[i] != null) {
+        encoded[i] = encodeDimension(value[i]);
+      }
+    }
+    return encoded;
+  }
+  
+  /** Encode single integer dimension */
+  public static byte[] encodeDimension(Integer value) {
+    byte encoded[] = new byte[Integer.BYTES];
+    NumericUtils.intToBytes(value, encoded, 0);
+    return encoded;
+  }
+  
+  /** Decode single integer dimension */
+  public static Integer decodeDimension(byte value[]) {
+    return NumericUtils.bytesToInt(value, 0);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 950101f..08a904c 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
@@ -89,4 +89,30 @@ public final class LongPoint extends Field {
   public LongPoint(String name, long... point) {
     super(name, pack(point), getType(point.length));
   }
+  
+  // public helper methods (e.g. for queries)
+  // TODO: try to rectify with pack() above, which works on a single concatenated array...
+
+  /** Encode n-dimensional long values into binary encoding */
+  public 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] = encodeDimension(value[i]);
+      }
+    }
+    return encoded;
+  }
+  
+  /** Encode single long dimension */
+  public static byte[] encodeDimension(Long value) {
+    byte encoded[] = new byte[Long.BYTES];
+    NumericUtils.longToBytes(value, encoded, 0);
+    return encoded;
+  }
+  
+  /** Decode single long dimension */
+  public static Long decodeDimension(byte value[]) {
+    return NumericUtils.bytesToLong(value, 0);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/lucene/core/src/java/org/apache/lucene/search/ExactPointQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ExactPointQuery.java b/lucene/core/src/java/org/apache/lucene/search/ExactPointQuery.java
deleted file mode 100644
index 5bcf113..0000000
--- a/lucene/core/src/java/org/apache/lucene/search/ExactPointQuery.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.search;
-
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Objects;
-
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.PointValues.IntersectVisitor;
-import org.apache.lucene.index.PointValues.Relation;
-import org.apache.lucene.index.PointValues;
-import org.apache.lucene.util.DocIdSetBuilder;
-import org.apache.lucene.util.NumericUtils;
-import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.StringHelper;
-
-/** Searches for single points in fields previously indexed using points
- *  e.g. {@link org.apache.lucene.document.LongPoint}. */
-
-public class ExactPointQuery extends Query {
-  final String field;
-  final int numDims;
-  final byte[][] point;
-  final int bytesPerDim;
-
-  public ExactPointQuery(String field, byte[][] point) {
-    this.field = field;
-    if (point == null) {
-      throw new IllegalArgumentException("point must not be null");
-    }
-    this.point = point;
-    this.numDims = point.length;
-
-    int bytesPerDim = -1;
-    for(byte[] value : point) {
-      if (value == null) {
-        throw new IllegalArgumentException("point's dimensional values must not be 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 = bytesPerDim;
-  }
-
-  /** Use in the 1D case when you indexed 1D int values using {@link org.apache.lucene.document.IntPoint} */
-  public static ExactPointQuery new1DIntExact(String field, int value) {
-    return new ExactPointQuery(field, pack(value));
-  }
-
-  /** Use in the 1D case when you indexed 1D long values using {@link org.apache.lucene.document.LongPoint} */
-  public static ExactPointQuery new1DLongExact(String field, long value) {
-    return new ExactPointQuery(field, pack(value));
-  }
-
-  /** Use in the 1D case when you indexed 1D float values using {@link org.apache.lucene.document.FloatPoint} */
-  public static ExactPointQuery new1DFloatExact(String field, float value) {
-    return new ExactPointQuery(field, pack(value));
-  }
-
-  /** Use in the 1D case when you indexed 1D double values using {@link org.apache.lucene.document.DoublePoint} */
-  public static ExactPointQuery new1DDoubleExact(String field, double value) {
-    return new ExactPointQuery(field, pack(value));
-  }
-
-  private static byte[][] pack(long value) {
-    byte[][] result = new byte[][] {new byte[RamUsageEstimator.NUM_BYTES_LONG]};
-    NumericUtils.longToBytes(value, result[0], 0);
-    return result;
-  }
-
-  private static byte[][] pack(double value) {
-    byte[][] result = new byte[][] {new byte[RamUsageEstimator.NUM_BYTES_LONG]};
-    NumericUtils.longToBytesDirect(NumericUtils.doubleToSortableLong(value), result[0], 0);
-    return result;
-  }
-
-  private static byte[][] pack(int value) {
-    byte[][] result = new byte[][] {new byte[RamUsageEstimator.NUM_BYTES_INT]};
-    NumericUtils.intToBytes(value, result[0], 0);
-    return result;
-  }
-
-  private static byte[][] pack(float value) {
-    byte[][] result = new byte[][] {new byte[RamUsageEstimator.NUM_BYTES_INT]};
-    NumericUtils.intToBytesDirect(NumericUtils.floatToSortableInt(value), result[0], 0);
-    return result;
-  }
-
-  @Override
-  public Query rewrite(IndexReader reader) throws IOException {
-    boolean[] inclusive = new boolean[] {true};
-    return new PointRangeQuery(field, point, inclusive, point, inclusive);
-  }
-
-  @Override
-  public int hashCode() {
-    int hash = super.hashCode();
-    hash += Arrays.hashCode(point)^0x14fa55fb;
-    hash += numDims^0x14fa55fb;
-    hash += Objects.hashCode(bytesPerDim);
-    return hash;
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (super.equals(other)) {
-      final ExactPointQuery q = (ExactPointQuery) other;
-      return q.numDims == numDims &&
-        q.bytesPerDim == bytesPerDim &&
-        Arrays.equals(point, q.point);
-    }
-
-    return false;
-  }
-
-  @Override
-  public String toString(String field) {
-    final StringBuilder sb = new StringBuilder();
-    sb.append(getClass().getSimpleName());
-    sb.append(':');
-    if (this.field.equals(field) == false) {
-      sb.append("field=");
-      sb.append(this.field);
-      sb.append(':');
-    }
-
-    return sb.append(" point=")
-      .append(Arrays.toString(point))
-      .toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 58f6d07..949cffe 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
@@ -24,18 +24,22 @@ import java.util.Objects;
 import org.apache.lucene.index.PointValues;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
 import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.document.BinaryPoint;
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.LongPoint;
 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.RamUsageEstimator;
 import org.apache.lucene.util.StringHelper;
 
 /** Searches for ranges in fields previously indexed using points e.g.
  *  {@link org.apache.lucene.document.LongPoint}.  In a 1D field this is
  *  a simple range query; in a multi-dimensional field it's a box shape. */
-
+// TODO: enhance this and add simple example
 public class PointRangeQuery extends Query {
   final String field;
   final int numDims;
@@ -46,16 +50,37 @@ public class PointRangeQuery extends Query {
   // This is null only in the "fully open range" case
   final Integer bytesPerDim;
 
-  public PointRangeQuery(String field,
+  /** 
+   * Expert: create a multidimensional range query for point values.
+   * <p>
+   * This is for subclasses and works on the underlying binary encoding: to
+   * create range queries for lucene's standard {@code Point} types, refer to these factory methods:
+   * <ul>
+   *   <li>{@link #newIntRange newIntRange()}/{@link #newMultiIntRange newMultiIntRange()} for fields indexed with {@link IntPoint}
+   *   <li>{@link #newIntRange newLongRange()}/{@link #newMultiIntRange newMultiLongRange()} for fields indexed with {@link LongPoint}
+   *   <li>{@link #newIntRange newFloatRange()}/{@link #newMultiIntRange newMultiFloatRange()} for fields indexed with {@link FloatPoint}
+   *   <li>{@link #newIntRange newDoubleRange()}/{@link #newMultiIntRange newMultiDoubleRange()} for fields indexed with {@link DoublePoint}
+   *   <li>{@link #newIntRange newBinaryRange()}/{@link #newMultiIntRange newMultiBinaryRange()} for fields indexed with {@link BinaryPoint}
+   * </ul>
+   * <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.
+   * @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) {
+    checkArgs(field, lowerPoint, upperPoint);
     this.field = field;
-    if (lowerPoint == null) {
-      throw new IllegalArgumentException("lowerPoint must not be null");
-    }
-    if (upperPoint == null) {
-      throw new IllegalArgumentException("upperPoint must not be null");
-    }
     numDims = lowerPoint.length;
     if (upperPoint.length != numDims) {
       throw new IllegalArgumentException("lowerPoint has length=" + numDims + " but upperPoint has different length=" + upperPoint.length);
@@ -91,69 +116,361 @@ public class PointRangeQuery extends Query {
     }
   }
 
-  /** Use in the 1D case when you indexed 1D int values using {@link org.apache.lucene.document.IntPoint} */
-  public static PointRangeQuery new1DIntRange(String field, Integer lowerValue, boolean lowerInclusive, Integer upperValue, boolean upperInclusive) {
-    return new PointRangeQuery(field, pack(lowerValue), new boolean[] {lowerInclusive}, pack(upperValue), new boolean[] {upperInclusive});
+  /** Check preconditions for all factory methods */
+  private static void checkArgs(String field, Object lowerPoint, Object upperPoint) {
+    if (field == null) {
+      throw new IllegalArgumentException("field must not be null");
+    }
+    if (lowerPoint == null) {
+      throw new IllegalArgumentException("lowerPoint must not be null");
+    }
+    if (upperPoint == null) {
+      throw new IllegalArgumentException("upperPoint must not be null");
+    }
   }
 
-  /** Use in the 1D case when you indexed 1D long values using {@link org.apache.lucene.document.LongPoint} */
-  public static PointRangeQuery new1DLongRange(String field, Long lowerValue, boolean lowerInclusive, Long upperValue, boolean upperInclusive) {
-    return new PointRangeQuery(field, pack(lowerValue), new boolean[] {lowerInclusive}, pack(upperValue), new boolean[] {upperInclusive});
+  /** 
+   * Create a range query for matching an exact integer value.
+   * <p>
+   * This is for simple one-dimension points, for multidimensional points use
+   * {@link #newMultiIntRange newMultiIntRange()} instead.
+   *
+   * @param field field name. must not be {@code null}.
+   * @param value exact value
+   * @throws IllegalArgumentException if {@code field} is null.
+   * @return a query matching documents with this exact value
+   */
+  public static PointRangeQuery newIntExact(String field, int value) {
+    return newIntRange(field, value, true, value, true);
   }
 
-  /** Use in the 1D case when you indexed 1D float values using {@link org.apache.lucene.document.FloatPoint} */
-  public static PointRangeQuery new1DFloatRange(String field, Float lowerValue, boolean lowerInclusive, Float upperValue, boolean upperInclusive) {
-    return new PointRangeQuery(field, pack(lowerValue), new boolean[] {lowerInclusive}, pack(upperValue), new boolean[] {upperInclusive});
+  /** 
+   * Create a range query for integer values indexed with {@link IntPoint}.
+   * <p>
+   * This is for simple one-dimension ranges, for multidimensional ranges use
+   * {@link #newMultiIntRange newMultiIntRange()} 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.
+   *
+   * @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.
+   * @return a query matching documents within this range.
+   */
+  public static PointRangeQuery newIntRange(String field, Integer lowerValue, boolean lowerInclusive, Integer upperValue, boolean upperInclusive) {
+    return newMultiIntRange(field, 
+                            new Integer[] { lowerValue },
+                            new boolean[] { lowerInclusive }, 
+                            new Integer[] { upperValue },
+                            new boolean[] { upperInclusive });
   }
 
-  /** Use in the 1D case when you indexed 1D double values using {@link org.apache.lucene.document.DoublePoint} */
-  public static PointRangeQuery new1DDoubleRange(String field, Double lowerValue, boolean lowerInclusive, Double upperValue, boolean upperInclusive) {
-    return new PointRangeQuery(field, pack(lowerValue), new boolean[] {lowerInclusive}, pack(upperValue), new boolean[] {upperInclusive});
+  /** 
+   * Create a multidimensional range query for integer values indexed with {@link IntPoint}.
+   * <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 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}
+   * @return a query matching documents within this range.
+   */
+  public static PointRangeQuery newMultiIntRange(String field, Integer[] lowerValue, boolean lowerInclusive[], Integer[] upperValue, boolean upperInclusive[]) {
+    checkArgs(field, lowerValue, upperValue);
+    return new PointRangeQuery(field, IntPoint.encode(lowerValue), lowerInclusive, IntPoint.encode(upperValue), upperInclusive) {
+      @Override
+      protected String toString(byte[] value) {
+        return IntPoint.decodeDimension(value).toString();
+      }
+    };
   }
 
-  /** Use in the 1D case when you indexed binary values using {@link org.apache.lucene.document.BinaryPoint} */
-  public static PointRangeQuery new1DBinaryRange(String field, byte[] lowerValue, boolean lowerInclusive, byte[] upperValue, boolean upperInclusive) {
-    return new PointRangeQuery(field, new byte[][] {lowerValue}, new boolean[] {lowerInclusive}, new byte[][] {upperValue}, new boolean[] {upperInclusive});
+  /** 
+   * Create a range query for matching an exact long value.
+   * <p>
+   * This is for simple one-dimension points, for multidimensional points use
+   * {@link #newMultiLongRange newMultiLongRange()} instead.
+   *
+   * @param field field name. must not be {@code null}.
+   * @param value exact value
+   * @throws IllegalArgumentException if {@code field} is null.
+   * @return a query matching documents with this exact value
+   */
+  public static PointRangeQuery newLongExact(String field, long value) {
+    return newLongRange(field, value, true, value, true);
   }
 
-  private static byte[][] pack(Long value) {
-    if (value == null) {
-      // OK: open ended range
-      return new byte[1][];
-    }
-    byte[][] result = new byte[][] {new byte[RamUsageEstimator.NUM_BYTES_LONG]};
-    NumericUtils.longToBytes(value, result[0], 0);
-    return result;
+  /** 
+   * Create a range query for long values indexed with {@link LongPoint}.
+   * <p>
+   * This is for simple one-dimension ranges, for multidimensional ranges use
+   * {@link #newMultiLongRange newMultiLongRange()} 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.
+   *
+   * @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.
+   * @return a query matching documents within this range.
+   */
+  public static PointRangeQuery newLongRange(String field, Long lowerValue, boolean lowerInclusive, Long upperValue, boolean upperInclusive) {
+    return newMultiLongRange(field, 
+                             new Long[] { lowerValue },
+                             new boolean[] { lowerInclusive }, 
+                             new Long[] { upperValue },
+                             new boolean[] { upperInclusive });
   }
 
-  private static byte[][] pack(Double value) {
-    if (value == null) {
-      // OK: open ended range
-      return new byte[1][];
-    }
-    byte[][] result = new byte[][] {new byte[RamUsageEstimator.NUM_BYTES_LONG]};
-    NumericUtils.longToBytesDirect(NumericUtils.doubleToSortableLong(value), result[0], 0);
-    return result;
+  /** 
+   * Create a multidimensional range query for long values indexed with {@link LongPoint}.
+   * <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 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}
+   * @return a query matching documents within this range.
+   */
+  public static PointRangeQuery newMultiLongRange(String field, Long[] lowerValue, boolean lowerInclusive[], Long[] upperValue, boolean upperInclusive[]) {
+    checkArgs(field, lowerValue, upperValue);
+    return new PointRangeQuery(field, LongPoint.encode(lowerValue), lowerInclusive, LongPoint.encode(upperValue), upperInclusive) {
+      @Override
+      protected String toString(byte[] value) {
+        return LongPoint.decodeDimension(value).toString();
+      }
+    };
   }
 
-  private static byte[][] pack(Integer value) {
-    if (value == null) {
-      // OK: open ended range
-      return new byte[1][];
-    }
-    byte[][] result = new byte[][] {new byte[RamUsageEstimator.NUM_BYTES_INT]};
-    NumericUtils.intToBytes(value, result[0], 0);
-    return result;
+  /** 
+   * Create a range query for matching an exact float value.
+   * <p>
+   * This is for simple one-dimension points, for multidimensional points use
+   * {@link #newMultiFloatRange newMultiFloatRange()} instead.
+   *
+   * @param field field name. must not be {@code null}.
+   * @param value float value
+   * @throws IllegalArgumentException if {@code field} is null.
+   * @return a query matching documents with this exact value
+   */
+  public static PointRangeQuery newFloatExact(String field, float value) {
+    return newFloatRange(field, value, true, value, true);
+  }
+  
+  /** 
+   * Create a range query for float values indexed with {@link FloatPoint}.
+   * <p>
+   * This is for simple one-dimension ranges, for multidimensional ranges use
+   * {@link #newMultiFloatRange newMultiFloatRange()} 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.
+   *
+   * @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.
+   * @return a query matching documents within this range.
+   */
+  public static PointRangeQuery newFloatRange(String field, Float lowerValue, boolean lowerInclusive, Float upperValue, boolean upperInclusive) {
+    return newMultiFloatRange(field, 
+                            new Float[] { lowerValue },
+                            new boolean[] { lowerInclusive }, 
+                            new Float[] { upperValue },
+                            new boolean[] { upperInclusive });
   }
 
-  private static byte[][] pack(Float value) {
+  /** 
+   * Create a multidimensional range query for float values indexed with {@link FloatPoint}.
+   * <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 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}
+   * @return a query matching documents within this range.
+   */
+  public static PointRangeQuery newMultiFloatRange(String field, Float[] lowerValue, boolean lowerInclusive[], Float[] upperValue, boolean upperInclusive[]) {
+    checkArgs(field, lowerValue, upperValue);
+    return new PointRangeQuery(field, FloatPoint.encode(lowerValue), lowerInclusive, FloatPoint.encode(upperValue), upperInclusive) {
+      @Override
+      protected String toString(byte[] value) {
+        return FloatPoint.decodeDimension(value).toString();
+      }
+    };
+  }
+
+  /** 
+   * Create a range query for matching an exact double value.
+   * <p>
+   * This is for simple one-dimension points, for multidimensional points use
+   * {@link #newMultiDoubleRange newMultiDoubleRange()} instead.
+   *
+   * @param field field name. must not be {@code null}.
+   * @param value double value
+   * @throws IllegalArgumentException if {@code field} is null.
+   * @return a query matching documents with this exact value
+   */
+  public static PointRangeQuery newDoubleExact(String field, double value) {
+    return newDoubleRange(field, value, true, value, true);
+  }
+  
+  /** 
+   * Create a range query for double values indexed with {@link DoublePoint}.
+   * <p>
+   * This is for simple one-dimension ranges, for multidimensional ranges use
+   * {@link #newMultiDoubleRange newMultiDoubleRange()} 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.
+   *
+   * @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.
+   * @return a query matching documents within this range.
+   */
+  public static PointRangeQuery newDoubleRange(String field, Double lowerValue, boolean lowerInclusive, Double upperValue, boolean upperInclusive) {
+    return newMultiDoubleRange(field, 
+                            new Double[] { lowerValue },
+                            new boolean[] { lowerInclusive }, 
+                            new Double[] { upperValue },
+                            new boolean[] { upperInclusive });
+  }
+
+  /** 
+   * Create a multidimensional range query for double values indexed with {@link DoublePoint}.
+   * <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 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}
+   * @return a query matching documents within this range.
+   */
+  public static PointRangeQuery newMultiDoubleRange(String field, Double[] lowerValue, boolean lowerInclusive[], Double[] upperValue, boolean upperInclusive[]) {
+    checkArgs(field, lowerValue, upperValue);
+    return new PointRangeQuery(field, DoublePoint.encode(lowerValue), lowerInclusive, DoublePoint.encode(upperValue), upperInclusive) {
+      @Override
+      protected String toString(byte[] value) {
+        return DoublePoint.decodeDimension(value).toString();
+      }
+    };
+  }
+
+  /** 
+   * Create a range query for matching an exact binary value.
+   * <p>
+   * This is for simple one-dimension points, for multidimensional points use
+   * {@link #newMultiBinaryRange newMultiBinaryRange()} instead.
+   *
+   * @param field field name. must not be {@code null}.
+   * @param value binary value
+   * @throws IllegalArgumentException if {@code field} is null or {@code value} is null
+   * @return a query matching documents with this exact value
+   */
+  public static PointRangeQuery newBinaryExact(String field, byte[] value) {
     if (value == null) {
-      // OK: open ended range
-      return new byte[1][];
+      throw new IllegalArgumentException("value cannot be null");
     }
-    byte[][] result = new byte[][] {new byte[RamUsageEstimator.NUM_BYTES_INT]};
-    NumericUtils.intToBytesDirect(NumericUtils.floatToSortableInt(value), result[0], 0);
-    return result;
+    return newBinaryRange(field, value, true, value, true);
+  }
+
+  /** 
+   * Create a range query for binary values indexed with {@link BinaryPoint}.
+   * <p>
+   * This is for simple one-dimension ranges, for multidimensional ranges use
+   * {@link #newMultiBinaryRange newMultiBinaryRange()} 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.
+   *
+   * @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.
+   * @return a query matching documents within this range.
+   */
+  public static PointRangeQuery newBinaryRange(String field, byte[] lowerValue, boolean lowerInclusive, byte[] upperValue, boolean upperInclusive) {
+    return newMultiBinaryRange(field, new byte[][] {lowerValue}, new boolean[] {lowerInclusive}, new byte[][] {upperValue}, new boolean[] {upperInclusive});
+  }
+  
+  /** 
+   * Create a multidimensional range query for binary values indexed with {@link BinaryPoint}.
+   * <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 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}
+   * @return a query matching documents within this range.
+   */
+  public static PointRangeQuery newMultiBinaryRange(String field, byte[][] lowerValue, boolean[] lowerInclusive, byte[][] upperValue, boolean[] upperInclusive) {
+    checkArgs(field, lowerValue, upperValue);
+    return new PointRangeQuery(field, lowerValue, lowerInclusive, upperValue, upperInclusive);
   }
 
   @Override
@@ -330,19 +647,67 @@ public class PointRangeQuery extends Query {
   @Override
   public String toString(String field) {
     final StringBuilder sb = new StringBuilder();
-    sb.append(getClass().getSimpleName());
-    sb.append(':');
     if (this.field.equals(field) == false) {
-      sb.append("field=");
       sb.append(this.field);
       sb.append(':');
     }
 
-    return sb.append('[')
-      .append(Arrays.toString(lowerPoint))
-      .append(" TO ")
-      .append(Arrays.toString(upperPoint))
-      .append(']')
-      .toString();
+    // print ourselves as "range per dimension"
+    for (int i = 0; i < numDims; i++) {
+      if (i > 0) {
+        sb.append(',');
+      }
+
+      if (lowerInclusive[i]) {
+        sb.append('[');
+      } else {
+        sb.append('{');
+      }
+
+      if (lowerPoint[i] == null) {
+        sb.append('*');
+      } else {
+        sb.append(toString(lowerPoint[i]));
+      }
+
+      sb.append(" TO ");
+
+      if (upperPoint[i] == null) {
+        sb.append('*');
+      } else {
+        sb.append(toString(upperPoint[i]));
+      }
+
+      if (upperInclusive[i]) {
+        sb.append(']');
+      } else {
+        sb.append('}');
+      }
+    }
+
+    return sb.toString();
+  }
+
+  /**
+   * Returns a string of a single value in a human-readable format for debugging.
+   * This is used by {@link #toString()}.
+   *
+   * The default implementation encodes the individual byte values.
+   *
+   * @param value single value, never null
+   * @return human readable value for debugging
+   */
+  protected String toString(byte[] value) {
+    assert value != null;
+    StringBuilder sb = new StringBuilder();
+    sb.append("binary(");
+    for (int i = 0; i < value.length; i++) {
+      if (i > 0) {
+        sb.append(' ');
+      }
+      sb.append(Integer.toHexString(value[i] & 0xFF));
+    }
+    sb.append(')');
+    return sb.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 59bf1c5..9a74231 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
@@ -1352,7 +1352,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
         max = x;
       }
 
-      TopDocs hits = s.search(PointRangeQuery.new1DLongRange("number", min, true, max, true), 100);
+      TopDocs hits = s.search(PointRangeQuery.newLongRange("number", min, true, max, true), 100);
       for(ScoreDoc scoreDoc : hits.scoreDocs) {
         long value = Long.parseLong(s.doc(scoreDoc.doc).get("text").split(" ")[1]);
         assertTrue(value >= min);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 5758546..cfab5fb 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -56,7 +56,6 @@ import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
@@ -305,7 +304,7 @@ public class TestPointQueries extends LuceneTestCase {
               }
 
               if (random().nextBoolean()) {
-                query = PointRangeQuery.new1DLongRange("sn_value", lower, includeLower, upper, includeUpper);
+                query = PointRangeQuery.newLongRange("sn_value", lower, includeLower, upper, includeUpper);
               } else {
                 byte[] lowerBytes;
                 if (lower == null) {
@@ -321,7 +320,7 @@ public class TestPointQueries extends LuceneTestCase {
                   upperBytes = new byte[8];
                   NumericUtils.longToBytes(upper, upperBytes, 0);
                 }
-                query = PointRangeQuery.new1DBinaryRange("ss_value", lowerBytes, includeLower, upperBytes, includeUpper);
+                query = PointRangeQuery.newBinaryRange("ss_value", lowerBytes, includeLower, upperBytes, includeUpper);
               }
 
               if (VERBOSE) {
@@ -737,9 +736,9 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexSearcher s = newSearcher(r);
 
-    assertEquals(1, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, 0L, true)));
-    assertEquals(1, s.count(PointRangeQuery.new1DLongRange("value", 0L, true, Long.MAX_VALUE, true)));
-    assertEquals(2, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
+    assertEquals(1, s.count(PointRangeQuery.newLongRange("value", Long.MIN_VALUE, true, 0L, true)));
+    assertEquals(1, s.count(PointRangeQuery.newLongRange("value", 0L, true, Long.MAX_VALUE, true)));
+    assertEquals(2, s.count(PointRangeQuery.newLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
 
     IOUtils.close(r, w, dir);
   }
@@ -775,47 +774,47 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexSearcher s = newSearcher(r);
 
-    assertEquals(1, s.count(PointRangeQuery.new1DBinaryRange("value",
+    assertEquals(1, s.count(PointRangeQuery.newBinaryRange("value",
         toUTF8("aaa"),
         true,
         toUTF8("bbb"),
         true)));
-    assertEquals(1, s.count(PointRangeQuery.new1DBinaryRange("value",
+    assertEquals(1, s.count(PointRangeQuery.newBinaryRange("value",
         toUTF8("c", 3),
         true,
         toUTF8("e", 3),
         true)));
-    assertEquals(2, s.count(PointRangeQuery.new1DBinaryRange("value",
+    assertEquals(2, s.count(PointRangeQuery.newBinaryRange("value",
         toUTF8("a", 3),
         true,
         toUTF8("z", 3),
         true)));
-    assertEquals(1, s.count(PointRangeQuery.new1DBinaryRange("value",
+    assertEquals(1, s.count(PointRangeQuery.newBinaryRange("value",
         null,
         true,
         toUTF8("abc"),
         true)));
-    assertEquals(1, s.count(PointRangeQuery.new1DBinaryRange("value",
+    assertEquals(1, s.count(PointRangeQuery.newBinaryRange("value",
         toUTF8("a", 3),
         true,
         toUTF8("abc"),
         true)));
-    assertEquals(0, s.count(PointRangeQuery.new1DBinaryRange("value",
+    assertEquals(0, s.count(PointRangeQuery.newBinaryRange("value",
         toUTF8("a", 3),
         true,
         toUTF8("abc"),
         false)));
-    assertEquals(1, s.count(PointRangeQuery.new1DBinaryRange("value",
+    assertEquals(1, s.count(PointRangeQuery.newBinaryRange("value",
         toUTF8("def"),
         true,
         null,
         false)));
-    assertEquals(1, s.count(PointRangeQuery.new1DBinaryRange("value",
+    assertEquals(1, s.count(PointRangeQuery.newBinaryRange("value",
         toUTF8(("def")),
         true,
         toUTF8("z", 3),
         true)));
-    assertEquals(0, s.count(PointRangeQuery.new1DBinaryRange("value",
+    assertEquals(0, s.count(PointRangeQuery.newBinaryRange("value",
         toUTF8("def"),
         false,
         toUTF8("z", 3),
@@ -840,12 +839,12 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexSearcher s = newSearcher(r);
 
-    assertEquals(2, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
-    assertEquals(1, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, false)));
-    assertEquals(1, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, true)));
-    assertEquals(0, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, false)));
+    assertEquals(2, s.count(PointRangeQuery.newLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
+    assertEquals(1, s.count(PointRangeQuery.newLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, false)));
+    assertEquals(1, s.count(PointRangeQuery.newLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, true)));
+    assertEquals(0, s.count(PointRangeQuery.newLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, false)));
 
-    assertEquals(2, s.count(PointRangeQuery.new1DBinaryRange("value", (byte[]) null, true, null, true)));
+    assertEquals(2, s.count(PointRangeQuery.newBinaryRange("value", (byte[]) null, true, null, true)));
 
     IOUtils.close(r, w, dir);
   }
@@ -867,12 +866,12 @@ public class TestPointQueries extends LuceneTestCase {
     // We can't wrap with "exotic" readers because the query must see the RangeTreeDVFormat:
     IndexSearcher s = newSearcher(r, false);
 
-    assertEquals(2, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
-    assertEquals(1, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, false)));
-    assertEquals(1, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, true)));
-    assertEquals(0, s.count(PointRangeQuery.new1DLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, false)));
+    assertEquals(2, s.count(PointRangeQuery.newLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, true)));
+    assertEquals(1, s.count(PointRangeQuery.newLongRange("value", Long.MIN_VALUE, true, Long.MAX_VALUE, false)));
+    assertEquals(1, s.count(PointRangeQuery.newLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, true)));
+    assertEquals(0, s.count(PointRangeQuery.newLongRange("value", Long.MIN_VALUE, false, Long.MAX_VALUE, false)));
 
-    assertEquals(2, s.count(PointRangeQuery.new1DLongRange("value", (Long) null, true, null, true)));
+    assertEquals(2, s.count(PointRangeQuery.newLongRange("value", (Long) null, true, null, true)));
 
     IOUtils.close(r, w, dir);
   }
@@ -892,9 +891,9 @@ public class TestPointQueries extends LuceneTestCase {
     IndexReader r = w.getReader();
 
     IndexSearcher s = newSearcher(r);
-    assertEquals(0, s.count(PointRangeQuery.new1DBinaryRange("value", toUTF8("m"), true, toUTF8("n"), false)));
+    assertEquals(0, s.count(PointRangeQuery.newBinaryRange("value", toUTF8("m"), true, toUTF8("n"), false)));
 
-    assertEquals(2, s.count(PointRangeQuery.new1DBinaryRange("value", (byte[]) null, true, null, true)));
+    assertEquals(2, s.count(PointRangeQuery.newBinaryRange("value", (byte[]) null, true, null, true)));
 
     IOUtils.close(r, w, dir);
   }
@@ -914,7 +913,7 @@ public class TestPointQueries extends LuceneTestCase {
     IndexReader r = w.getReader();
 
     IndexSearcher s = new IndexSearcher(r);
-    assertEquals(0, s.count(PointRangeQuery.new1DLongRange("value", 17L, true, 13L, false)));
+    assertEquals(0, s.count(PointRangeQuery.newLongRange("value", 17L, true, 13L, false)));
 
     IOUtils.close(r, w, dir);
   }
@@ -929,7 +928,7 @@ public class TestPointQueries extends LuceneTestCase {
     IndexReader r = w.getReader();
 
     IndexSearcher s = newSearcher(r);
-    assertEquals(0, s.count(PointRangeQuery.new1DLongRange("value", 17L, true, 13L, false)));
+    assertEquals(0, s.count(PointRangeQuery.newLongRange("value", 17L, true, 13L, false)));
 
     IOUtils.close(r, w, dir);
   }
@@ -1045,7 +1044,7 @@ public class TestPointQueries extends LuceneTestCase {
     }
   }
 
-  public void testExactPointQuery() throws Exception {
+  public void testExactPoints() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setCodec(getCodec());
@@ -1069,19 +1068,50 @@ public class TestPointQueries extends LuceneTestCase {
 
     IndexReader r = DirectoryReader.open(w);
     IndexSearcher s = newSearcher(r);
-    assertEquals(1, s.count(ExactPointQuery.new1DIntExact("int", 42)));
-    assertEquals(0, s.count(ExactPointQuery.new1DIntExact("int", 41)));
+    assertEquals(1, s.count(PointRangeQuery.newIntExact("int", 42)));
+    assertEquals(0, s.count(PointRangeQuery.newIntExact("int", 41)));
 
-    assertEquals(1, s.count(ExactPointQuery.new1DLongExact("long", 5L)));
-    assertEquals(0, s.count(ExactPointQuery.new1DLongExact("long", -1L)));
+    assertEquals(1, s.count(PointRangeQuery.newLongExact("long", 5L)));
+    assertEquals(0, s.count(PointRangeQuery.newLongExact("long", -1L)));
 
-    assertEquals(1, s.count(ExactPointQuery.new1DFloatExact("float", 2.0f)));
-    assertEquals(0, s.count(ExactPointQuery.new1DFloatExact("float", 1.0f)));
+    assertEquals(1, s.count(PointRangeQuery.newFloatExact("float", 2.0f)));
+    assertEquals(0, s.count(PointRangeQuery.newFloatExact("float", 1.0f)));
 
-    assertEquals(1, s.count(ExactPointQuery.new1DDoubleExact("double", 1.0)));
-    assertEquals(0, s.count(ExactPointQuery.new1DDoubleExact("double", 2.0)));
+    assertEquals(1, s.count(PointRangeQuery.newDoubleExact("double", 1.0)));
+    assertEquals(0, s.count(PointRangeQuery.newDoubleExact("double", 2.0)));
     w.close();
     r.close();
     dir.close();
   }
+
+  public void testToString() throws Exception {
+    
+    // ints
+    assertEquals("field:[1 TO 2}", PointRangeQuery.newIntRange("field", 1, true, 2, false).toString());
+    assertEquals("field:{-2 TO 1]", PointRangeQuery.newIntRange("field", -2, false, 1, true).toString());
+    assertEquals("field:[* TO 2}", PointRangeQuery.newIntRange("field", null, true, 2, false).toString());
+
+    // longs
+    assertEquals("field:[1099511627776 TO 2199023255552}", PointRangeQuery.newLongRange("field", 1L<<40, true, 1L<<41, false).toString());
+    assertEquals("field:{-5 TO 6]", PointRangeQuery.newLongRange("field", -5L, false, 6L, true).toString());
+    assertEquals("field:[* TO 2}", PointRangeQuery.newLongRange("field", null, true, 2L, false).toString());
+    
+    // floats
+    assertEquals("field:[1.3 TO 2.5}", PointRangeQuery.newFloatRange("field", 1.3F, true, 2.5F, false).toString());
+    assertEquals("field:{-2.9 TO 1.0]", PointRangeQuery.newFloatRange("field", -2.9F, false, 1.0F, true).toString());
+    assertEquals("field:{-2.9 TO *]", PointRangeQuery.newFloatRange("field", -2.9F, false, null, true).toString());
+    
+    // doubles
+    assertEquals("field:[1.3 TO 2.5}", PointRangeQuery.newDoubleRange("field", 1.3, true, 2.5, false).toString());
+    assertEquals("field:{-2.9 TO 1.0]", PointRangeQuery.newDoubleRange("field", -2.9, false, 1.0, true).toString());
+    assertEquals("field:{-2.9 TO *]", PointRangeQuery.newDoubleRange("field", -2.9, false, null, true).toString());
+    
+    // n-dimensional double
+    assertEquals("field:[1.3 TO 2.5},{-2.9 TO 1.0]", PointRangeQuery.newMultiDoubleRange("field", 
+                                                                      new Double[] { 1.3, -2.9 }, 
+                                                                      new boolean[] { true, false }, 
+                                                                      new Double[] { 2.5, 1.0 },
+                                                                      new boolean[] { false, true }).toString());
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 33c9458..323351e 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(PointRangeQuery.new1DIntRange("intField", 1, true, 1000, true)));
+    assertTrue(UsageTrackingQueryCachingPolicy.isCostly(PointRangeQuery.newIntRange("intField", 1, true, 1000, true)));
     assertFalse(UsageTrackingQueryCachingPolicy.isCostly(new TermQuery(new Term("field", "value"))));
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 bf3f185..083e41c 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
@@ -181,7 +181,7 @@ public class DistanceFacetsExample implements Closeable {
     BooleanQuery.Builder f = new BooleanQuery.Builder();
 
     // Add latitude range filter:
-    f.add(PointRangeQuery.new1DDoubleRange("latitude", Math.toDegrees(minLat), true, Math.toDegrees(maxLat), true),
+    f.add(PointRangeQuery.newDoubleRange("latitude", Math.toDegrees(minLat), true, Math.toDegrees(maxLat), true),
           BooleanClause.Occur.FILTER);
 
     // Add longitude range filter:
@@ -189,13 +189,13 @@ public class DistanceFacetsExample implements Closeable {
       // The bounding box crosses the international date
       // line:
       BooleanQuery.Builder lonF = new BooleanQuery.Builder();
-      lonF.add(PointRangeQuery.new1DDoubleRange("longitude", Math.toDegrees(minLng), true, null, true),
+      lonF.add(PointRangeQuery.newDoubleRange("longitude", Math.toDegrees(minLng), true, null, true),
                BooleanClause.Occur.SHOULD);
-      lonF.add(PointRangeQuery.new1DDoubleRange("longitude", null, true, Math.toDegrees(maxLng), true),
+      lonF.add(PointRangeQuery.newDoubleRange("longitude", null, true, Math.toDegrees(maxLng), true),
                BooleanClause.Occur.SHOULD);
       f.add(lonF.build(), BooleanClause.Occur.MUST);
     } else {
-      f.add(PointRangeQuery.new1DDoubleRange("longitude", Math.toDegrees(minLng), true, Math.toDegrees(maxLng), true),
+      f.add(PointRangeQuery.newDoubleRange("longitude", Math.toDegrees(minLng), true, Math.toDegrees(maxLng), true),
             BooleanClause.Occur.FILTER);
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 cbc19b8..1a7a984 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
@@ -107,7 +107,7 @@ public class RangeFacetsExample implements Closeable {
     // documents ("browse only"):
     DrillDownQuery q = new DrillDownQuery(getConfig());
 
-    q.add("timestamp", PointRangeQuery.new1DLongRange("timestamp", range.min, range.minInclusive, range.max, range.maxInclusive));
+    q.add("timestamp", PointRangeQuery.newLongRange("timestamp", range.min, range.minInclusive, range.max, range.maxInclusive));
 
     return searcher.search(q, 10);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 7d154b3..87f0579 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
@@ -281,7 +281,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
 
     // Third search, drill down on "less than or equal to 10":
     ddq = new DrillDownQuery(config);
-    ddq.add("field", PointRangeQuery.new1DLongRange("field", 0L, true, 10L, true));
+    ddq.add("field", PointRangeQuery.newLongRange("field", 0L, true, 10L, true));
     dsr = ds.search(null, ddq, 10);
 
     assertEquals(11, dsr.hits.totalHits);
@@ -461,9 +461,9 @@ public class TestRangeFacetCounts extends FacetTestCase {
       Query fastMatchQuery;
       if (random().nextBoolean()) {
         if (random().nextBoolean()) {
-          fastMatchQuery = PointRangeQuery.new1DLongRange("field", minValue, true, maxValue, true);
+          fastMatchQuery = PointRangeQuery.newLongRange("field", minValue, true, maxValue, true);
         } else {
-          fastMatchQuery = PointRangeQuery.new1DLongRange("field", minAcceptedValue, true, maxAcceptedValue, true);
+          fastMatchQuery = PointRangeQuery.newLongRange("field", minAcceptedValue, true, maxAcceptedValue, true);
         }
       } else {
         fastMatchQuery = null;
@@ -485,7 +485,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
         // Test drill-down:
         DrillDownQuery ddq = new DrillDownQuery(config);
         if (random().nextBoolean()) {
-          ddq.add("field", PointRangeQuery.new1DLongRange("field", range.min, range.minInclusive, range.max, range.maxInclusive));
+          ddq.add("field", PointRangeQuery.newLongRange("field", range.min, range.minInclusive, range.max, range.maxInclusive));
         } else {
           ddq.add("field", range.getQuery(fastMatchQuery, vs));
         }
@@ -616,9 +616,9 @@ public class TestRangeFacetCounts extends FacetTestCase {
       Query fastMatchQuery;
       if (random().nextBoolean()) {
         if (random().nextBoolean()) {
-          fastMatchQuery = PointRangeQuery.new1DFloatRange("field", minValue, true, maxValue, true);
+          fastMatchQuery = PointRangeQuery.newFloatRange("field", minValue, true, maxValue, true);
         } else {
-          fastMatchQuery = PointRangeQuery.new1DFloatRange("field", minAcceptedValue, true, maxAcceptedValue, true);
+          fastMatchQuery = PointRangeQuery.newFloatRange("field", minAcceptedValue, true, maxAcceptedValue, true);
         }
       } else {
         fastMatchQuery = null;
@@ -640,7 +640,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
         // Test drill-down:
         DrillDownQuery ddq = new DrillDownQuery(config);
         if (random().nextBoolean()) {
-          ddq.add("field", PointRangeQuery.new1DFloatRange("field", (float) range.min, range.minInclusive, (float) range.max, range.maxInclusive));
+          ddq.add("field", PointRangeQuery.newFloatRange("field", (float) range.min, range.minInclusive, (float) range.max, range.maxInclusive));
         } else {
           ddq.add("field", range.getQuery(fastMatchQuery, vs));
         }
@@ -755,9 +755,9 @@ public class TestRangeFacetCounts extends FacetTestCase {
       Query fastMatchFilter;
       if (random().nextBoolean()) {
         if (random().nextBoolean()) {
-          fastMatchFilter = PointRangeQuery.new1DDoubleRange("field", minValue, true, maxValue, true);
+          fastMatchFilter = PointRangeQuery.newDoubleRange("field", minValue, true, maxValue, true);
         } else {
-          fastMatchFilter = PointRangeQuery.new1DDoubleRange("field", minAcceptedValue, true, maxAcceptedValue, true);
+          fastMatchFilter = PointRangeQuery.newDoubleRange("field", minAcceptedValue, true, maxAcceptedValue, true);
         }
       } else {
         fastMatchFilter = null;
@@ -779,7 +779,7 @@ public class TestRangeFacetCounts extends FacetTestCase {
         // Test drill-down:
         DrillDownQuery ddq = new DrillDownQuery(config);
         if (random().nextBoolean()) {
-          ddq.add("field", PointRangeQuery.new1DDoubleRange("field", range.min, range.minInclusive, range.max, range.maxInclusive));
+          ddq.add("field", PointRangeQuery.newDoubleRange("field", range.min, range.minInclusive, range.max, range.maxInclusive));
         } else {
           ddq.add("field", range.getQuery(fastMatchFilter, vs));
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 26ad755..58f1eb9 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
@@ -584,7 +584,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 = PointRangeQuery.new1DIntRange(NUMERIC_FIELD_NAME, 2, true, 6, true);
+    query = PointRangeQuery.newIntRange(NUMERIC_FIELD_NAME, 2, true, 6, true);
     searcher = newSearcher(reader);
     hits = searcher.search(query, 100);
     int maxNumFragmentsRequired = 2;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 7960ce0..7e23aad 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesRangeQuery.java
@@ -57,7 +57,7 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
         iw.addDocument(doc);
       }
       if (random().nextBoolean()) {
-        iw.deleteDocuments(PointRangeQuery.new1DLongRange("idx", 0L, true, 10L, true));
+        iw.deleteDocuments(PointRangeQuery.newLongRange("idx", 0L, true, 10L, true));
       }
       iw.commit();
       final IndexReader reader = iw.getReader();
@@ -69,7 +69,7 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
         final Long max = random().nextBoolean() ? null : TestUtil.nextLong(random(), -100, 1000);
         final boolean minInclusive = random().nextBoolean();
         final boolean maxInclusive = random().nextBoolean();
-        final Query q1 = PointRangeQuery.new1DLongRange("idx", min, minInclusive, max, maxInclusive);
+        final Query q1 = PointRangeQuery.newLongRange("idx", min, minInclusive, max, maxInclusive);
         final Query q2 = DocValuesRangeQuery.newLongRange("dv", min, max, minInclusive, maxInclusive);
         assertSameMatches(searcher, q1, q2, false);
       }
@@ -185,7 +185,7 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
       iw.addDocument(doc);
     }
     if (random().nextBoolean()) {
-      iw.deleteDocuments(PointRangeQuery.new1DLongRange("idx", 0L, true, 10L, true));
+      iw.deleteDocuments(PointRangeQuery.newLongRange("idx", 0L, true, 10L, true));
     }
     iw.commit();
     final IndexReader reader = iw.getReader();
@@ -199,7 +199,7 @@ public class TestDocValuesRangeQuery extends LuceneTestCase {
       final boolean maxInclusive = random().nextBoolean();
 
       BooleanQuery.Builder ref = new BooleanQuery.Builder();
-      ref.add(PointRangeQuery.new1DLongRange("idx", min, minInclusive, max, maxInclusive), Occur.FILTER);
+      ref.add(PointRangeQuery.newLongRange("idx", min, minInclusive, max, maxInclusive), Occur.FILTER);
       ref.add(new TermQuery(new Term("f", "a")), Occur.MUST);
 
       BooleanQuery.Builder bq1 = new BooleanQuery.Builder();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 3d2759d..05f38bb 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
@@ -302,7 +302,7 @@ public class TestSuggestField extends LuceneTestCase {
       }
     }
 
-    iw.deleteDocuments(PointRangeQuery.new1DIntRange("weight_fld", 2, true, null, false));
+    iw.deleteDocuments(PointRangeQuery.newIntRange("weight_fld", 2, true, null, false));
 
     DirectoryReader reader = DirectoryReader.open(iw);
     SuggestIndexSearcher indexSearcher = new SuggestIndexSearcher(reader);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/54a544b3/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 653fc33..99f8f89 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
@@ -33,8 +33,8 @@ import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
 import org.apache.lucene.index.PointValues.Relation;
-import org.apache.lucene.search.ExactPointQuery;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.PointRangeQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.Bits;
@@ -847,8 +847,8 @@ public abstract class BasePointFormatTestCase extends BaseIndexFileFormatTestCas
 
     DirectoryReader r = w.getReader();
     IndexSearcher s = newSearcher(r);
-    assertEquals(2, s.count(ExactPointQuery.new1DIntExact("int1", 17)));
-    assertEquals(2, s.count(ExactPointQuery.new1DIntExact("int2", 42)));
+    assertEquals(2, s.count(PointRangeQuery.newIntExact("int1", 17)));
+    assertEquals(2, s.count(PointRangeQuery.newIntExact("int2", 42)));
     r.close();
     w.close();
     dir.close();