You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by is...@apache.org on 2017/03/19 15:57:45 UTC

[11/12] lucene-solr:jira/solr-6736: Merge master into jira/solr-6736 branch

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java b/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
index 0832bdb..9fb474a 100644
--- a/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
+++ b/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java
@@ -19,7 +19,6 @@ package org.apache.lucene.util;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 
@@ -512,21 +511,20 @@ public class QueryBuilder {
       lastState = end;
       final Query queryPos;
       if (graph.hasSidePath(start)) {
-        List<Query> queries = new ArrayList<> ();
-        Iterator<TokenStream> it = graph.getFiniteStrings(start, end);
-        while (it.hasNext()) {
-          TokenStream ts = it.next();
-          // This is a synonym path so all terms are mandatory (MUST).
-          Query q = createFieldQuery(ts, BooleanClause.Occur.MUST, field, getAutoGenerateMultiTermSynonymsPhraseQuery(), 0);
-          if (q != null) {
-            queries.add(q);
+        final Iterator<TokenStream> it = graph.getFiniteStrings(start, end);
+        Iterator<Query> queries = new Iterator<Query>() {
+          @Override
+          public boolean hasNext() {
+            return it.hasNext();
           }
-        }
-        if (queries.size() > 0) {
-          queryPos = newGraphSynonymQuery(queries.toArray(new Query[queries.size()]));
-        } else {
-          queryPos = null;
-        }
+
+          @Override
+          public Query next() {
+            TokenStream ts = it.next();
+            return createFieldQuery(ts, BooleanClause.Occur.MUST, field, getAutoGenerateMultiTermSynonymsPhraseQuery(), 0);
+          }
+        };
+        queryPos = newGraphSynonymQuery(queries);
       } else {
         Term[] terms = graph.getTerms(field, start);
         assert terms.length > 0;
@@ -636,16 +634,16 @@ public class QueryBuilder {
    * This is intended for subclasses that wish to customize the generated queries.
    * @return new Query instance
    */
-  protected Query newGraphSynonymQuery(Query queries[]) {
-    if (queries == null) {
-      return new BooleanQuery.Builder().build();
-    } else if (queries.length == 1) {
-      return queries[0];
-    } else {
-      BooleanQuery.Builder builder = new BooleanQuery.Builder();
-      Arrays.stream(queries).forEachOrdered(qry -> builder.add(qry, BooleanClause.Occur.SHOULD));
-      return builder.build();
+  protected Query newGraphSynonymQuery(Iterator<Query> queries) {
+    BooleanQuery.Builder builder = new BooleanQuery.Builder();
+    while (queries.hasNext()) {
+      builder.add(queries.next(), BooleanClause.Occur.SHOULD);
     }
+    BooleanQuery bq = builder.build();
+    if (bq.clauses().size() == 1) {
+      return bq.clauses().get(0).getQuery();
+    }
+    return bq;
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/core/src/java/org/apache/lucene/util/Version.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/Version.java b/lucene/core/src/java/org/apache/lucene/util/Version.java
index 895f169..da6d653 100644
--- a/lucene/core/src/java/org/apache/lucene/util/Version.java
+++ b/lucene/core/src/java/org/apache/lucene/util/Version.java
@@ -102,6 +102,13 @@ public final class Version {
   public static final Version LUCENE_6_5_0 = new Version(6, 5, 0);
 
   /**
+   * Match settings and bugs in Lucene's 6.6.0 release.
+   * @deprecated Use latest
+   */
+  @Deprecated
+  public static final Version LUCENE_6_6_0 = new Version(6, 6, 0);
+
+  /**
    * Match settings and bugs in Lucene's 7.0.0 release.
    *  <p>
    *  Use this to get the latest &amp; greatest settings, bug

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/core/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
new file mode 100644
index 0000000..49ca710
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestDoubleRangeFieldQueries.java
@@ -0,0 +1,251 @@
+/*
+ * 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.util.Arrays;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleRange;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory;
+
+/**
+ * Random testing for RangeFieldQueries.
+ */
+public class TestDoubleRangeFieldQueries extends BaseRangeFieldQueryTestCase {
+  private static final String FIELD_NAME = "doubleRangeField";
+
+  private double nextDoubleInternal() {
+    if (rarely()) {
+      return random().nextBoolean() ? Double.POSITIVE_INFINITY : Double.NEGATIVE_INFINITY;
+    }
+    double max = Double.MAX_VALUE / 2;
+    return (max + max) * random().nextDouble() - max;
+  }
+
+  @Override
+  protected Range nextRange(int dimensions) throws Exception {
+    double[] min = new double[dimensions];
+    double[] max = new double[dimensions];
+
+    double minV, maxV;
+    for (int d=0; d<dimensions; ++d) {
+      minV = nextDoubleInternal();
+      maxV = nextDoubleInternal();
+      min[d] = Math.min(minV, maxV);
+      max[d] = Math.max(minV, maxV);
+    }
+    return new DoubleTestRange(min, max);
+  }
+
+  @Override
+  protected DoubleRange newRangeField(Range r) {
+    return new DoubleRange(FIELD_NAME, ((DoubleTestRange)r).min, ((DoubleTestRange)r).max);
+  }
+
+  @Override
+  protected Query newIntersectsQuery(Range r) {
+    return DoubleRange.newIntersectsQuery(FIELD_NAME, ((DoubleTestRange)r).min, ((DoubleTestRange)r).max);
+  }
+
+  @Override
+  protected Query newContainsQuery(Range r) {
+    return DoubleRange.newContainsQuery(FIELD_NAME, ((DoubleTestRange)r).min, ((DoubleTestRange)r).max);
+  }
+
+  @Override
+  protected Query newWithinQuery(Range r) {
+    return DoubleRange.newWithinQuery(FIELD_NAME, ((DoubleTestRange)r).min, ((DoubleTestRange)r).max);
+  }
+
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return DoubleRange.newCrossesQuery(FIELD_NAME, ((DoubleTestRange)r).min, ((DoubleTestRange)r).max);
+  }
+
+  /** Basic test */
+  public void testBasics() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // intersects (within)
+    Document document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {-10.0, -10.0}, new double[] {9.1, 10.1}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {10.0, -10.0}, new double[] {20.0, 10.0}));
+    writer.addDocument(document);
+
+    // intersects (contains, crosses)
+    document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {-20.0, -20.0}, new double[] {30.0, 30.1}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {-11.1, -11.2}, new double[] {1.23, 11.5}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {12.33, 1.2}, new double[] {15.1, 29.9}));
+    writer.addDocument(document);
+
+    // disjoint
+    document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {-122.33, 1.2}, new double[] {-115.1, 29.9}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {Double.NEGATIVE_INFINITY, 1.2}, new double[] {-11.0, 29.9}));
+    writer.addDocument(document);
+
+    // equal (within, contains, intersects)
+    document = new Document();
+    document.add(new DoubleRange(FIELD_NAME, new double[] {-11, -15}, new double[] {15, 20}));
+    writer.addDocument(document);
+
+    // search
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    assertEquals(7, searcher.count(DoubleRange.newIntersectsQuery(FIELD_NAME,
+        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
+    assertEquals(2, searcher.count(DoubleRange.newWithinQuery(FIELD_NAME,
+        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
+    assertEquals(2, searcher.count(DoubleRange.newContainsQuery(FIELD_NAME,
+        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
+    assertEquals(5, searcher.count(DoubleRange.newCrossesQuery(FIELD_NAME,
+        new double[] {-11.0, -15.0}, new double[] {15.0, 20.0})));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+
+  /** DoubleRange test class implementation - use to validate DoubleRange */
+  private class DoubleTestRange extends Range {
+    double[] min;
+    double[] max;
+
+    DoubleTestRange(double[] min, double[] max) {
+      assert min != null && max != null && min.length > 0 && max.length > 0
+          : "test box: min/max cannot be null or empty";
+      assert min.length == max.length : "test box: min/max length do not agree";
+      this.min = min;
+      this.max = max;
+    }
+
+    @Override
+    protected int numDimensions() {
+      return min.length;
+    }
+
+    @Override
+    protected Double getMin(int dim) {
+      return min[dim];
+    }
+
+    @Override
+    protected void setMin(int dim, Object val) {
+      double v = (Double)val;
+      if (min[dim] < v) {
+        max[dim] = v;
+      } else {
+        min[dim] = v;
+      }
+    }
+
+    @Override
+    protected Double getMax(int dim) {
+      return max[dim];
+    }
+
+    @Override
+    protected void setMax(int dim, Object val) {
+      double v = (Double)val;
+      if (max[dim] > v) {
+        min[dim] = v;
+      } else {
+        max[dim] = v;
+      }
+    }
+
+    @Override
+    protected boolean isEqual(Range other) {
+      DoubleTestRange o = (DoubleTestRange)other;
+      return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
+    }
+
+    @Override
+    protected boolean isDisjoint(Range o) {
+      DoubleTestRange other = (DoubleTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
+          // disjoint:
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    protected boolean isWithin(Range o) {
+      DoubleTestRange other = (DoubleTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
+          // not within:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    protected boolean contains(Range o) {
+      DoubleTestRange other = (DoubleTestRange) o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
+          // not contains:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder b = new StringBuilder();
+      b.append("Box(");
+      b.append(min[0]);
+      b.append(" TO ");
+      b.append(max[0]);
+      for (int d=1; d<min.length; ++d) {
+        b.append(", ");
+        b.append(min[d]);
+        b.append(" TO ");
+        b.append(max[d]);
+      }
+      b.append(")");
+
+      return b.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/core/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java
new file mode 100644
index 0000000..6dc5907
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestFloatRangeFieldQueries.java
@@ -0,0 +1,251 @@
+/*
+ * 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.util.Arrays;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.FloatRange;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory;
+
+/**
+ * Random testing for FloatRange Queries.
+ */
+public class TestFloatRangeFieldQueries extends BaseRangeFieldQueryTestCase {
+  private static final String FIELD_NAME = "floatRangeField";
+
+  private float nextFloatInternal() {
+    if (rarely()) {
+      return random().nextBoolean() ? Float.NEGATIVE_INFINITY : Float.POSITIVE_INFINITY;
+    }
+    float max = Float.MAX_VALUE / 2;
+    return (max + max) * random().nextFloat() - max;
+  }
+
+  @Override
+  protected Range nextRange(int dimensions) throws Exception {
+    float[] min = new float[dimensions];
+    float[] max = new float[dimensions];
+
+    float minV, maxV;
+    for (int d=0; d<dimensions; ++d) {
+      minV = nextFloatInternal();
+      maxV = nextFloatInternal();
+      min[d] = Math.min(minV, maxV);
+      max[d] = Math.max(minV, maxV);
+    }
+    return new FloatTestRange(min, max);
+  }
+
+  @Override
+  protected FloatRange newRangeField(Range r) {
+    return new FloatRange(FIELD_NAME, ((FloatTestRange)r).min, ((FloatTestRange)r).max);
+  }
+
+  @Override
+  protected Query newIntersectsQuery(Range r) {
+    return FloatRange.newIntersectsQuery(FIELD_NAME, ((FloatTestRange)r).min, ((FloatTestRange)r).max);
+  }
+
+  @Override
+  protected Query newContainsQuery(Range r) {
+    return FloatRange.newContainsQuery(FIELD_NAME, ((FloatTestRange)r).min, ((FloatTestRange)r).max);
+  }
+
+  @Override
+  protected Query newWithinQuery(Range r) {
+    return FloatRange.newWithinQuery(FIELD_NAME, ((FloatTestRange)r).min, ((FloatTestRange)r).max);
+  }
+
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return FloatRange.newCrossesQuery(FIELD_NAME, ((FloatTestRange)r).min, ((FloatTestRange)r).max);
+  }
+
+  /** Basic test */
+  public void testBasics() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // intersects (within)
+    Document document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {-10.0f, -10.0f}, new float[] {9.1f, 10.1f}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {10.0f, -10.0f}, new float[] {20.0f, 10.0f}));
+    writer.addDocument(document);
+
+    // intersects (contains, crosses)
+    document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {-20.0f, -20.0f}, new float[] {30.0f, 30.1f}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {-11.1f, -11.2f}, new float[] {1.23f, 11.5f}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {12.33f, 1.2f}, new float[] {15.1f, 29.9f}));
+    writer.addDocument(document);
+
+    // disjoint
+    document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {-122.33f, 1.2f}, new float[] {-115.1f, 29.9f}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {Float.NEGATIVE_INFINITY, 1.2f}, new float[] {-11.0f, 29.9f}));
+    writer.addDocument(document);
+
+    // equal (within, contains, intersects)
+    document = new Document();
+    document.add(new FloatRange(FIELD_NAME, new float[] {-11f, -15f}, new float[] {15f, 20f}));
+    writer.addDocument(document);
+
+    // search
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    assertEquals(7, searcher.count(FloatRange.newIntersectsQuery(FIELD_NAME,
+        new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
+    assertEquals(2, searcher.count(FloatRange.newWithinQuery(FIELD_NAME,
+        new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
+    assertEquals(2, searcher.count(FloatRange.newContainsQuery(FIELD_NAME,
+        new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
+    assertEquals(5, searcher.count(FloatRange.newCrossesQuery(FIELD_NAME,
+        new float[] {-11.0f, -15.0f}, new float[] {15.0f, 20.0f})));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+
+  /** FloatRange test class implementation - use to validate FloatRange */
+  private class FloatTestRange extends Range {
+    float[] min;
+    float[] max;
+
+    FloatTestRange(float[] min, float[] max) {
+      assert min != null && max != null && min.length > 0 && max.length > 0
+          : "test box: min/max cannot be null or empty";
+      assert min.length == max.length : "test box: min/max length do not agree";
+      this.min = min;
+      this.max = max;
+    }
+
+    @Override
+    protected int numDimensions() {
+      return min.length;
+    }
+
+    @Override
+    protected Float getMin(int dim) {
+      return min[dim];
+    }
+
+    @Override
+    protected void setMin(int dim, Object val) {
+      float v = (Float)val;
+      if (min[dim] < v) {
+        max[dim] = v;
+      } else {
+        min[dim] = v;
+      }
+    }
+
+    @Override
+    protected Float getMax(int dim) {
+      return max[dim];
+    }
+
+    @Override
+    protected void setMax(int dim, Object val) {
+      float v = (Float)val;
+      if (max[dim] > v) {
+        min[dim] = v;
+      } else {
+        max[dim] = v;
+      }
+    }
+
+    @Override
+    protected boolean isEqual(Range other) {
+      FloatTestRange o = (FloatTestRange)other;
+      return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
+    }
+
+    @Override
+    protected boolean isDisjoint(Range o) {
+      FloatTestRange other = (FloatTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
+          // disjoint:
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    protected boolean isWithin(Range o) {
+      FloatTestRange other = (FloatTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
+          // not within:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    protected boolean contains(Range o) {
+      FloatTestRange other = (FloatTestRange) o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
+          // not contains:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder b = new StringBuilder();
+      b.append("Box(");
+      b.append(min[0]);
+      b.append(" TO ");
+      b.append(max[0]);
+      for (int d=1; d<min.length; ++d) {
+        b.append(", ");
+        b.append(min[d]);
+        b.append(" TO ");
+        b.append(max[d]);
+      }
+      b.append(")");
+
+      return b.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/core/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java
new file mode 100644
index 0000000..14771c9
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestIntRangeFieldQueries.java
@@ -0,0 +1,251 @@
+/*
+ * 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.util.Arrays;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.IntRange;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory;
+
+/**
+ * Random testing for IntRange Queries.
+ */
+public class TestIntRangeFieldQueries extends BaseRangeFieldQueryTestCase {
+  private static final String FIELD_NAME = "intRangeField";
+
+  private int nextIntInternal() {
+    if (rarely()) {
+      return random().nextBoolean() ? Integer.MAX_VALUE : Integer.MIN_VALUE;
+    }
+    int max = Integer.MAX_VALUE / 2;
+    return (max + max) * random().nextInt() - max;
+  }
+
+  @Override
+  protected Range nextRange(int dimensions) throws Exception {
+    int[] min = new int[dimensions];
+    int[] max = new int[dimensions];
+
+    int minV, maxV;
+    for (int d=0; d<dimensions; ++d) {
+      minV = nextIntInternal();
+      maxV = nextIntInternal();
+      min[d] = Math.min(minV, maxV);
+      max[d] = Math.max(minV, maxV);
+    }
+    return new IntTestRange(min, max);
+  }
+
+  @Override
+  protected org.apache.lucene.document.IntRange newRangeField(Range r) {
+    return new IntRange(FIELD_NAME, ((IntTestRange)r).min, ((IntTestRange)r).max);
+  }
+
+  @Override
+  protected Query newIntersectsQuery(Range r) {
+    return IntRange.newIntersectsQuery(FIELD_NAME, ((IntTestRange)r).min, ((IntTestRange)r).max);
+  }
+
+  @Override
+  protected Query newContainsQuery(Range r) {
+    return IntRange.newContainsQuery(FIELD_NAME, ((IntTestRange)r).min, ((IntTestRange)r).max);
+  }
+
+  @Override
+  protected Query newWithinQuery(Range r) {
+    return IntRange.newWithinQuery(FIELD_NAME, ((IntTestRange)r).min, ((IntTestRange)r).max);
+  }
+
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return IntRange.newCrossesQuery(FIELD_NAME, ((IntTestRange)r).min, ((IntTestRange)r).max);
+  }
+
+  /** Basic test */
+  public void testBasics() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // intersects (within)
+    Document document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {-10, -10}, new int[] {9, 10}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {10, -10}, new int[] {20, 10}));
+    writer.addDocument(document);
+
+    // intersects (contains / crosses)
+    document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {-20, -20}, new int[] {30, 30}));
+    writer.addDocument(document);
+
+    // intersects (within)
+    document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {-11, -11}, new int[] {1, 11}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {12, 1}, new int[] {15, 29}));
+    writer.addDocument(document);
+
+    // disjoint
+    document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {-122, 1}, new int[] {-115, 29}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {Integer.MIN_VALUE, 1}, new int[] {-11, 29}));
+    writer.addDocument(document);
+
+    // equal (within, contains, intersects)
+    document = new Document();
+    document.add(new IntRange(FIELD_NAME, new int[] {-11, -15}, new int[] {15, 20}));
+    writer.addDocument(document);
+
+    // search
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    assertEquals(7, searcher.count(IntRange.newIntersectsQuery(FIELD_NAME,
+        new int[] {-11, -15}, new int[] {15, 20})));
+    assertEquals(3, searcher.count(IntRange.newWithinQuery(FIELD_NAME,
+        new int[] {-11, -15}, new int[] {15, 20})));
+    assertEquals(2, searcher.count(IntRange.newContainsQuery(FIELD_NAME,
+        new int[] {-11, -15}, new int[] {15, 20})));
+    assertEquals(4, searcher.count(IntRange.newCrossesQuery(FIELD_NAME,
+        new int[] {-11, -15}, new int[] {15, 20})));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+
+  /** IntRange test class implementation - use to validate IntRange */
+  private class IntTestRange extends Range {
+    int[] min;
+    int[] max;
+
+    IntTestRange(int[] min, int[] max) {
+      assert min != null && max != null && min.length > 0 && max.length > 0
+          : "test box: min/max cannot be null or empty";
+      assert min.length == max.length : "test box: min/max length do not agree";
+      this.min = min;
+      this.max = max;
+    }
+
+    @Override
+    protected int numDimensions() {
+      return min.length;
+    }
+
+    @Override
+    protected Integer getMin(int dim) {
+      return min[dim];
+    }
+
+    @Override
+    protected void setMin(int dim, Object val) {
+      int v = (Integer)val;
+      if (min[dim] < v) {
+        max[dim] = v;
+      } else {
+        min[dim] = v;
+      }
+    }
+
+    @Override
+    protected Integer getMax(int dim) {
+      return max[dim];
+    }
+
+    @Override
+    protected void setMax(int dim, Object val) {
+      int v = (Integer)val;
+      if (max[dim] > v) {
+        min[dim] = v;
+      } else {
+        max[dim] = v;
+      }
+    }
+
+    @Override
+    protected boolean isEqual(Range other) {
+      IntTestRange o = (IntTestRange)other;
+      return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
+    }
+
+    @Override
+    protected boolean isDisjoint(Range o) {
+      IntTestRange other = (IntTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
+          // disjoint:
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    protected boolean isWithin(Range o) {
+      IntTestRange other = (IntTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
+          // not within:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    protected boolean contains(Range o) {
+      IntTestRange other = (IntTestRange) o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
+          // not contains:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder b = new StringBuilder();
+      b.append("Box(");
+      b.append(min[0]);
+      b.append(" TO ");
+      b.append(max[0]);
+      for (int d=1; d<min.length; ++d) {
+        b.append(", ");
+        b.append(min[d]);
+        b.append(" TO ");
+        b.append(max[d]);
+      }
+      b.append(")");
+
+      return b.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/core/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java
new file mode 100644
index 0000000..60d7ea3
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLongRangeFieldQueries.java
@@ -0,0 +1,251 @@
+/*
+ * 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.util.Arrays;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.LongRange;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory;
+
+/**
+ * Random testing for LongRange Queries.
+ */
+public class TestLongRangeFieldQueries extends BaseRangeFieldQueryTestCase {
+  private static final String FIELD_NAME = "longRangeField";
+
+  private long nextLongInternal() {
+    if (rarely()) {
+      return random().nextBoolean() ? Long.MAX_VALUE : Long.MIN_VALUE;
+    }
+    long max = Long.MAX_VALUE / 2;
+    return (max + max) * random().nextLong() - max;
+  }
+
+  @Override
+  protected Range nextRange(int dimensions) throws Exception {
+    long[] min = new long[dimensions];
+    long[] max = new long[dimensions];
+
+    long minV, maxV;
+    for (int d=0; d<dimensions; ++d) {
+      minV = nextLongInternal();
+      maxV = nextLongInternal();
+      min[d] = Math.min(minV, maxV);
+      max[d] = Math.max(minV, maxV);
+    }
+    return new LongTestRange(min, max);
+  }
+
+  @Override
+  protected LongRange newRangeField(Range r) {
+    return new LongRange(FIELD_NAME, ((LongTestRange)r).min, ((LongTestRange)r).max);
+  }
+
+  @Override
+  protected Query newIntersectsQuery(Range r) {
+    return LongRange.newIntersectsQuery(FIELD_NAME, ((LongTestRange)r).min, ((LongTestRange)r).max);
+  }
+
+  @Override
+  protected Query newContainsQuery(Range r) {
+    return LongRange.newContainsQuery(FIELD_NAME, ((LongTestRange)r).min, ((LongTestRange)r).max);
+  }
+
+  @Override
+  protected Query newWithinQuery(Range r) {
+    return LongRange.newWithinQuery(FIELD_NAME, ((LongTestRange)r).min, ((LongTestRange)r).max);
+  }
+
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return LongRange.newCrossesQuery(FIELD_NAME, ((LongTestRange)r).min, ((LongTestRange)r).max);
+  }
+
+  /** Basic test */
+  public void testBasics() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // intersects (within)
+    Document document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {-10, -10}, new long[] {9, 10}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {10, -10}, new long[] {20, 10}));
+    writer.addDocument(document);
+
+    // intersects (contains, crosses)
+    document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {-20, -20}, new long[] {30, 30}));
+    writer.addDocument(document);
+
+    // intersects (within)
+    document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {-11, -11}, new long[] {1, 11}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {12, 1}, new long[] {15, 29}));
+    writer.addDocument(document);
+
+    // disjoint
+    document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {-122, 1}, new long[] {-115, 29}));
+    writer.addDocument(document);
+
+    // intersects (crosses)
+    document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {Long.MIN_VALUE, 1}, new long[] {-11, 29}));
+    writer.addDocument(document);
+
+    // equal (within, contains, intersects)
+    document = new Document();
+    document.add(new LongRange(FIELD_NAME, new long[] {-11, -15}, new long[] {15, 20}));
+    writer.addDocument(document);
+
+    // search
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    assertEquals(7, searcher.count(LongRange.newIntersectsQuery(FIELD_NAME,
+        new long[] {-11, -15}, new long[] {15, 20})));
+    assertEquals(3, searcher.count(LongRange.newWithinQuery(FIELD_NAME,
+        new long[] {-11, -15}, new long[] {15, 20})));
+    assertEquals(2, searcher.count(LongRange.newContainsQuery(FIELD_NAME,
+        new long[] {-11, -15}, new long[] {15, 20})));
+    assertEquals(4, searcher.count(LongRange.newCrossesQuery(FIELD_NAME,
+        new long[] {-11, -15}, new long[] {15, 20})));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+
+  /** LongRange test class implementation - use to validate LongRange */
+  private class LongTestRange extends Range {
+    long[] min;
+    long[] max;
+
+    LongTestRange(long[] min, long[] max) {
+      assert min != null && max != null && min.length > 0 && max.length > 0
+          : "test box: min/max cannot be null or empty";
+      assert min.length == max.length : "test box: min/max length do not agree";
+      this.min = min;
+      this.max = max;
+    }
+
+    @Override
+    protected int numDimensions() {
+      return min.length;
+    }
+
+    @Override
+    protected Long getMin(int dim) {
+      return min[dim];
+    }
+
+    @Override
+    protected void setMin(int dim, Object val) {
+      long v = (Long)val;
+      if (min[dim] < v) {
+        max[dim] = v;
+      } else {
+        min[dim] = v;
+      }
+    }
+
+    @Override
+    protected Long getMax(int dim) {
+      return max[dim];
+    }
+
+    @Override
+    protected void setMax(int dim, Object val) {
+      long v = (Long)val;
+      if (max[dim] > v) {
+        min[dim] = v;
+      } else {
+        max[dim] = v;
+      }
+    }
+
+    @Override
+    protected boolean isEqual(Range other) {
+      LongTestRange o = (LongTestRange)other;
+      return Arrays.equals(min, o.min) && Arrays.equals(max, o.max);
+    }
+
+    @Override
+    protected boolean isDisjoint(Range o) {
+      LongTestRange other = (LongTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if (this.min[d] > other.max[d] || this.max[d] < other.min[d]) {
+          // disjoint:
+          return true;
+        }
+      }
+      return false;
+    }
+
+    @Override
+    protected boolean isWithin(Range o) {
+      LongTestRange other = (LongTestRange)o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] >= other.min[d] && this.max[d] <= other.max[d]) == false) {
+          // not within:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    protected boolean contains(Range o) {
+      LongTestRange other = (LongTestRange) o;
+      for (int d=0; d<this.min.length; ++d) {
+        if ((this.min[d] <= other.min[d] && this.max[d] >= other.max[d]) == false) {
+          // not contains:
+          return false;
+        }
+      }
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder b = new StringBuilder();
+      b.append("Box(");
+      b.append(min[0]);
+      b.append(" TO ");
+      b.append(max[0]);
+      for (int d=1; d<min.length; ++d) {
+        b.append(", ");
+        b.append(min[d]);
+        b.append(" TO ");
+        b.append(max[d]);
+      }
+      b.append(")");
+
+      return b.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
index dcce285..7a8e5f5 100644
--- a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
+++ b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
@@ -624,10 +624,10 @@ public class TestFSTs extends LuceneTestCase {
     int idx = 0;
     while (idx < args.length) {
       if (args[idx].equals("-prune")) {
-        prune = Integer.valueOf(args[1 + idx]);
+        prune = Integer.parseInt(args[1 + idx]);
         idx++;
       } else if (args[idx].equals("-limit")) {
-        limit = Integer.valueOf(args[1 + idx]);
+        limit = Integer.parseInt(args[1 + idx]);
         idx++;
       } else if (args[idx].equals("-utf8")) {
         inputMode = 0;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
index a13e66f..da3c20e 100644
--- a/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
+++ b/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
@@ -1011,7 +1011,7 @@ public class TestBlockJoin extends LuceneTestCase {
     TopDocs childHits = new TopDocs(0, new ScoreDoc[0], 0f);
     for (ScoreDoc controlHit : controlHits.scoreDocs) {
       Document controlDoc = r.document(controlHit.doc);
-      int parentID = Integer.valueOf(controlDoc.get("parentID"));
+      int parentID = Integer.parseInt(controlDoc.get("parentID"));
       if (parentID != currentParentID) {
         assertEquals(childHitSlot, childHits.scoreDocs.length);
         currentParentID = parentID;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/misc/src/java/org/apache/lucene/document/InetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/document/InetAddressPoint.java b/lucene/misc/src/java/org/apache/lucene/document/InetAddressPoint.java
new file mode 100644
index 0000000..5cda742
--- /dev/null
+++ b/lucene/misc/src/java/org/apache/lucene/document/InetAddressPoint.java
@@ -0,0 +1,313 @@
+/*
+ * 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.document;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Arrays;
+import java.util.Comparator;
+
+import org.apache.lucene.index.PointValues;
+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.NumericUtils;
+import org.apache.lucene.util.StringHelper;
+
+/** 
+ * An indexed 128-bit {@code InetAddress} field.
+ * <p>
+ * Finding all documents within a range at search time is
+ * efficient.  Multiple values for the same field in one document
+ * is allowed. 
+ * <p>
+ * This field defines static factory methods for creating common queries:
+ * <ul>
+ *   <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 network addresses.
+ * </ul>
+ * <p>
+ * This field supports both IPv4 and IPv6 addresses: IPv4 addresses are converted
+ * to <a href="https://tools.ietf.org/html/rfc4291#section-2.5.5">IPv4-Mapped IPv6 Addresses</a>:
+ * indexing {@code 1.2.3.4} is the same as indexing {@code ::FFFF:1.2.3.4}.
+ * @see PointValues
+ */
+public class InetAddressPoint extends Field {
+
+  // implementation note: we convert all addresses to IPv6: we expect prefix compression of values,
+  // so its not wasteful, but allows one field to handle both IPv4 and IPv6.
+  /** The number of bytes per dimension: 128 bits */
+  public static final int BYTES = 16;
+  
+  // rfc4291 prefix
+  static final byte[] IPV4_PREFIX = new byte[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -1, -1 }; 
+
+  private static final FieldType TYPE;
+  static {
+    TYPE = new FieldType();
+    TYPE.setDimensions(1, BYTES);
+    TYPE.freeze();
+  }
+
+  /** The minimum value that an ip address can hold. */
+  public static final InetAddress MIN_VALUE;
+  /** The maximum value that an ip address can hold. */
+  public static final InetAddress MAX_VALUE;
+  static {
+    MIN_VALUE = decode(new byte[BYTES]);
+    byte[] maxValueBytes = new byte[BYTES];
+    Arrays.fill(maxValueBytes, (byte) 0xFF);
+    MAX_VALUE = decode(maxValueBytes);
+  }
+
+  /**
+   * Return the {@link InetAddress} that compares immediately greater than
+   * {@code address}.
+   * @throws ArithmeticException if the provided address is the
+   *              {@link #MAX_VALUE maximum ip address}
+   */
+  public static InetAddress nextUp(InetAddress address) {
+    if (address.equals(MAX_VALUE)) {
+      throw new ArithmeticException("Overflow: there is no greater InetAddress than "
+          + address.getHostAddress());
+    }
+    byte[] delta = new byte[BYTES];
+    delta[BYTES-1] = 1;
+    byte[] nextUpBytes = new byte[InetAddressPoint.BYTES];
+    NumericUtils.add(InetAddressPoint.BYTES, 0, encode(address), delta, nextUpBytes);
+    return decode(nextUpBytes);
+  }
+
+  /**
+   * Return the {@link InetAddress} that compares immediately less than
+   * {@code address}.
+   * @throws ArithmeticException if the provided address is the
+   *              {@link #MIN_VALUE minimum ip address}
+   */
+  public static InetAddress nextDown(InetAddress address) {
+    if (address.equals(MIN_VALUE)) {
+      throw new ArithmeticException("Underflow: there is no smaller InetAddress than "
+          + address.getHostAddress());
+    }
+    byte[] delta = new byte[BYTES];
+    delta[BYTES-1] = 1;
+    byte[] nextDownBytes = new byte[InetAddressPoint.BYTES];
+    NumericUtils.subtract(InetAddressPoint.BYTES, 0, encode(address), delta, nextDownBytes);
+    return decode(nextDownBytes);
+  }
+
+  /** Change the values of this field */
+  public void setInetAddressValue(InetAddress value) {
+    if (value == null) {
+      throw new IllegalArgumentException("point must not be null");
+    }
+    fieldsData = new BytesRef(encode(value));
+  }
+
+  @Override
+  public void setBytesValue(BytesRef bytes) {
+    throw new IllegalArgumentException("cannot change value type from InetAddress to BytesRef");
+  }
+
+  /** Creates a new InetAddressPoint, indexing the
+   *  provided address.
+   *
+   *  @param name field name
+   *  @param point InetAddress value
+   *  @throws IllegalArgumentException if the field name or value is null.
+   */
+  public InetAddressPoint(String name, InetAddress point) {
+    super(name, TYPE);
+    setInetAddressValue(point);
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder result = new StringBuilder();
+    result.append(getClass().getSimpleName());
+    result.append(" <");
+    result.append(name);
+    result.append(':');
+
+    // IPv6 addresses are bracketed, to not cause confusion with historic field:value representation
+    BytesRef bytes = (BytesRef) fieldsData;
+    InetAddress address = decode(BytesRef.deepCopyOf(bytes).bytes);
+    if (address.getAddress().length == 16) {
+      result.append('[');
+      result.append(address.getHostAddress());
+      result.append(']');
+    } else {
+      result.append(address.getHostAddress());
+    }
+
+    result.append('>');
+    return result.toString();
+  }
+  
+  // public helper methods (e.g. for queries)
+
+  /** Encode InetAddress value into binary encoding */
+  public static byte[] encode(InetAddress value) {
+    byte[] address = value.getAddress();
+    if (address.length == 4) {
+      byte[] mapped = new byte[16];
+      System.arraycopy(IPV4_PREFIX, 0, mapped, 0, IPV4_PREFIX.length);
+      System.arraycopy(address, 0, mapped, IPV4_PREFIX.length, address.length);
+      address = mapped;
+    } else if (address.length != 16) {
+      // more of an assertion, how did you create such an InetAddress :)
+      throw new UnsupportedOperationException("Only IPv4 and IPv6 addresses are supported");
+    }
+    return address;
+  }
+  
+  /** Decodes InetAddress value from binary encoding */
+  public static InetAddress decode(byte value[]) {
+    try {
+      return InetAddress.getByAddress(value);
+    } catch (UnknownHostException e) {
+      // this only happens if value.length != 4 or 16, strange exception class
+      throw new IllegalArgumentException("encoded bytes are of incorrect length", e);
+    }
+  }
+
+  // static methods for generating queries
+
+  /** 
+   * Create a query for matching a network address.
+   *
+   * @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 Query newExactQuery(String field, InetAddress value) {
+    return newRangeQuery(field, value, value);
+  }
+  
+  /** 
+   * Create a prefix query for matching a CIDR network range.
+   *
+   * @param field field name. must not be {@code null}.
+   * @param value any host address
+   * @param prefixLength the network prefix length for this address. This is also known as the subnet mask in the context of IPv4 addresses.
+   * @throws IllegalArgumentException if {@code field} is null, or prefixLength is invalid.
+   * @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 must not 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");
+    }
+    // create the lower value by zeroing out the host portion, upper value by filling it with all ones.
+    byte lower[] = value.getAddress();
+    byte upper[] = value.getAddress();
+    for (int i = prefixLength; i < 8 * lower.length; i++) {
+      int m = 1 << (7 - (i & 7));
+      lower[i >> 3] &= ~m;
+      upper[i >> 3] |= m;
+    }
+    try {
+      return newRangeQuery(field, InetAddress.getByAddress(lower), InetAddress.getByAddress(upper));
+    } catch (UnknownHostException e) {
+      throw new AssertionError(e); // values are coming from InetAddress
+    }
+  }
+
+  /** 
+   * 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 {@code lowerValue = InetAddressPoint.MIN_VALUE} or
+   * {@code upperValue = InetAddressPoint.MAX_VALUE}.
+   * <p> Ranges are inclusive. For exclusive ranges, pass {@code InetAddressPoint#nextUp(lowerValue)}
+   * or {@code InetAddressPoint#nexDown(upperValue)}.
+   *
+   * @param field field name. must not be {@code 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, InetAddress upperValue) {
+    PointRangeQuery.checkArgs(field, lowerValue, upperValue);
+    return new PointRangeQuery(field, encode(lowerValue), encode(upperValue), 1) {
+      @Override
+      protected String toString(int dimension, byte[] value) {
+        return decode(value).getHostAddress(); // for ranges, the range itself is already bracketed
+      }
+    };
+  }
+
+  /**
+   * 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 values all values to match
+   */
+  public static Query newSetQuery(String field, InetAddress... values) {
+
+    // We must compare the encoded form (InetAddress doesn't implement Comparable, and even if it
+    // did, we do our own thing with ipv4 addresses):
+
+    // NOTE: we could instead convert-per-comparison and save this extra array, at cost of slower sort:
+    byte[][] sortedValues = new byte[values.length][];
+    for(int i=0;i<values.length;i++) {
+      sortedValues[i] = encode(values[i]);
+    }
+
+    Arrays.sort(sortedValues,
+                new Comparator<byte[]>() {
+                  @Override
+                  public int compare(byte[] a, byte[] b) {
+                    return StringHelper.compare(BYTES, a, 0, b, 0);
+                  }
+                });
+
+    final BytesRef encoded = new BytesRef(new byte[BYTES]);
+
+    return new PointInSetQuery(field, 1, BYTES,
+                               new PointInSetQuery.Stream() {
+
+                                 int upto;
+
+                                 @Override
+                                 public BytesRef next() {
+                                   if (upto == sortedValues.length) {
+                                     return null;
+                                   } else {
+                                     encoded.bytes = sortedValues[upto];
+                                     assert encoded.bytes.length == encoded.length;
+                                     upto++;
+                                     return encoded;
+                                   }
+                                 }
+                               }) {
+      @Override
+      protected String toString(byte[] value) {
+        assert value.length == BYTES;
+        return decode(value).getHostAddress();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/misc/src/java/org/apache/lucene/document/InetAddressRange.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/document/InetAddressRange.java b/lucene/misc/src/java/org/apache/lucene/document/InetAddressRange.java
new file mode 100644
index 0000000..5fa1fb9
--- /dev/null
+++ b/lucene/misc/src/java/org/apache/lucene/document/InetAddressRange.java
@@ -0,0 +1,168 @@
+/*
+ * 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.document;
+
+import java.net.InetAddress;
+
+import org.apache.lucene.document.RangeFieldQuery.QueryType;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.StringHelper;
+
+/**
+ * An indexed InetAddress Range Field
+ * <p>
+ * This field indexes an {@code InetAddress} range defined as a min/max pairs. It is single
+ * dimension only (indexed as two 16 byte paired values).
+ * <p>
+ * Multiple values are supported.
+ *
+ * <p>
+ * This field defines the following static factory methods for common search operations over Ip Ranges
+ * <ul>
+ *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ip ranges that intersect the defined search range.
+ *   <li>{@link #newWithinQuery newWithinQuery()} matches ip ranges that are within the defined search range.
+ *   <li>{@link #newContainsQuery newContainsQuery()} matches ip ranges that contain the defined search range.
+ *   <li>{@link #newCrossesQuery newCrossesQuery()} matches ip ranges that cross the defined search range
+ * </ul>
+ */
+public class InetAddressRange extends Field {
+  /** The number of bytes per dimension : sync w/ {@code InetAddressPoint} */
+  public static final int BYTES = InetAddressPoint.BYTES;
+
+  private static final FieldType TYPE;
+  static {
+    TYPE = new FieldType();
+    TYPE.setDimensions(2, BYTES);
+    TYPE.freeze();
+  }
+
+  /**
+   * Create a new InetAddressRange from min/max value
+   * @param name field name. must not be null.
+   * @param min range min value; defined as an {@code InetAddress}
+   * @param max range max value; defined as an {@code InetAddress}
+   */
+  public InetAddressRange(String name, final InetAddress min, final InetAddress max) {
+    super(name, TYPE);
+    setRangeValues(min, max);
+  }
+
+  /**
+   * Change (or set) the min/max values of the field.
+   * @param min range min value; defined as an {@code InetAddress}
+   * @param max range max value; defined as an {@code InetAddress}
+   */
+  public void setRangeValues(InetAddress min, InetAddress max) {
+    if (StringHelper.compare(BYTES, min.getAddress(), 0, max.getAddress(), 0) > 0) {
+      throw new IllegalArgumentException("min value cannot be greater than max value for range field (name=" + name + ")");
+    }
+    final byte[] bytes;
+    if (fieldsData == null) {
+      bytes = new byte[BYTES*2];
+      fieldsData = new BytesRef(bytes);
+    } else {
+      bytes = ((BytesRef)fieldsData).bytes;
+    }
+    encode(min, max, bytes);
+  }
+
+  /** encode the min/max range into the provided byte array */
+  private static void encode(final InetAddress min, final InetAddress max, final byte[] bytes) {
+    System.arraycopy(InetAddressPoint.encode(min), 0, bytes, 0, BYTES);
+    System.arraycopy(InetAddressPoint.encode(max), 0, bytes, BYTES, BYTES);
+  }
+
+  /** encode the min/max range and return the byte array */
+  private static byte[] encode(InetAddress min, InetAddress max) {
+    byte[] b = new byte[BYTES*2];
+    encode(min, max, b);
+    return b;
+  }
+
+  /**
+   * Create a query for matching indexed ip ranges that {@code INTERSECT} the defined range.
+   * @param field field name. must not be null.
+   * @param min range min value; provided as an {@code InetAddress}
+   * @param max range max value; provided as an {@code InetAddress}
+   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newIntersectsQuery(String field, final InetAddress min, final InetAddress max) {
+    return newRelationQuery(field, min, max, QueryType.INTERSECTS);
+  }
+
+  /**
+   * Create a query for matching indexed ip ranges that {@code CONTAINS} the defined range.
+   * @param field field name. must not be null.
+   * @param min range min value; provided as an {@code InetAddress}
+   * @param max range max value; provided as an {@code InetAddress}
+   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newContainsQuery(String field, final InetAddress min, final InetAddress max) {
+    return newRelationQuery(field, min, max, QueryType.CONTAINS);
+  }
+
+  /**
+   * Create a query for matching indexed ip ranges that are {@code WITHIN} the defined range.
+   * @param field field name. must not be null.
+   * @param min range min value; provided as an {@code InetAddress}
+   * @param max range max value; provided as an {@code InetAddress}
+   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newWithinQuery(String field, final InetAddress min, final InetAddress max) {
+    return newRelationQuery(field, min, max, QueryType.WITHIN);
+  }
+
+  /**
+   * Create a query for matching indexed ip ranges that {@code CROSS} the defined range.
+   * @param field field name. must not be null.
+   * @param min range min value; provided as an {@code InetAddress}
+   * @param max range max value; provided as an {@code InetAddress}
+   * @return query for matching intersecting ranges (overlap, within, crosses, or contains)
+   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
+   */
+  public static Query newCrossesQuery(String field, final InetAddress min, final InetAddress max) {
+    return newRelationQuery(field, min, max, QueryType.CROSSES);
+  }
+
+  /** helper method for creating the desired relational query */
+  private static Query newRelationQuery(String field, final InetAddress min, final InetAddress max, QueryType relation) {
+    return new RangeFieldQuery(field, encode(min, max), 1, relation) {
+      @Override
+      protected String toString(byte[] ranges, int dimension) {
+        return InetAddressRange.toString(ranges, dimension);
+      }
+    };
+  }
+
+  /**
+   * Returns the String representation for the range at the given dimension
+   * @param ranges the encoded ranges, never null
+   * @param dimension the dimension of interest (not used for this field)
+   * @return The string representation for the range at the provided dimension
+   */
+  private static String toString(byte[] ranges, int dimension) {
+    byte[] min = new byte[BYTES];
+    System.arraycopy(ranges, 0, min, 0, BYTES);
+    byte[] max = new byte[BYTES];
+    System.arraycopy(ranges, BYTES, max, 0, BYTES);
+    return "[" + InetAddressPoint.decode(min) + " : " + InetAddressPoint.decode(max) + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/misc/src/test/org/apache/lucene/document/TestInetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/document/TestInetAddressPoint.java b/lucene/misc/src/test/org/apache/lucene/document/TestInetAddressPoint.java
new file mode 100644
index 0000000..0e0901b
--- /dev/null
+++ b/lucene/misc/src/test/org/apache/lucene/document/TestInetAddressPoint.java
@@ -0,0 +1,176 @@
+/*
+ * 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.document;
+
+import java.net.InetAddress;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
+/** Simple tests for {@link InetAddressPoint} */
+public class TestInetAddressPoint extends LuceneTestCase {
+
+  /** Add a single address and search for it */
+  public void testBasics() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // add a doc with an address
+    Document document = new Document();
+    InetAddress address = InetAddress.getByName("1.2.3.4");
+    document.add(new InetAddressPoint("field", address));
+    writer.addDocument(document);
+    
+    // search and verify we found our doc
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    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"), InetAddress.getByName("1.2.3.5"))));
+    assertEquals(1, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.4"))));
+    assertEquals(1, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.4"), InetAddress.getByName("1.2.3.5"))));
+    assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field", InetAddress.getByName("1.2.3.3"))));
+    assertEquals(0, searcher.count(InetAddressPoint.newSetQuery("field")));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+  
+  /** Add a single address and search for it */
+  public void testBasicsV6() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+
+    // add a doc with an address
+    Document document = new Document();
+    InetAddress address = InetAddress.getByName("fec0::f66d");
+    document.add(new InetAddressPoint("field", address));
+    writer.addDocument(document);
+    
+    // search and verify we found our doc
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    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"), InetAddress.getByName("fec0::f66e"))));
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+    
+  public void testToString() throws Exception {
+    assertEquals("InetAddressPoint <field:1.2.3.4>", new InetAddressPoint("field", InetAddress.getByName("1.2.3.4")).toString());
+    assertEquals("InetAddressPoint <field:1.2.3.4>", new InetAddressPoint("field", InetAddress.getByName("::FFFF:1.2.3.4")).toString());
+    assertEquals("InetAddressPoint <field:[fdc8:57ed:f042:ad1:f66d:4ff:fe90:ce0c]>", new InetAddressPoint("field", InetAddress.getByName("fdc8:57ed:f042:0ad1:f66d:4ff:fe90:ce0c")).toString());
+    
+    assertEquals("field:[1.2.3.4 TO 1.2.3.4]", InetAddressPoint.newExactQuery("field", InetAddress.getByName("1.2.3.4")).toString());
+    assertEquals("field:[0:0:0:0:0:0:0:1 TO 0:0:0:0:0:0:0:1]", InetAddressPoint.newExactQuery("field", InetAddress.getByName("::1")).toString());
+    
+    assertEquals("field:[1.2.3.0 TO 1.2.3.255]", InetAddressPoint.newPrefixQuery("field", InetAddress.getByName("1.2.3.4"), 24).toString());
+    assertEquals("field:[fdc8:57ed:f042:ad1:0:0:0:0 TO fdc8:57ed:f042:ad1:ffff:ffff:ffff:ffff]", InetAddressPoint.newPrefixQuery("field", InetAddress.getByName("fdc8:57ed:f042:0ad1:f66d:4ff:fe90:ce0c"), 64).toString());
+    assertEquals("field:{fdc8:57ed:f042:ad1:f66d:4ff:fe90:ce0c}", InetAddressPoint.newSetQuery("field", InetAddress.getByName("fdc8:57ed:f042:0ad1:f66d:4ff:fe90:ce0c")).toString());
+  }
+
+  public void testQueryEquals() throws Exception {
+    Query q1, q2;
+    q1 = InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"));
+    q2 = InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"));
+    assertEquals(q1, q2);
+    assertEquals(q1.hashCode(), q2.hashCode());
+    assertFalse(q1.equals(InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.7"))));
+    assertFalse(q1.equals(InetAddressPoint.newRangeQuery("b", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"))));
+
+    q1 = InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.3"), 16);
+    q2 = InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.3"), 16);
+    assertEquals(q1, q2);
+    assertEquals(q1.hashCode(), q2.hashCode());
+    assertFalse(q1.equals(InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.1.3.5"), 16)));
+    assertFalse(q1.equals(InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.5"), 24)));
+
+    q1 = InetAddressPoint.newExactQuery("a", InetAddress.getByName("1.2.3.3"));
+    q2 = InetAddressPoint.newExactQuery("a", InetAddress.getByName("1.2.3.3"));
+    assertEquals(q1, q2);
+    assertEquals(q1.hashCode(), q2.hashCode());
+    assertFalse(q1.equals(InetAddressPoint.newExactQuery("a", InetAddress.getByName("1.2.3.5"))));
+
+    q1 = InetAddressPoint.newSetQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"));
+    q2 = InetAddressPoint.newSetQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.5"));
+    assertEquals(q1, q2);
+    assertEquals(q1.hashCode(), q2.hashCode());
+    assertFalse(q1.equals(InetAddressPoint.newSetQuery("a", InetAddress.getByName("1.2.3.3"), InetAddress.getByName("1.2.3.7"))));
+  }
+
+  public void testPrefixQuery() throws Exception {
+    assertEquals(
+        InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.0"), InetAddress.getByName("1.2.3.255")),
+        InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.127"), 24));
+    assertEquals(
+        InetAddressPoint.newRangeQuery("a", InetAddress.getByName("1.2.3.128"), InetAddress.getByName("1.2.3.255")),
+        InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("1.2.3.213"), 25));
+    assertEquals(
+        InetAddressPoint.newRangeQuery("a", InetAddress.getByName("2001::a000:0"), InetAddress.getByName("2001::afff:ffff")),
+        InetAddressPoint.newPrefixQuery("a", InetAddress.getByName("2001::a6bd:fc80"), 100));
+  }
+
+  public void testNextUp() throws Exception {
+    assertEquals(InetAddress.getByName("::1"),
+        InetAddressPoint.nextUp(InetAddress.getByName("::")));
+
+    assertEquals(InetAddress.getByName("::1:0"),
+        InetAddressPoint.nextUp(InetAddress.getByName("::ffff")));
+
+    assertEquals(InetAddress.getByName("1.2.4.0"),
+        InetAddressPoint.nextUp(InetAddress.getByName("1.2.3.255")));
+
+    assertEquals(InetAddress.getByName("0.0.0.0"),
+        InetAddressPoint.nextUp(InetAddress.getByName("::fffe:ffff:ffff")));
+
+    assertEquals(InetAddress.getByName("::1:0:0:0"),
+        InetAddressPoint.nextUp(InetAddress.getByName("255.255.255.255")));
+
+    ArithmeticException e = expectThrows(ArithmeticException.class,
+        () -> InetAddressPoint.nextUp(InetAddress.getByName("ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff")));
+    assertEquals("Overflow: there is no greater InetAddress than ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff", e.getMessage());
+  }
+
+  public void testNextDown() throws Exception {
+    assertEquals(InetAddress.getByName("ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe"),
+        InetAddressPoint.nextDown(InetAddress.getByName("ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff")));
+
+    assertEquals(InetAddress.getByName("::ffff"),
+        InetAddressPoint.nextDown(InetAddress.getByName("::1:0")));
+
+    assertEquals(InetAddress.getByName("1.2.3.255"),
+        InetAddressPoint.nextDown(InetAddress.getByName("1.2.4.0")));
+
+    assertEquals(InetAddress.getByName("::fffe:ffff:ffff"),
+        InetAddressPoint.nextDown(InetAddress.getByName("0.0.0.0")));
+
+    assertEquals(InetAddress.getByName("255.255.255.255"),
+        InetAddressPoint.nextDown(InetAddress.getByName("::1:0:0:0")));
+
+    ArithmeticException e = expectThrows(ArithmeticException.class,
+        () -> InetAddressPoint.nextDown(InetAddress.getByName("::")));
+    assertEquals("Underflow: there is no smaller InetAddress than 0:0:0:0:0:0:0:0", e.getMessage());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
index 043141a..f07793a 100644
--- a/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
@@ -392,7 +392,7 @@ public class TestDiversifiedTopDocsCollector extends LuceneTestCase {
     for (int i = 0; i < hitsOfThe60s.length; i++) {
       String cols[] = hitsOfThe60s[i].split("\t");
       Record record = new Record(String.valueOf(i), cols[0], cols[1], cols[2],
-          Float.valueOf(cols[3]));
+          Float.parseFloat(cols[3]));
       parsedRecords.put(record.id, record);
       idField.setStringValue(record.id);
       yearField.setStringValue(record.year);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java b/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java
new file mode 100644
index 0000000..e22cf9b
--- /dev/null
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java
@@ -0,0 +1,215 @@
+/*
+ * 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.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.lucene.document.InetAddressRange;
+import org.apache.lucene.util.StringHelper;
+
+/**
+ * Random testing for {@link InetAddressRange}
+ */
+public class TestInetAddressRangeQueries extends BaseRangeFieldQueryTestCase {
+  private static final String FIELD_NAME = "ipRangeField";
+
+  private IPVersion ipVersion;
+
+  private enum IPVersion {IPv4, IPv6}
+
+  @Override
+  protected Range nextRange(int dimensions) throws Exception {
+    InetAddress min = nextInetaddress();
+    byte[] bMin = min.getAddress();
+    InetAddress max = nextInetaddress();
+    byte[] bMax = max.getAddress();
+    if (StringHelper.compare(bMin.length, bMin, 0, bMax, 0) > 0) {
+      return new IpRange(max, min);
+    }
+    return new IpRange(min, max);
+  }
+
+  /** return random IPv4 or IPv6 address */
+  private InetAddress nextInetaddress() throws UnknownHostException {
+    byte[] b;
+    switch (ipVersion) {
+      case IPv4:
+        b = new byte[4];
+        break;
+      case IPv6:
+        b = new byte[16];
+        break;
+      default:
+        throw new IllegalArgumentException("incorrect IP version: " + ipVersion);
+    }
+    random().nextBytes(b);
+    return InetAddress.getByAddress(b);
+  }
+
+  /** randomly select version across tests */
+  private IPVersion ipVersion() {
+    return random().nextBoolean() ? IPVersion.IPv4 : IPVersion.IPv6;
+  }
+
+  @Override
+  public void testRandomTiny() throws Exception {
+    ipVersion = ipVersion();
+    super.testRandomTiny();
+  }
+
+  @Override
+  public void testMultiValued() throws Exception {
+    ipVersion = ipVersion();
+    super.testRandomMedium();
+  }
+
+  @Override
+  public void testRandomMedium() throws Exception {
+    ipVersion = ipVersion();
+    super.testMultiValued();
+  }
+
+  @Nightly
+  @Override
+  public void testRandomBig() throws Exception {
+    ipVersion = ipVersion();
+    super.testRandomBig();
+  }
+
+  /** return random range */
+  @Override
+  protected InetAddressRange newRangeField(Range r) {
+    return new InetAddressRange(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** return random intersects query */
+  @Override
+  protected Query newIntersectsQuery(Range r) {
+    return InetAddressRange.newIntersectsQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** return random contains query */
+  @Override
+  protected Query newContainsQuery(Range r) {
+    return InetAddressRange.newContainsQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** return random within query */
+  @Override
+  protected Query newWithinQuery(Range r) {
+    return InetAddressRange.newWithinQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** return random crosses query */
+  @Override
+  protected Query newCrossesQuery(Range r) {
+    return InetAddressRange.newCrossesQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max);
+  }
+
+  /** encapsulated IpRange for test validation */
+  private class IpRange extends Range {
+    InetAddress min;
+    InetAddress max;
+
+    IpRange(InetAddress min, InetAddress max) {
+      this.min = min;
+      this.max = max;
+    }
+
+    @Override
+    protected int numDimensions() {
+      return 1;
+    }
+
+    @Override
+    protected InetAddress getMin(int dim) {
+      return min;
+    }
+
+    @Override
+    protected void setMin(int dim, Object val) {
+      byte[] v = ((InetAddress)val).getAddress();
+
+      if (StringHelper.compare(v.length, min.getAddress(), 0, v, 0) < 0) {
+        max = (InetAddress)val;
+      } else {
+        min = (InetAddress) val;
+      }
+    }
+
+    @Override
+    protected InetAddress getMax(int dim) {
+      return max;
+    }
+
+    @Override
+    protected void setMax(int dim, Object val) {
+      byte[] v = ((InetAddress)val).getAddress();
+
+      if (StringHelper.compare(v.length, max.getAddress(), 0, v, 0) > 0) {
+        min = (InetAddress)val;
+      } else {
+        max = (InetAddress) val;
+      }
+    }
+
+    @Override
+    protected boolean isEqual(Range o) {
+      IpRange other = (IpRange)o;
+      return this.min.equals(other.min) && this.max.equals(other.max);
+    }
+
+    @Override
+    protected boolean isDisjoint(Range o) {
+      IpRange other = (IpRange)o;
+      byte[] bMin = min.getAddress();
+      byte[] bMax = max.getAddress();
+      return StringHelper.compare(bMin.length, bMin, 0, other.max.getAddress(), 0) > 0 ||
+          StringHelper.compare(bMax.length, bMax, 0, other.min.getAddress(), 0) < 0;
+    }
+
+    @Override
+    protected boolean isWithin(Range o) {
+      IpRange other = (IpRange)o;
+      byte[] bMin = min.getAddress();
+      byte[] bMax = max.getAddress();
+      return StringHelper.compare(bMin.length, bMin, 0, other.min.getAddress(), 0) >= 0 &&
+          StringHelper.compare(bMax.length, bMax, 0, other.max.getAddress(), 0) <= 0;
+    }
+
+    @Override
+    protected boolean contains(Range o) {
+      IpRange other = (IpRange)o;
+      byte[] bMin = min.getAddress();
+      byte[] bMax = max.getAddress();
+      return StringHelper.compare(bMin.length, bMin, 0, other.min.getAddress(), 0) <= 0 &&
+          StringHelper.compare(bMax.length, bMax, 0, other.max.getAddress(), 0) >= 0;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder b = new StringBuilder();
+      b.append("Box(");
+      b.append(min.getHostAddress());
+      b.append(" TO ");
+      b.append(max.getHostAddress());
+      b.append(")");
+      return b.toString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/queries/src/test/org/apache/lucene/queries/function/TestValueSources.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/function/TestValueSources.java b/lucene/queries/src/test/org/apache/lucene/queries/function/TestValueSources.java
index e008293..8008590 100644
--- a/lucene/queries/src/test/org/apache/lucene/queries/function/TestValueSources.java
+++ b/lucene/queries/src/test/org/apache/lucene/queries/function/TestValueSources.java
@@ -123,9 +123,9 @@ public class TestValueSources extends LuceneTestCase {
       document.add(new StringField("id", doc[0], Field.Store.NO));
       document.add(new SortedDocValuesField("id", new BytesRef(doc[0])));
       document.add(new NumericDocValuesField("double", Double.doubleToRawLongBits(Double.parseDouble(doc[1]))));
-      document.add(new NumericDocValuesField("float", Float.floatToRawIntBits(Float.valueOf(doc[2]))));
-      document.add(new NumericDocValuesField("int", Integer.valueOf(doc[3])));
-      document.add(new NumericDocValuesField("long", Long.valueOf(doc[4])));
+      document.add(new NumericDocValuesField("float", Float.floatToRawIntBits(Float.parseFloat(doc[2]))));
+      document.add(new NumericDocValuesField("int", Integer.parseInt(doc[3])));
+      document.add(new NumericDocValuesField("long", Long.parseLong(doc[4])));
       document.add(new StringField("string", doc[5], Field.Store.NO));
       document.add(new SortedDocValuesField("string", new BytesRef(doc[5])));
       document.add(new TextField("text", doc[6], Field.Store.NO));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
index 3cfa7d0..cff9efa 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
@@ -837,7 +837,7 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
     Query q;
     float fms = fuzzyMinSim;
     try {
-      fms = Float.valueOf(fuzzySlop.image.substring(1)).floatValue();
+      fms = Float.parseFloat(fuzzySlop.image.substring(1));
     } catch (Exception ignored) { }
     if(fms < 0.0f){
       throw new ParseException("Minimum similarity for a FuzzyQuery has to be between 0.0f and 1.0f !");
@@ -853,7 +853,7 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
     int s = phraseSlop;  // default
     if (fuzzySlop != null) {
       try {
-        s = Float.valueOf(fuzzySlop.image.substring(1)).intValue();
+        s = (int)Float.parseFloat(fuzzySlop.image.substring(1));
       }
       catch (Exception ignored) { }
     }
@@ -865,7 +865,7 @@ public abstract class QueryParserBase extends QueryBuilder implements CommonQuer
     if (boost != null) {
       float f = (float) 1.0;
       try {
-        f = Float.valueOf(boost.image).floatValue();
+        f = Float.parseFloat(boost.image);
       }
       catch (Exception ignored) {
     /* Should this be handled somehow? (defaults to "no boost", if

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.java
index ed0d67c..8ba34a6 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.java
@@ -466,7 +466,7 @@ public class StandardSyntaxParser implements SyntaxParser, StandardSyntaxParserC
       if (boost != null) {
       float f = (float)1.0;
       try {
-        f = Float.valueOf(boost.image).floatValue();
+        f = Float.parseFloat(boost.image);
         // avoid boosting null queries, such as those caused by stop words
           if (q != null) {
             q = new BoostQueryNode(q, f);
@@ -542,7 +542,7 @@ public class StandardSyntaxParser implements SyntaxParser, StandardSyntaxParserC
        if (fuzzy) {
            float fms = defaultMinSimilarity;
            try {
-            fms = Float.valueOf(fuzzySlop.image.substring(1)).floatValue();
+            fms = Float.parseFloat(fuzzySlop.image.substring(1));
            } catch (Exception ignored) { }
            if(fms < 0.0f){
                 {if (true) throw new ParseException(new MessageImpl(QueryParserMessages.INVALID_SYNTAX_FUZZY_LIMITS));}
@@ -661,7 +661,7 @@ public class StandardSyntaxParser implements SyntaxParser, StandardSyntaxParserC
 
          if (fuzzySlop != null) {
            try {
-             phraseSlop = Float.valueOf(fuzzySlop.image.substring(1)).intValue();
+             phraseSlop = (int)Float.parseFloat(fuzzySlop.image.substring(1));
              q = new SlopQueryNode(q, phraseSlop);
            }
            catch (Exception ignored) {
@@ -679,7 +679,7 @@ public class StandardSyntaxParser implements SyntaxParser, StandardSyntaxParserC
     if (boost != null) {
       float f = (float)1.0;
       try {
-        f = Float.valueOf(boost.image).floatValue();
+        f = Float.parseFloat(boost.image);
         // avoid boosting null queries, such as those caused by stop words
           if (q != null) {
             q = new BoostQueryNode(q, f);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.jj
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.jj b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.jj
index 868b257..b53bab3 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.jj
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/StandardSyntaxParser.jj
@@ -391,7 +391,7 @@ QueryNode Clause(CharSequence field) : {
       if (boost != null) {
       float f = (float)1.0;
       try {
-        f = Float.valueOf(boost.image).floatValue();
+        f = Float.parseFloat(boost.image);
         // avoid boosting null queries, such as those caused by stop words
           if (q != null) {
             q = new BoostQueryNode(q, f);
@@ -431,7 +431,7 @@ QueryNode Term(CharSequence field) : {
        if (fuzzy) {
            float fms = defaultMinSimilarity;
            try {
-            fms = Float.valueOf(fuzzySlop.image.substring(1)).floatValue();
+            fms = Float.parseFloat(fuzzySlop.image.substring(1));
            } catch (Exception ignored) { }
            if(fms < 0.0f){
                 throw new ParseException(new MessageImpl(QueryParserMessages.INVALID_SYNTAX_FUZZY_LIMITS));
@@ -472,7 +472,7 @@ QueryNode Term(CharSequence field) : {
 
          if (fuzzySlop != null) {
            try {
-             phraseSlop = Float.valueOf(fuzzySlop.image.substring(1)).intValue();
+             phraseSlop = (int)Float.parseFloat(fuzzySlop.image.substring(1));
              q = new SlopQueryNode(q, phraseSlop);    
            }
            catch (Exception ignored) {
@@ -488,7 +488,7 @@ QueryNode Term(CharSequence field) : {
     if (boost != null) {
       float f = (float)1.0;
       try {
-        f = Float.valueOf(boost.image).floatValue();
+        f = Float.parseFloat(boost.image);
         // avoid boosting null queries, such as those caused by stop words
           if (q != null) {
             q = new BoostQueryNode(q, f);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/887d1b18/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/QueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/QueryParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/QueryParser.java
index bd91f04..f0f4b34 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/QueryParser.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/QueryParser.java
@@ -481,7 +481,7 @@ public class QueryParser implements QueryParserConstants {
       weight = jj_consume_token(NUMBER);
       float f;
       try {
-        f = Float.valueOf(weight.image).floatValue();
+        f = Float.parseFloat(weight.image);
       } catch (Exception floatExc) {
         {if (true) throw new ParseException(boostErrorMessage + weight.image + " (" + floatExc + ")");}
       }