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

[11/23] lucene-solr:jira/solr-9835: LUCENE-7740: Refactor Range Fields to remove Field suffix (e.g., DoubleRange), move InetAddressRange and InetAddressPoint from sandbox to misc module, and refactor all other range fields from sandbox to core.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
deleted file mode 100644
index 5cda742..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressPoint.java
+++ /dev/null
@@ -1,313 +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.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/d34d81f9/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressRangeField.java b/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressRangeField.java
deleted file mode 100644
index c6ebc83..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/document/InetAddressRangeField.java
+++ /dev/null
@@ -1,168 +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.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 InetAddressRangeField 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 InetAddressRangeField 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 InetAddressRangeField(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 InetAddressRangeField.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/d34d81f9/lucene/sandbox/src/java/org/apache/lucene/document/IntRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/IntRangeField.java b/lucene/sandbox/src/java/org/apache/lucene/document/IntRangeField.java
deleted file mode 100644
index 53a3311..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/document/IntRangeField.java
+++ /dev/null
@@ -1,282 +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.document;
-
-import org.apache.lucene.document.RangeFieldQuery.QueryType;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.NumericUtils;
-
-/**
- * An indexed Integer Range field.
- * <p>
- * This field indexes dimensional ranges defined as min/max pairs. It supports
- * up to a maximum of 4 dimensions (indexed as 8 numeric values). With 1 dimension representing a single integer range,
- * 2 dimensions representing a bounding box, 3 dimensions a bounding cube, and 4 dimensions a tesseract.
- * <p>
- * Multiple values for the same field in one document is supported, and open ended ranges can be defined using
- * {@code Integer.MIN_VALUE} and {@code Integer.MAX_VALUE}.
- *
- * <p>
- * This field defines the following static factory methods for common search operations over integer ranges:
- * <ul>
- *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ranges that intersect the defined search range.
- *   <li>{@link #newWithinQuery newWithinQuery()} matches ranges that are within the defined search range.
- *   <li>{@link #newContainsQuery newContainsQuery()} matches ranges that contain the defined search range.
- * </ul>
- */
-public class IntRangeField extends Field {
-  /** stores integer values so number of bytes is 4 */
-  public static final int BYTES = Integer.BYTES;
-
-  /**
-   * Create a new IntRangeField type, from min/max parallel arrays
-   *
-   * @param name field name. must not be null.
-   * @param min range min values; each entry is the min value for the dimension
-   * @param max range max values; each entry is the max value for the dimension
-   */
-  public IntRangeField(String name, final int[] min, final int[] max) {
-    super(name, getType(min.length));
-    setRangeValues(min, max);
-  }
-
-  /** set the field type */
-  private static FieldType getType(int dimensions) {
-    if (dimensions > 4) {
-      throw new IllegalArgumentException("IntRangeField does not support greater than 4 dimensions");
-    }
-
-    FieldType ft = new FieldType();
-    // dimensions is set as 2*dimension size (min/max per dimension)
-    ft.setDimensions(dimensions*2, BYTES);
-    ft.freeze();
-    return ft;
-  }
-
-  /**
-   * Changes the values of the field.
-   * @param min array of min values. (accepts {@code Integer.NEGATIVE_INFINITY})
-   * @param max array of max values. (accepts {@code Integer.POSITIVE_INFINITY})
-   * @throws IllegalArgumentException if {@code min} or {@code max} is invalid
-   */
-  public void setRangeValues(int[] min, int[] max) {
-    checkArgs(min, max);
-    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
-      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
-          + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
-    }
-
-    final byte[] bytes;
-    if (fieldsData == null) {
-      bytes = new byte[BYTES*2*min.length];
-      fieldsData = new BytesRef(bytes);
-    } else {
-      bytes = ((BytesRef)fieldsData).bytes;
-    }
-    verifyAndEncode(min, max, bytes);
-  }
-
-  /** validate the arguments */
-  private static void checkArgs(final int[] min, final int[] max) {
-    if (min == null || max == null || min.length == 0 || max.length == 0) {
-      throw new IllegalArgumentException("min/max range values cannot be null or empty");
-    }
-    if (min.length != max.length) {
-      throw new IllegalArgumentException("min/max ranges must agree");
-    }
-    if (min.length > 4) {
-      throw new IllegalArgumentException("IntRangeField does not support greater than 4 dimensions");
-    }
-  }
-
-  /**
-   * Encodes the min, max ranges into a byte array
-   */
-  private static byte[] encode(int[] min, int[] max) {
-    checkArgs(min, max);
-    byte[] b = new byte[BYTES*2*min.length];
-    verifyAndEncode(min, max, b);
-    return b;
-  }
-
-  /**
-   * encode the ranges into a sortable byte array ({@code Double.NaN} not allowed)
-   * <p>
-   * example for 4 dimensions (8 bytes per dimension value):
-   * minD1 ... minD4 | maxD1 ... maxD4
-   */
-  static void verifyAndEncode(int[] min, int[] max, byte[] bytes) {
-    for (int d=0,i=0,j=min.length*BYTES; d<min.length; ++d, i+=BYTES, j+=BYTES) {
-      if (Double.isNaN(min[d])) {
-        throw new IllegalArgumentException("invalid min value (" + Double.NaN + ")" + " in IntRangeField");
-      }
-      if (Double.isNaN(max[d])) {
-        throw new IllegalArgumentException("invalid max value (" + Double.NaN + ")" + " in IntRangeField");
-      }
-      if (min[d] > max[d]) {
-        throw new IllegalArgumentException("min value (" + min[d] + ") is greater than max value (" + max[d] + ")");
-      }
-      encode(min[d], bytes, i);
-      encode(max[d], bytes, j);
-    }
-  }
-
-  /** encode the given value into the byte array at the defined offset */
-  private static void encode(int val, byte[] bytes, int offset) {
-    NumericUtils.intToSortableBytes(val, bytes, offset);
-  }
-
-  /**
-   * Get the min value for the given dimension
-   * @param dimension the dimension, always positive
-   * @return the decoded min value
-   */
-  public int getMin(int dimension) {
-    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
-      throw new IllegalArgumentException("dimension request (" + dimension +
-          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
-    }
-    return decodeMin(((BytesRef)fieldsData).bytes, dimension);
-  }
-
-  /**
-   * Get the max value for the given dimension
-   * @param dimension the dimension, always positive
-   * @return the decoded max value
-   */
-  public int getMax(int dimension) {
-    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
-      throw new IllegalArgumentException("dimension request (" + dimension +
-          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
-    }
-    return decodeMax(((BytesRef)fieldsData).bytes, dimension);
-  }
-
-  /** decodes the min value (for the defined dimension) from the encoded input byte array */
-  static int decodeMin(byte[] b, int dimension) {
-    int offset = dimension*BYTES;
-    return NumericUtils.sortableBytesToInt(b, offset);
-  }
-
-  /** decodes the max value (for the defined dimension) from the encoded input byte array */
-  static int decodeMax(byte[] b, int dimension) {
-    int offset = b.length/2 + dimension*BYTES;
-    return NumericUtils.sortableBytesToInt(b, offset);
-  }
-
-  /**
-   * Create a query for matching indexed ranges that intersect the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Integer.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Integer.MAX_VALUE})
-   * @return query for matching intersecting ranges (overlap, within, or contains)
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newIntersectsQuery(String field, final int[] min, final int[] max) {
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.INTERSECTS) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return IntRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that contain the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Integer.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Integer.MAX_VALUE})
-   * @return query for matching ranges that contain the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newContainsQuery(String field, final int[] min, final int[] max) {
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CONTAINS) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return IntRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that are within the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Integer.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Integer.MAX_VALUE})
-   * @return query for matching ranges within the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newWithinQuery(String field, final int[] min, final int[] max) {
-    checkArgs(min, max);
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.WITHIN) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return IntRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that cross the defined range.
-   * A CROSSES is defined as any set of ranges that are not disjoint and not wholly contained by
-   * the query. Effectively, its the complement of union(WITHIN, DISJOINT).
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Integer.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Integer.MAX_VALUE})
-   * @return query for matching ranges within the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newCrossesQuery(String field, final int[] min, final int[] max) {
-    checkArgs(min, max);
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CROSSES) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return IntRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append(getClass().getSimpleName());
-    sb.append(" <");
-    sb.append(name);
-    sb.append(':');
-    byte[] b = ((BytesRef)fieldsData).bytes;
-    toString(b, 0);
-    for (int d=1; d<type.pointDimensionCount(); ++d) {
-      sb.append(' ');
-      toString(b, d);
-    }
-    sb.append('>');
-
-    return sb.toString();
-  }
-
-  /**
-   * Returns the String representation for the range at the given dimension
-   * @param ranges the encoded ranges, never null
-   * @param dimension the dimension of interest
-   * @return The string representation for the range at the provided dimension
-   */
-  private static String toString(byte[] ranges, int dimension) {
-    return "[" + Integer.toString(decodeMin(ranges, dimension)) + " : "
-        + Integer.toString(decodeMax(ranges, dimension)) + "]";
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/java/org/apache/lucene/document/LongRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LongRangeField.java b/lucene/sandbox/src/java/org/apache/lucene/document/LongRangeField.java
deleted file mode 100644
index 7addaf5..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LongRangeField.java
+++ /dev/null
@@ -1,280 +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.document;
-
-import org.apache.lucene.document.RangeFieldQuery.QueryType;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.NumericUtils;
-
-/**
- * An indexed Long Range field.
- * <p>
- * This field indexes dimensional ranges defined as min/max pairs. It supports
- * up to a maximum of 4 dimensions (indexed as 8 numeric values). With 1 dimension representing a single long range,
- * 2 dimensions representing a bounding box, 3 dimensions a bounding cube, and 4 dimensions a tesseract.
- * <p>
- * Multiple values for the same field in one document is supported, and open ended ranges can be defined using
- * {@code Long.MIN_VALUE} and {@code Long.MAX_VALUE}.
- *
- * <p>
- * This field defines the following static factory methods for common search operations over long ranges:
- * <ul>
- *   <li>{@link #newIntersectsQuery newIntersectsQuery()} matches ranges that intersect the defined search range.
- *   <li>{@link #newWithinQuery newWithinQuery()} matches ranges that are within the defined search range.
- *   <li>{@link #newContainsQuery newContainsQuery()} matches ranges that contain the defined search range.
- * </ul>
- */
-public class LongRangeField extends Field {
-  /** stores long values so number of bytes is 8 */
-  public static final int BYTES = Long.BYTES;
-
-  /**
-   * Create a new LongRangeField type, from min/max parallel arrays
-   *
-   * @param name field name. must not be null.
-   * @param min range min values; each entry is the min value for the dimension
-   * @param max range max values; each entry is the max value for the dimension
-   */
-  public LongRangeField(String name, final long[] min, final long[] max) {
-    super(name, getType(min.length));
-    setRangeValues(min, max);
-  }
-
-  /** set the field type */
-  private static FieldType getType(int dimensions) {
-    if (dimensions > 4) {
-      throw new IllegalArgumentException("LongRangeField does not support greater than 4 dimensions");
-    }
-
-    FieldType ft = new FieldType();
-    // dimensions is set as 2*dimension size (min/max per dimension)
-    ft.setDimensions(dimensions*2, BYTES);
-    ft.freeze();
-    return ft;
-  }
-
-  /**
-   * Changes the values of the field.
-   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
-   * @throws IllegalArgumentException if {@code min} or {@code max} is invalid
-   */
-  public void setRangeValues(long[] min, long[] max) {
-    checkArgs(min, max);
-    if (min.length*2 != type.pointDimensionCount() || max.length*2 != type.pointDimensionCount()) {
-      throw new IllegalArgumentException("field (name=" + name + ") uses " + type.pointDimensionCount()/2
-          + " dimensions; cannot change to (incoming) " + min.length + " dimensions");
-    }
-
-    final byte[] bytes;
-    if (fieldsData == null) {
-      bytes = new byte[BYTES*2*min.length];
-      fieldsData = new BytesRef(bytes);
-    } else {
-      bytes = ((BytesRef)fieldsData).bytes;
-    }
-    verifyAndEncode(min, max, bytes);
-  }
-
-  /** validate the arguments */
-  private static void checkArgs(final long[] min, final long[] max) {
-    if (min == null || max == null || min.length == 0 || max.length == 0) {
-      throw new IllegalArgumentException("min/max range values cannot be null or empty");
-    }
-    if (min.length != max.length) {
-      throw new IllegalArgumentException("min/max ranges must agree");
-    }
-    if (min.length > 4) {
-      throw new IllegalArgumentException("LongRangeField does not support greater than 4 dimensions");
-    }
-  }
-
-  /** Encodes the min, max ranges into a byte array */
-  private static byte[] encode(long[] min, long[] max) {
-    checkArgs(min, max);
-    byte[] b = new byte[BYTES*2*min.length];
-    verifyAndEncode(min, max, b);
-    return b;
-  }
-
-  /**
-   * encode the ranges into a sortable byte array ({@code Double.NaN} not allowed)
-   * <p>
-   * example for 4 dimensions (8 bytes per dimension value):
-   * minD1 ... minD4 | maxD1 ... maxD4
-   */
-  static void verifyAndEncode(long[] min, long[] max, byte[] bytes) {
-    for (int d=0,i=0,j=min.length*BYTES; d<min.length; ++d, i+=BYTES, j+=BYTES) {
-      if (Double.isNaN(min[d])) {
-        throw new IllegalArgumentException("invalid min value (" + Double.NaN + ")" + " in IntRangeField");
-      }
-      if (Double.isNaN(max[d])) {
-        throw new IllegalArgumentException("invalid max value (" + Double.NaN + ")" + " in IntRangeField");
-      }
-      if (min[d] > max[d]) {
-        throw new IllegalArgumentException("min value (" + min[d] + ") is greater than max value (" + max[d] + ")");
-      }
-      encode(min[d], bytes, i);
-      encode(max[d], bytes, j);
-    }
-  }
-
-  /** encode the given value into the byte array at the defined offset */
-  private static void encode(long val, byte[] bytes, int offset) {
-    NumericUtils.longToSortableBytes(val, bytes, offset);
-  }
-
-  /**
-   * Get the min value for the given dimension
-   * @param dimension the dimension, always positive
-   * @return the decoded min value
-   */
-  public long getMin(int dimension) {
-    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
-      throw new IllegalArgumentException("dimension request (" + dimension +
-          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
-    }
-    return decodeMin(((BytesRef)fieldsData).bytes, dimension);
-  }
-
-  /**
-   * Get the max value for the given dimension
-   * @param dimension the dimension, always positive
-   * @return the decoded max value
-   */
-  public long getMax(int dimension) {
-    if (dimension < 0 || dimension >= type.pointDimensionCount()/2) {
-      throw new IllegalArgumentException("dimension request (" + dimension +
-          ") out of bounds for field (name=" + name + " dimensions=" + type.pointDimensionCount()/2 + "). ");
-    }
-    return decodeMax(((BytesRef)fieldsData).bytes, dimension);
-  }
-
-  /** decodes the min value (for the defined dimension) from the encoded input byte array */
-  static long decodeMin(byte[] b, int dimension) {
-    int offset = dimension*BYTES;
-    return NumericUtils.sortableBytesToLong(b, offset);
-  }
-
-  /** decodes the max value (for the defined dimension) from the encoded input byte array */
-  static long decodeMax(byte[] b, int dimension) {
-    int offset = b.length/2 + dimension*BYTES;
-    return NumericUtils.sortableBytesToLong(b, offset);
-  }
-
-  /**
-   * Create a query for matching indexed ranges that intersect the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
-   * @return query for matching intersecting ranges (overlap, within, or contains)
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newIntersectsQuery(String field, final long[] min, final long[] max) {
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.INTERSECTS) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return LongRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that contain the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
-   * @return query for matching ranges that contain the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newContainsQuery(String field, final long[] min, final long[] max) {
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CONTAINS) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return LongRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that are within the defined range.
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
-   * @return query for matching ranges within the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newWithinQuery(String field, final long[] min, final long[] max) {
-    checkArgs(min, max);
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.WITHIN) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return LongRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  /**
-   * Create a query for matching indexed ranges that cross the defined range.
-   * A CROSSES is defined as any set of ranges that are not disjoint and not wholly contained by
-   * the query. Effectively, its the complement of union(WITHIN, DISJOINT).
-   * @param field field name. must not be null.
-   * @param min array of min values. (accepts {@code Long.MIN_VALUE})
-   * @param max array of max values. (accepts {@code Long.MAX_VALUE})
-   * @return query for matching ranges within the defined range
-   * @throws IllegalArgumentException if {@code field} is null, {@code min} or {@code max} is invalid
-   */
-  public static Query newCrossesQuery(String field, final long[] min, final long[] max) {
-    checkArgs(min, max);
-    return new RangeFieldQuery(field, encode(min, max), min.length, QueryType.CROSSES) {
-      @Override
-      protected String toString(byte[] ranges, int dimension) {
-        return LongRangeField.toString(ranges, dimension);
-      }
-    };
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append(getClass().getSimpleName());
-    sb.append(" <");
-    sb.append(name);
-    sb.append(':');
-    byte[] b = ((BytesRef)fieldsData).bytes;
-    toString(b, 0);
-    for (int d=1; d<type.pointDimensionCount(); ++d) {
-      sb.append(' ');
-      toString(b, d);
-    }
-    sb.append('>');
-
-    return sb.toString();
-  }
-
-  /**
-   * Returns the String representation for the range at the given dimension
-   * @param ranges the encoded ranges, never null
-   * @param dimension the dimension of interest
-   * @return The string representation for the range at the provided dimension
-   */
-  private static String toString(byte[] ranges, int dimension) {
-    return "[" + Long.toString(decodeMin(ranges, dimension)) + " : "
-        + Long.toString(decodeMax(ranges, dimension)) + "]";
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
deleted file mode 100644
index 10f10fa..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
+++ /dev/null
@@ -1,340 +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.document;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Objects;
-import java.util.function.IntPredicate;
-import java.util.function.Predicate;
-
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.PointValues;
-import org.apache.lucene.index.PointValues.Relation;
-import org.apache.lucene.index.PointValues.IntersectVisitor;
-import org.apache.lucene.search.ConstantScoreScorer;
-import org.apache.lucene.search.ConstantScoreWeight;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Weight;
-import org.apache.lucene.util.DocIdSetBuilder;
-import org.apache.lucene.util.StringHelper;
-
-/**
- * Query class for searching {@code RangeField} types by a defined {@link Relation}.
- */
-abstract class RangeFieldQuery extends Query {
-  /** field name */
-  final String field;
-  /** query relation
-   * intersects: {@code CELL_CROSSES_QUERY},
-   * contains: {@code CELL_CONTAINS_QUERY},
-   * within: {@code CELL_WITHIN_QUERY} */
-  final QueryType queryType;
-  /** number of dimensions - max 4 */
-  final int numDims;
-  /** ranges encoded as a sortable byte array */
-  final byte[] ranges;
-  /** number of bytes per dimension */
-  final int bytesPerDim;
-
-  /** Used by {@code RangeFieldQuery} to check how each internal or leaf node relates to the query. */
-  enum QueryType {
-    /** Use this for intersects queries. */
-    INTERSECTS,
-    /** Use this for within queries. */
-    WITHIN,
-    /** Use this for contains */
-    CONTAINS,
-    /** Use this for crosses queries */
-    CROSSES
-  }
-
-  /**
-   * Create a query for searching indexed ranges that match the provided relation.
-   * @param field field name. must not be null.
-   * @param ranges encoded range values; this is done by the {@code RangeField} implementation
-   * @param queryType the query relation
-   */
-  RangeFieldQuery(String field, final byte[] ranges, final int numDims, final QueryType queryType) {
-    checkArgs(field, ranges, numDims);
-    if (queryType == null) {
-      throw new IllegalArgumentException("Query type cannot be null");
-    }
-    this.field = field;
-    this.queryType = queryType;
-    this.numDims = numDims;
-    this.ranges = ranges;
-    this.bytesPerDim = ranges.length / (2*numDims);
-  }
-
-  /** check input arguments */
-  private static void checkArgs(String field, final byte[] ranges, final int numDims) {
-    if (field == null) {
-      throw new IllegalArgumentException("field must not be null");
-    }
-    if (numDims > 4) {
-      throw new IllegalArgumentException("dimension size cannot be greater than 4");
-    }
-    if (ranges == null || ranges.length == 0) {
-      throw new IllegalArgumentException("encoded ranges cannot be null or empty");
-    }
-  }
-
-  /** Check indexed field info against the provided query data. */
-  private void checkFieldInfo(FieldInfo fieldInfo) {
-    if (fieldInfo.getPointDimensionCount()/2 != numDims) {
-      throw new IllegalArgumentException("field=\"" + field + "\" was indexed with numDims="
-          + fieldInfo.getPointDimensionCount()/2 + " but this query has numDims=" + numDims);
-    }
-  }
-
-  @Override
-  public final Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
-    return new ConstantScoreWeight(this, boost) {
-      final RangeFieldComparator target = new RangeFieldComparator();
-      private DocIdSet buildMatchingDocIdSet(LeafReader reader, PointValues values) throws IOException {
-        DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
-        values.intersect(
-            new IntersectVisitor() {
-              DocIdSetBuilder.BulkAdder adder;
-              @Override
-              public void grow(int count) {
-                adder = result.grow(count);
-              }
-              @Override
-              public void visit(int docID) throws IOException {
-                adder.add(docID);
-              }
-              @Override
-              public void visit(int docID, byte[] leaf) throws IOException {
-                if (target.matches(leaf)) {
-                  adder.add(docID);
-                }
-              }
-              @Override
-              public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-                byte[] node = getInternalRange(minPackedValue, maxPackedValue);
-                // compute range relation for BKD traversal
-                if (target.intersects(node) == false) {
-                  return Relation.CELL_OUTSIDE_QUERY;
-                } else if (target.within(node)) {
-                  // target within cell; continue traversing:
-                  return Relation.CELL_CROSSES_QUERY;
-                } else if (target.contains(node)) {
-                  // target contains cell; add iff queryType is not a CONTAINS or CROSSES query:
-                  return (queryType == QueryType.CONTAINS || queryType == QueryType.CROSSES) ?
-                      Relation.CELL_OUTSIDE_QUERY : Relation.CELL_INSIDE_QUERY;
-                }
-                // target intersects cell; continue traversing:
-                return Relation.CELL_CROSSES_QUERY;
-              }
-            });
-        return result.build();
-      }
-
-      @Override
-      public Scorer scorer(LeafReaderContext context) throws IOException {
-        LeafReader reader = context.reader();
-        PointValues values = reader.getPointValues(field);
-        if (values == null) {
-          // no docs in this segment indexed any ranges
-          return null;
-        }
-        FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field);
-        if (fieldInfo == null) {
-          // no docs in this segment indexed this field
-          return null;
-        }
-        checkFieldInfo(fieldInfo);
-        boolean allDocsMatch = true;
-        if (values.getDocCount() == reader.maxDoc()) {
-          // if query crosses, docs need to be further scrutinized
-          byte[] range = getInternalRange(values.getMinPackedValue(), values.getMaxPackedValue());
-          // if the internal node is not equal and not contained by the query, all docs do not match
-          if (queryType == QueryType.CROSSES || (!Arrays.equals(ranges, range)
-              && (target.contains(range) == false || queryType != QueryType.WITHIN))) {
-            allDocsMatch = false;
-          }
-        } else {
-          allDocsMatch = false;
-        }
-
-        DocIdSetIterator iterator = allDocsMatch == true ?
-            DocIdSetIterator.all(reader.maxDoc()) : buildMatchingDocIdSet(reader, values).iterator();
-        return new ConstantScoreScorer(this, score(), iterator);
-      }
-
-      /** get an encoded byte representation of the internal node; this is
-       *  the lower half of the min array and the upper half of the max array */
-      private byte[] getInternalRange(byte[] min, byte[] max) {
-        byte[] range = new byte[min.length];
-        final int dimSize = numDims * bytesPerDim;
-        System.arraycopy(min, 0, range, 0, dimSize);
-        System.arraycopy(max, dimSize, range, dimSize, dimSize);
-        return range;
-      }
-    };
-  }
-
-  /**
-   * RangeFieldComparator class provides the core comparison logic for accepting or rejecting indexed
-   * {@code RangeField} types based on the defined query range and relation.
-   */
-  class RangeFieldComparator {
-    final Predicate<byte[]> predicate;
-
-    /** constructs the comparator based on the query type */
-    RangeFieldComparator() {
-      switch (queryType) {
-        case INTERSECTS:
-          predicate = this::intersects;
-          break;
-        case WITHIN:
-          predicate = this::contains;
-          break;
-        case CONTAINS:
-          predicate = this::within;
-          break;
-        case CROSSES:
-          // crosses first checks intersection (disjoint automatic fails),
-          // then ensures the query doesn't wholly contain the leaf:
-          predicate = (byte[] leaf) -> this.intersects(leaf)
-              && this.contains(leaf) == false;
-          break;
-        default:
-          throw new IllegalArgumentException("invalid queryType [" + queryType + "] found.");
-      }
-    }
-
-    /** determines if the candidate range matches the query request */
-    private boolean matches(final byte[] candidate) {
-      return (Arrays.equals(ranges, candidate) && queryType != QueryType.CROSSES)
-          || predicate.test(candidate);
-    }
-
-    /** check if query intersects candidate range */
-    private boolean intersects(final byte[] candidate) {
-      return relate((int d) -> compareMinMax(candidate, d) > 0 || compareMaxMin(candidate, d) < 0);
-    }
-
-    /** check if query is within candidate range */
-    private boolean within(final byte[] candidate) {
-      return relate((int d) -> compareMinMin(candidate, d) < 0 || compareMaxMax(candidate, d) > 0);
-    }
-
-    /** check if query contains candidate range */
-    private boolean contains(final byte[] candidate) {
-      return relate((int d) -> compareMinMin(candidate, d) > 0 || compareMaxMax(candidate, d) < 0);
-    }
-
-    /** internal method used by each relation method to test range relation logic */
-    private boolean relate(IntPredicate predicate) {
-      for (int d=0; d<numDims; ++d) {
-        if (predicate.test(d)) {
-          return false;
-        }
-      }
-      return true;
-    }
-
-    /** compare the encoded min value (for the defined query dimension) with the encoded min value in the byte array */
-    private int compareMinMin(byte[] b, int dimension) {
-      // convert dimension to offset:
-      dimension *= bytesPerDim;
-      return StringHelper.compare(bytesPerDim, ranges, dimension, b, dimension);
-    }
-
-    /** compare the encoded min value (for the defined query dimension) with the encoded max value in the byte array */
-    private int compareMinMax(byte[] b, int dimension) {
-      // convert dimension to offset:
-      dimension *= bytesPerDim;
-      return StringHelper.compare(bytesPerDim, ranges, dimension, b, numDims * bytesPerDim + dimension);
-    }
-
-    /** compare the encoded max value (for the defined query dimension) with the encoded min value in the byte array */
-    private int compareMaxMin(byte[] b, int dimension) {
-      // convert dimension to offset:
-      dimension *= bytesPerDim;
-      return StringHelper.compare(bytesPerDim, ranges, numDims * bytesPerDim + dimension, b, dimension);
-    }
-
-    /** compare the encoded max value (for the defined query dimension) with the encoded max value in the byte array */
-    private int compareMaxMax(byte[] b, int dimension) {
-      // convert dimension to max offset:
-      dimension = numDims * bytesPerDim + dimension * bytesPerDim;
-      return StringHelper.compare(bytesPerDim, ranges, dimension, b, dimension);
-    }
-  }
-
-  @Override
-  public int hashCode() {
-    int hash = classHash();
-    hash = 31 * hash + field.hashCode();
-    hash = 31 * hash + numDims;
-    hash = 31 * hash + queryType.hashCode();
-    hash = 31 * hash + Arrays.hashCode(ranges);
-
-    return hash;
-  }
-
-  @Override
-  public final boolean equals(Object o) {
-    return sameClassAs(o) &&
-        equalsTo(getClass().cast(o));
-  }
-
-  protected boolean equalsTo(RangeFieldQuery other) {
-    return Objects.equals(field, other.field) &&
-        numDims == other.numDims &&
-        Arrays.equals(ranges, other.ranges) &&
-        other.queryType == queryType;
-  }
-
-  @Override
-  public String toString(String field) {
-    StringBuilder sb = new StringBuilder();
-    if (this.field.equals(field) == false) {
-      sb.append(this.field);
-      sb.append(':');
-    }
-    sb.append("<ranges:");
-    sb.append(toString(ranges, 0));
-    for (int d=1; d<numDims; ++d) {
-      sb.append(' ');
-      sb.append(toString(ranges, d));
-    }
-    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()}.
-   *
-   * @param dimension dimension of the particular value
-   * @param ranges encoded ranges, never null
-   * @return human readable value for debugging
-   */
-  protected abstract String toString(byte[] ranges, int dimension);
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/java/org/apache/lucene/document/package.html
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/package.html b/lucene/sandbox/src/java/org/apache/lucene/document/package.html
index b6a077e..4783b5e 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/package.html
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/package.html
@@ -26,8 +26,7 @@
 This package contains several point types:
 <ul>
    <li>{@link org.apache.lucene.document.BigIntegerPoint BigIntegerPoint} for 128-bit integers</li>
-   <li>{@link org.apache.lucene.document.InetAddressPoint InetAddressPoint} for IPv4 and IPv6 network addresses</li>
-   <li>{@link org.apache.lucene.document.LatLonPoint LatLonPoint} for latitude/longitude geospatial points</li> 
+   <li>{@link org.apache.lucene.document.LatLonPoint LatLonPoint} for latitude/longitude geospatial points</li>
 </ul>
 </body>
 </html>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/test/org/apache/lucene/document/TestDoubleRangeField.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestDoubleRangeField.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestDoubleRangeField.java
index 188aab6..8a8130b 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestDoubleRangeField.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/document/TestDoubleRangeField.java
@@ -30,11 +30,11 @@ public class TestDoubleRangeField extends LuceneTestCase {
     IllegalArgumentException expected;
 
     expected = expectThrows(IllegalArgumentException.class, () ->
-        doc.add(new DoubleRangeField(FIELD_NAME, new double[] {Double.NaN}, new double[] {5})));
+        doc.add(new DoubleRange(FIELD_NAME, new double[] {Double.NaN}, new double[] {5})));
     assertTrue(expected.getMessage().contains("invalid min value"));
 
     expected = expectThrows(IllegalArgumentException.class, () ->
-        doc.add(new DoubleRangeField(FIELD_NAME, new double[] {5}, new double[] {Double.NaN})));
+        doc.add(new DoubleRange(FIELD_NAME, new double[] {5}, new double[] {Double.NaN})));
     assertTrue(expected.getMessage().contains("invalid max value"));
   }
 
@@ -43,7 +43,7 @@ public class TestDoubleRangeField extends LuceneTestCase {
     Document doc = new Document();
     IllegalArgumentException expected;
     expected = expectThrows(IllegalArgumentException.class, () ->
-        doc.add(new DoubleRangeField(FIELD_NAME, new double[] {5, 6}, new double[] {5})));
+        doc.add(new DoubleRange(FIELD_NAME, new double[] {5, 6}, new double[] {5})));
     assertTrue(expected.getMessage().contains("min/max ranges must agree"));
   }
 
@@ -52,7 +52,7 @@ public class TestDoubleRangeField extends LuceneTestCase {
     Document doc = new Document();
     IllegalArgumentException expected;
     expected = expectThrows(IllegalArgumentException.class, () ->
-        doc.add(new DoubleRangeField(FIELD_NAME, new double[] {1, 2, 3, 4, 5}, new double[] {5})));
+        doc.add(new DoubleRange(FIELD_NAME, new double[] {1, 2, 3, 4, 5}, new double[] {5})));
     assertTrue(expected.getMessage().contains("does not support greater than 4 dimensions"));
   }
 
@@ -61,7 +61,7 @@ public class TestDoubleRangeField extends LuceneTestCase {
     Document doc = new Document();
     IllegalArgumentException expected;
     expected = expectThrows(IllegalArgumentException.class, () ->
-      doc.add(new DoubleRangeField(FIELD_NAME, new double[] {3, 4}, new double[] {1, 2})));
+      doc.add(new DoubleRange(FIELD_NAME, new double[] {3, 4}, new double[] {1, 2})));
     assertTrue(expected.getMessage().contains("is greater than max value"));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d34d81f9/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java b/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
deleted file mode 100644
index 0e0901b..0000000
--- a/lucene/sandbox/src/test/org/apache/lucene/document/TestInetAddressPoint.java
+++ /dev/null
@@ -1,176 +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.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/d34d81f9/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
deleted file mode 100644
index 53f3b82..0000000
--- a/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
+++ /dev/null
@@ -1,344 +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.HashSet;
-import java.util.Set;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.MultiDocValues;
-import org.apache.lucene.index.MultiFields;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.SerialMergeScheduler;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LuceneTestCase;
-
-/**
- * Abstract class to do basic tests for a RangeField query. Testing rigor inspired by {@code BaseGeoPointTestCase}
- */
-public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
-  protected abstract Field newRangeField(Range box);
-
-  protected abstract Query newIntersectsQuery(Range box);
-
-  protected abstract Query newContainsQuery(Range box);
-
-  protected abstract Query newWithinQuery(Range box);
-
-  protected abstract Query newCrossesQuery(Range box);
-
-  protected abstract Range nextRange(int dimensions);
-
-  protected int dimension() {
-    return random().nextInt(4) + 1;
-  }
-
-  public void testRandomTiny() throws Exception {
-    // Make sure single-leaf-node case is OK:
-    doTestRandom(10, false);
-  }
-
-  public void testRandomMedium() throws Exception {
-    doTestRandom(10000, false);
-  }
-
-  @Nightly
-  public void testRandomBig() throws Exception {
-    doTestRandom(200000, false);
-  }
-
-  public void testMultiValued() throws Exception {
-    doTestRandom(10000, true);
-  }
-
-  private void doTestRandom(int count, boolean multiValued) throws Exception {
-    int numDocs = atLeast(count);
-    int dimensions = dimension();
-
-    if (VERBOSE) {
-      System.out.println("TEST: numDocs=" + numDocs);
-    }
-
-    Range[][] ranges = new Range[numDocs][];
-
-    boolean haveRealDoc = true;
-
-    nextdoc: for (int id=0; id<numDocs; ++id) {
-      int x = random().nextInt(20);
-      if (ranges[id] == null) {
-        ranges[id] = new Range[] {nextRange(dimensions)};
-      }
-      if (x == 17) {
-        // some docs don't have a box:
-        ranges[id][0].isMissing = true;
-        if (VERBOSE) {
-          System.out.println("  id=" + id + " is missing");
-        }
-        continue;
-      }
-
-      if (multiValued == true && random().nextBoolean()) {
-        // randomly add multi valued documents (up to 2 fields)
-        int n = random().nextInt(2) + 1;
-        ranges[id] = new Range[n];
-        for (int i=0; i<n; ++i) {
-          ranges[id][i] = nextRange(dimensions);
-        }
-      }
-
-      if (id > 0 && x < 9 && haveRealDoc) {
-        int oldID;
-        int i=0;
-        // don't step on missing ranges:
-        while (true) {
-          oldID = random().nextInt(id);
-          if (ranges[oldID][0].isMissing == false) {
-            break;
-          } else if (++i > id) {
-            continue nextdoc;
-          }
-        }
-
-        if (x == dimensions*2) {
-          // Fully identical box (use first box in case current is multivalued but old is not)
-          for (int d=0; d<dimensions; ++d) {
-            ranges[id][0].setMin(d, ranges[oldID][0].getMin(d));
-            ranges[id][0].setMax(d, ranges[oldID][0].getMax(d));
-          }
-          if (VERBOSE) {
-            System.out.println("  id=" + id + " box=" + ranges[id] + " (same box as doc=" + oldID + ")");
-          }
-        } else {
-          for (int m = 0, even = dimensions % 2; m < dimensions * 2; ++m) {
-            if (x == m) {
-              int d = (int)Math.floor(m/2);
-              // current could be multivalue but old may not be, so use first box
-              if (even == 0) {
-                ranges[id][0].setMin(d, ranges[oldID][0].getMin(d));
-                if (VERBOSE) {
-                  System.out.println("  id=" + id + " box=" + ranges[id] + " (same min[" + d + "] as doc=" + oldID + ")");
-                }
-              } else {
-                ranges[id][0].setMax(d, ranges[oldID][0].getMax(d));
-                if (VERBOSE) {
-                  System.out.println("  id=" + id + " box=" + ranges[id] + " (same max[" + d + "] as doc=" + oldID + ")");
-                }
-              }
-            }
-          }
-        }
-      }
-    }
-    verify(ranges);
-  }
-
-  private void verify(Range[][] ranges) throws Exception {
-    IndexWriterConfig iwc = newIndexWriterConfig();
-    // Else seeds may not reproduce:
-    iwc.setMergeScheduler(new SerialMergeScheduler());
-    // Else we can get O(N^2) merging
-    int mbd = iwc.getMaxBufferedDocs();
-    if (mbd != -1 && mbd < ranges.length/100) {
-      iwc.setMaxBufferedDocs(ranges.length/100);
-    }
-    Directory dir;
-    if (ranges.length > 50000) {
-      dir = newFSDirectory(createTempDir(getClass().getSimpleName()));
-    } else {
-      dir = newDirectory();
-    }
-
-    Set<Integer> deleted = new HashSet<>();
-    IndexWriter w = new IndexWriter(dir, iwc);
-    for (int id=0; id < ranges.length; ++id) {
-      Document doc = new Document();
-      doc.add(newStringField("id", ""+id, Field.Store.NO));
-      doc.add(new NumericDocValuesField("id", id));
-      if (ranges[id][0].isMissing == false) {
-        for (int n=0; n<ranges[id].length; ++n) {
-          doc.add(newRangeField(ranges[id][n]));
-        }
-      }
-      w.addDocument(doc);
-      if (id > 0 && random().nextInt(100) == 1) {
-        int idToDelete = random().nextInt(id);
-        w.deleteDocuments(new Term("id", ""+idToDelete));
-        deleted.add(idToDelete);
-        if (VERBOSE) {
-          System.out.println("  delete id=" + idToDelete);
-        }
-      }
-    }
-
-    if (random().nextBoolean()) {
-      w.forceMerge(1);
-    }
-    final IndexReader r = DirectoryReader.open(w);
-    w.close();
-    IndexSearcher s = newSearcher(r);
-
-    int dimensions = ranges[0][0].numDimensions();
-    int iters = atLeast(25);
-    Bits liveDocs = MultiFields.getLiveDocs(s.getIndexReader());
-    int maxDoc = s.getIndexReader().maxDoc();
-
-    for (int iter=0; iter<iters; ++iter) {
-      if (VERBOSE) {
-        System.out.println("\nTEST: iter=" + iter + " s=" + s);
-      }
-
-      // occasionally test open ended bounding ranges
-      Range queryRange = nextRange(dimensions);
-      int rv = random().nextInt(4);
-      Query query;
-      Range.QueryType queryType;
-      if (rv == 0) {
-        queryType = Range.QueryType.INTERSECTS;
-        query = newIntersectsQuery(queryRange);
-      } else if (rv == 1)  {
-        queryType = Range.QueryType.CONTAINS;
-        query = newContainsQuery(queryRange);
-      } else if (rv == 2) {
-        queryType = Range.QueryType.WITHIN;
-        query = newWithinQuery(queryRange);
-      } else {
-        queryType = Range.QueryType.CROSSES;
-        query = newCrossesQuery(queryRange);
-      }
-
-      if (VERBOSE) {
-        System.out.println("  query=" + query);
-      }
-
-      final FixedBitSet hits = new FixedBitSet(maxDoc);
-      s.search(query, new SimpleCollector() {
-        private int docBase;
-
-        @Override
-        public void collect(int doc) {
-          hits.set(docBase + doc);
-        }
-
-        @Override
-        protected void doSetNextReader(LeafReaderContext context) throws IOException {
-          docBase = context.docBase;
-        }
-
-        @Override
-        public boolean needsScores() { return false; }
-      });
-
-      NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
-      for (int docID=0; docID<maxDoc; ++docID) {
-        assertEquals(docID, docIDToID.nextDoc());
-        int id = (int) docIDToID.longValue();
-        boolean expected;
-        if (liveDocs != null && liveDocs.get(docID) == false) {
-          // document is deleted
-          expected = false;
-        } else if (ranges[id][0].isMissing) {
-          expected = false;
-        } else {
-          expected = expectedResult(queryRange, ranges[id], queryType);
-        }
-
-        if (hits.get(docID) != expected) {
-          StringBuilder b = new StringBuilder();
-          b.append("FAIL (iter " + iter + "): ");
-          if (expected == true) {
-            b.append("id=" + id + (ranges[id].length > 1 ? " (MultiValue) " : " ") + "should match but did not\n");
-          } else {
-            b.append("id=" + id + " should not match but did\n");
-          }
-          b.append(" queryRange=" + queryRange + "\n");
-          b.append(" box" + ((ranges[id].length > 1) ? "es=" : "=" ) + ranges[id][0]);
-          for (int n=1; n<ranges[id].length; ++n) {
-            b.append(", ");
-            b.append(ranges[id][n]);
-          }
-          b.append("\n queryType=" + queryType + "\n");
-          b.append(" deleted?=" + (liveDocs != null && liveDocs.get(docID) == false));
-          fail("wrong hit (first of possibly more):\n\n" + b);
-        }
-      }
-    }
-    IOUtils.close(r, dir);
-  }
-
-  protected boolean expectedResult(Range queryRange, Range[] range, Range.QueryType queryType) {
-    for (int i=0; i<range.length; ++i) {
-      if (expectedBBoxQueryResult(queryRange, range[i], queryType) == true) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  protected boolean expectedBBoxQueryResult(Range queryRange, Range range, Range.QueryType queryType) {
-    if (queryRange.isEqual(range) && queryType != Range.QueryType.CROSSES) {
-      return true;
-    }
-    Range.QueryType relation = range.relate(queryRange);
-    if (queryType == Range.QueryType.INTERSECTS) {
-      return relation != null;
-    } else if (queryType == Range.QueryType.CROSSES) {
-      // by definition, RangeFields that CONTAIN the query are also considered to cross
-      return relation == queryType || relation == Range.QueryType.CONTAINS;
-    }
-    return relation == queryType;
-  }
-
-  abstract static class Range {
-    protected boolean isMissing = false;
-
-    enum QueryType { INTERSECTS, WITHIN, CONTAINS, CROSSES }
-
-    protected abstract int numDimensions();
-    protected abstract Object getMin(int dim);
-    protected abstract void setMin(int dim, Object val);
-    protected abstract Object getMax(int dim);
-    protected abstract void setMax(int dim, Object val);
-    protected abstract boolean isEqual(Range other);
-    protected abstract boolean isDisjoint(Range other);
-    protected abstract boolean isWithin(Range other);
-    protected abstract boolean contains(Range other);
-
-    protected QueryType relate(Range other) {
-      if (isDisjoint(other)) {
-        // if disjoint; return null:
-        return null;
-      } else if (isWithin(other)) {
-        return QueryType.WITHIN;
-      } else if (contains(other)) {
-        return QueryType.CONTAINS;
-      }
-      return QueryType.CROSSES;
-    }
-  }
-}