You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2019/06/10 14:00:59 UTC

[lucene-solr] 03/03: LUCENE-8362: Introduce DocValues Fields and Range Queries for native Range Field Types

This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git

commit d2ff7ffafcd0ca65d93e481138785567c5861bac
Author: Atri Sharma <at...@apache.org>
AuthorDate: Mon May 27 17:57:25 2019 +0530

    LUCENE-8362: Introduce DocValues Fields and Range Queries for native Range Field Types
    
    This commit introduces a new DocValues field and corresponding
    range query for binary ranges. These classes are extended into
    concrete implementations for each of Int, Long, Float and Double
    range fields.
---
 lucene/CHANGES.txt                                 |   2 +
 .../lucene/document/BinaryRangeDocValues.java      |  96 +++++++++
 .../lucene/document/BinaryRangeDocValuesField.java |  36 ++++
 .../document/BinaryRangeFieldRangeQuery.java       | 131 ++++++++++++
 .../org/apache/lucene/document/DoubleRange.java    |   2 +-
 .../lucene/document/DoubleRangeDocValuesField.java |  88 ++++++++
 .../lucene/document/DoubleRangeSlowRangeQuery.java |  94 +++++++++
 .../org/apache/lucene/document/FloatRange.java     |   2 +-
 .../lucene/document/FloatRangeDocValuesField.java  |  88 ++++++++
 .../lucene/document/FloatRangeSlowRangeQuery.java  |  93 +++++++++
 .../java/org/apache/lucene/document/IntRange.java  |   2 +-
 .../lucene/document/IntRangeDocValuesField.java    |  88 ++++++++
 .../lucene/document/IntRangeSlowRangeQuery.java    |  92 +++++++++
 .../java/org/apache/lucene/document/LongRange.java |   2 +-
 .../lucene/document/LongRangeDocValuesField.java   |  88 ++++++++
 .../lucene/document/LongRangeSlowRangeQuery.java   |  93 +++++++++
 .../apache/lucene/search/TestDocValuesQueries.java |   1 -
 .../search/TestRangeFieldsDocValuesQuery.java      | 229 +++++++++++++++++++++
 18 files changed, 1222 insertions(+), 5 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 5a2da5b..99a4db2 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -24,6 +24,8 @@ New Features
   and process decimal point. It is similar to the JapaneseNumberFilter.
   (Namgyu Kim)
 
+* LUCENE-8362: Add doc-value support to range fields. (Atri Sharma via Adrien Grand)
+
 Bug Fixes
 
 * LUCENE-8831: Fixed LatLonShapeBoundingBoxQuery .hashCode methods. (Ignacio Vera)
diff --git a/lucene/core/src/java/org/apache/lucene/document/BinaryRangeDocValues.java b/lucene/core/src/java/org/apache/lucene/document/BinaryRangeDocValues.java
new file mode 100644
index 0000000..d5e511f
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/BinaryRangeDocValues.java
@@ -0,0 +1,96 @@
+/*
+ * 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 org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.util.BytesRef;
+
+class BinaryRangeDocValues extends BinaryDocValues {
+  private final BinaryDocValues in;
+  private byte[] packedValue;
+  private final int numDims;
+  private final int numBytesPerDimension;
+  private int docID = -1;
+
+  BinaryRangeDocValues(BinaryDocValues in, int numDims, int numBytesPerDimension) {
+    this.in = in;
+    this.numBytesPerDimension = numBytesPerDimension;
+    this.numDims = numDims;
+    this.packedValue = new byte[2 * numDims * numBytesPerDimension];
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    docID = in.nextDoc();
+
+    if (docID != NO_MORE_DOCS) {
+      decodeRanges();
+    }
+
+    return docID;
+  }
+
+  @Override
+  public int docID() {
+    return in.docID();
+  }
+
+  @Override
+  public long cost() {
+    return in.cost();
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    int res = in.advance(target);
+    if (res != NO_MORE_DOCS) {
+      decodeRanges();
+    }
+
+    return res;
+  }
+
+  @Override
+  public boolean advanceExact(int target) throws IOException {
+    boolean res = in.advanceExact(target);
+    if (res) {
+      decodeRanges();
+    }
+
+    return res;
+  }
+
+  @Override
+  public BytesRef binaryValue() throws IOException {
+    return in.binaryValue();
+  }
+
+  public byte[] getPackedValue() {
+    return packedValue;
+  }
+
+  private void decodeRanges() throws IOException {
+    BytesRef bytesRef = in.binaryValue();
+
+    // We reuse the existing allocated memory for packed values since all docvalues in this iterator
+    // should be exactly same in indexed structure, hence the byte representations in length should be identical
+    System.arraycopy(bytesRef.bytes, bytesRef.offset, packedValue, 0, 2 * numDims * numBytesPerDimension);
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/BinaryRangeDocValuesField.java b/lucene/core/src/java/org/apache/lucene/document/BinaryRangeDocValuesField.java
new file mode 100644
index 0000000..1d0504a
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/BinaryRangeDocValuesField.java
@@ -0,0 +1,36 @@
+/*
+ * 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.util.BytesRef;
+
+abstract class BinaryRangeDocValuesField extends BinaryDocValuesField {
+  
+  final String field;
+  final byte[] packedValue;
+  final int numDims;
+  final int numBytesPerDimension;
+
+  BinaryRangeDocValuesField(String field, byte[] packedValue, int numDims, int numBytesPerDimension) {
+    super(field, new BytesRef(packedValue));
+    this.field = field;
+    this.packedValue = packedValue;
+    this.numDims = numDims;
+    this.numBytesPerDimension = numBytesPerDimension;
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/BinaryRangeFieldRangeQuery.java b/lucene/core/src/java/org/apache/lucene/document/BinaryRangeFieldRangeQuery.java
new file mode 100644
index 0000000..9134bb0
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/BinaryRangeFieldRangeQuery.java
@@ -0,0 +1,131 @@
+/*
+ * 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 org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TwoPhaseIterator;
+import org.apache.lucene.search.Weight;
+
+abstract class BinaryRangeFieldRangeQuery extends Query {
+  private final String field;
+  private byte[] queryPackedValue;
+  private final int numBytesPerDimension;
+  private final int numDims;
+  private final RangeFieldQuery.QueryType queryType;
+
+  BinaryRangeFieldRangeQuery(String field, byte[] queryPackedValue, int numBytesPerDimension, int numDims,
+                             RangeFieldQuery.QueryType queryType) {
+    this.field = field;
+    this.queryPackedValue = queryPackedValue;
+    this.numBytesPerDimension = numBytesPerDimension;
+    this.numDims = numDims;
+
+    if (!(queryType == RangeFieldQuery.QueryType.INTERSECTS)) {
+      throw new UnsupportedOperationException("INTERSECTS is the only query type supported for this field type right now");
+    }
+
+    this.queryType = queryType;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (sameClassAs(obj) == false) {
+      return false;
+    }
+    BinaryRangeFieldRangeQuery that = (BinaryRangeFieldRangeQuery) obj;
+    return Objects.equals(field, that.field)
+        && Arrays.equals(queryPackedValue, that.queryPackedValue);
+  }
+
+  @Override
+  public int hashCode() {
+    int h = classHash();
+    h = 31 * h + field.hashCode();
+    h = 31 * h + Arrays.hashCode(queryPackedValue);
+    return h;
+  }
+
+  @Override
+  public void visit(QueryVisitor visitor) {
+    if (visitor.acceptField(field)) {
+      visitor.visitLeaf(this);
+    }
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    return super.rewrite(reader);
+  }
+
+  private BinaryRangeDocValues getValues(LeafReader reader, String field) throws IOException {
+    BinaryDocValues binaryDocValues = reader.getBinaryDocValues(field);
+
+    return new BinaryRangeDocValues(binaryDocValues, numDims, numBytesPerDimension);
+  }
+
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+    return new ConstantScoreWeight(this, boost) {
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        BinaryRangeDocValues values = getValues(context.reader(), field);
+        if (values == null) {
+          return null;
+        }
+
+        final TwoPhaseIterator iterator;
+        iterator = new TwoPhaseIterator(values) {
+          @Override
+          public boolean matches() {
+            return queryType.matches(queryPackedValue, values.getPackedValue(), numDims, numBytesPerDimension);
+          }
+
+          @Override
+          public float matchCost() {
+            return queryPackedValue.length;
+          }
+        };
+
+        return new ConstantScoreScorer(this, score(), scoreMode, iterator);
+      }
+
+      @Override
+      public boolean isCacheable(LeafReaderContext ctx) {
+        return DocValues.isCacheable(ctx, field);
+      }
+
+    };
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java b/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java
index 726d0a1..95f87e3 100644
--- a/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DoubleRange.java
@@ -108,7 +108,7 @@ public class DoubleRange extends Field {
   /**
    * Encodes the min, max ranges into a byte array
    */
-  private static byte[] encode(double[] min, double[] max) {
+  static byte[] encode(double[] min, double[] max) {
     checkArgs(min, max);
     byte[] b = new byte[BYTES*2*min.length];
     verifyAndEncode(min, max, b);
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoubleRangeDocValuesField.java b/lucene/core/src/java/org/apache/lucene/document/DoubleRangeDocValuesField.java
new file mode 100644
index 0000000..629e33c
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/DoubleRangeDocValuesField.java
@@ -0,0 +1,88 @@
+/*
+ * 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.search.Query;
+
+/**
+ * DocValues field for DoubleRange. This is a single valued field i.e. can have only one
+ * value per document due to being an extension of BinaryDocValuesField.
+ */
+public class DoubleRangeDocValuesField extends BinaryRangeDocValuesField{
+  final String field;
+  final double[] min;
+  final double[] max;
+
+  /** Sole constructor. */
+  public DoubleRangeDocValuesField(String field, final double[] min, final double[] max) {
+    super(field, DoubleRange.encode(min, max), min.length, DoubleRange.BYTES);
+    checkArgs(min, max);
+    this.field = field;
+    this.min = min;
+    this.max = max;
+  }
+
+  /** Get the minimum value for the given dimension. */
+  public double getMin(int dimension) {
+    if (dimension > 4 || dimension > min.length) {
+      throw new IllegalArgumentException("Dimension out of valid range");
+    }
+
+    return min[dimension];
+  }
+
+  /** Get the maximum value for the given dimension. */
+  public double getMax(int dimension) {
+    if (dimension > 4 || dimension > min.length) {
+      throw new IllegalArgumentException("Dimension out of valid range");
+    }
+
+    return max[dimension];
+  }
+
+  private static Query newSlowRangeQuery(String field, final double[] min, final double[] max,
+                                        RangeFieldQuery.QueryType queryType) {
+    checkArgs(min, max);
+    return new DoubleRangeSlowRangeQuery(field, min, max, queryType);
+  }
+
+  /** Create a new range query that finds all ranges that intersect using doc values.
+   *  NOTE: This doesn't leverage indexing and may be slow.
+   *  @see DoubleRange#newIntersectsQuery */
+  public static Query newSlowIntersectsQuery(String field, final double[] min, final double[] max) {
+    return newSlowRangeQuery(field, min, max, RangeFieldQuery.QueryType.INTERSECTS);
+  }
+
+  /**
+   * validate the arguments
+   */
+  private static void checkArgs(final double[] min, final double[] 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");
+    }
+
+    for (int i = 0; i < min.length; i++) {
+      if (min[i] > max[i]) {
+        throw new IllegalArgumentException("min should be less than max");
+      }
+    }
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoubleRangeSlowRangeQuery.java b/lucene/core/src/java/org/apache/lucene/document/DoubleRangeSlowRangeQuery.java
new file mode 100644
index 0000000..5f506e2
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/DoubleRangeSlowRangeQuery.java
@@ -0,0 +1,94 @@
+/*
+ * 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 org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+
+class DoubleRangeSlowRangeQuery extends BinaryRangeFieldRangeQuery {
+  private final String field;
+  private final double[] min;
+  private final double[] max;
+
+  DoubleRangeSlowRangeQuery(String field, double[] min, double[] max, RangeFieldQuery.QueryType queryType) {
+    super(field, encodeRanges(min, max), DoubleRange.BYTES, min.length,
+        queryType);
+    this.field = field;
+    this.min = min;
+    this.max = max;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (sameClassAs(obj) == false) {
+      return false;
+    }
+    DoubleRangeSlowRangeQuery that = (DoubleRangeSlowRangeQuery) obj;
+    return Objects.equals(field, that.field)
+        && Arrays.equals(min, that.min)
+        && Arrays.equals(max, that.max);
+  }
+
+  @Override
+  public int hashCode() {
+    int h = classHash();
+    h = 31 * h + field.hashCode();
+    h = 31 * h + Arrays.hashCode(min);
+    h = 31 * h + Arrays.hashCode(max);
+    return h;
+  }
+
+  @Override
+  public void visit(QueryVisitor visitor) {
+    if (visitor.acceptField(field)) {
+      visitor.visitLeaf(this);
+    }
+  }
+
+  @Override
+  public String toString(String field) {
+    StringBuilder b = new StringBuilder();
+    if (this.field.equals(field) == false) {
+      b.append(this.field).append(":");
+    }
+    return b
+        .append("[")
+        .append(Arrays.toString(min))
+        .append(" TO ")
+        .append(Arrays.toString(max))
+        .append("]")
+        .toString();
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    return super.rewrite(reader);
+  }
+
+  private static byte[] encodeRanges(double[] min, double[] max) {
+    byte[] result = new byte[2 * DoubleRange.BYTES * min.length];
+
+    DoubleRange.verifyAndEncode(min, max, result);
+    return result;
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatRange.java b/lucene/core/src/java/org/apache/lucene/document/FloatRange.java
index 191631b..2507746 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FloatRange.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatRange.java
@@ -108,7 +108,7 @@ public class FloatRange extends Field {
   /**
    * Encodes the min, max ranges into a byte array
    */
-  private static byte[] encode(float[] min, float[] max) {
+  static byte[] encode(float[] min, float[] max) {
     checkArgs(min, max);
     byte[] b = new byte[BYTES*2*min.length];
     verifyAndEncode(min, max, b);
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatRangeDocValuesField.java b/lucene/core/src/java/org/apache/lucene/document/FloatRangeDocValuesField.java
new file mode 100644
index 0000000..c5fefc4
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatRangeDocValuesField.java
@@ -0,0 +1,88 @@
+/*
+ * 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.search.Query;
+
+/**
+ * DocValues field for FloatRange. This is a single valued field per document
+ * due to being an extension of BinaryDocValuesField.
+ */
+public class FloatRangeDocValuesField extends BinaryRangeDocValuesField{
+  final String field;
+  final float[] min;
+  final float[] max;
+
+  /** Sole constructor. */
+  public FloatRangeDocValuesField(String field, final float[] min, final float[] max) {
+    super(field, FloatRange.encode(min, max), min.length, FloatRange.BYTES);
+    checkArgs(min, max);
+    this.field = field;
+    this.min = min;
+    this.max = max;
+  }
+
+  /** Get the minimum value for the given dimension. */
+  public float getMin(int dimension) {
+    if (dimension > 4 || dimension > min.length) {
+      throw new IllegalArgumentException("Dimension out of valid range");
+    }
+
+    return min[dimension];
+  }
+
+  /** Get the maximum value for the given dimension. */
+  public float getMax(int dimension) {
+    if (dimension > 4 || dimension > min.length) {
+      throw new IllegalArgumentException("Dimension out of valid range");
+    }
+
+    return max[dimension];
+  }
+
+  private static Query newSlowRangeQuery(String field, final float[] min, final float[] max,
+                                        RangeFieldQuery.QueryType queryType) {
+    checkArgs(min, max);
+    return new FloatRangeSlowRangeQuery(field, min, max, queryType);
+  }
+
+  /** Create a new range query that finds all ranges that intersect using doc values.
+   *  NOTE: This doesn't leverage indexing and may be slow.
+   *  @see FloatRange#newIntersectsQuery */
+  public static Query newSlowIntersectsQuery(String field, final float[] min, final float[] max) {
+    return newSlowRangeQuery(field, min, max, RangeFieldQuery.QueryType.INTERSECTS);
+  }
+
+  /**
+   * validate the arguments
+   */
+  private static void checkArgs(final float[] min, final float[] 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");
+    }
+
+    for (int i = 0; i < min.length; i++) {
+      if (min[i] > max[i]) {
+        throw new IllegalArgumentException("min should be less than max");
+      }
+    }
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatRangeSlowRangeQuery.java b/lucene/core/src/java/org/apache/lucene/document/FloatRangeSlowRangeQuery.java
new file mode 100644
index 0000000..dd208aa
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatRangeSlowRangeQuery.java
@@ -0,0 +1,93 @@
+/*
+ * 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 org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+
+class FloatRangeSlowRangeQuery extends BinaryRangeFieldRangeQuery {
+  private final String field;
+  private final float[] min;
+  private final float[] max;
+
+  FloatRangeSlowRangeQuery(String field, float[] min, float[] max, RangeFieldQuery.QueryType queryType) {
+    super(field, encodeRanges(min, max), FloatRange.BYTES, min.length, queryType);
+    this.field = field;
+    this.min = min;
+    this.max = max;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (sameClassAs(obj) == false) {
+      return false;
+    }
+    FloatRangeSlowRangeQuery that = (FloatRangeSlowRangeQuery) obj;
+    return Objects.equals(field, that.field)
+        && Arrays.equals(min, that.min)
+        && Arrays.equals(max, that.max);
+  }
+
+  @Override
+  public int hashCode() {
+    int h = classHash();
+    h = 31 * h + field.hashCode();
+    h = 31 * h + Arrays.hashCode(min);
+    h = 31 * h + Arrays.hashCode(max);
+    return h;
+  }
+
+  @Override
+  public void visit(QueryVisitor visitor) {
+    if (visitor.acceptField(field)) {
+      visitor.visitLeaf(this);
+    }
+  }
+
+  @Override
+  public String toString(String field) {
+    StringBuilder b = new StringBuilder();
+    if (this.field.equals(field) == false) {
+      b.append(this.field).append(":");
+    }
+    return b
+        .append("[")
+        .append(Arrays.toString(min))
+        .append(" TO ")
+        .append(Arrays.toString(max))
+        .append("]")
+        .toString();
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    return super.rewrite(reader);
+  }
+
+  private static byte[] encodeRanges(float[] min, float[] max) {
+    byte[] result = new byte[2 * FloatRange.BYTES * min.length];
+
+    FloatRange.verifyAndEncode(min, max, result);
+    return result;
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntRange.java b/lucene/core/src/java/org/apache/lucene/document/IntRange.java
index b24e80a..881ac13 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntRange.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntRange.java
@@ -108,7 +108,7 @@ public class IntRange extends Field {
   /**
    * Encodes the min, max ranges into a byte array
    */
-  private static byte[] encode(int[] min, int[] max) {
+  static byte[] encode(int[] min, int[] max) {
     checkArgs(min, max);
     byte[] b = new byte[BYTES*2*min.length];
     verifyAndEncode(min, max, b);
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntRangeDocValuesField.java b/lucene/core/src/java/org/apache/lucene/document/IntRangeDocValuesField.java
new file mode 100644
index 0000000..f23d152
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/IntRangeDocValuesField.java
@@ -0,0 +1,88 @@
+/*
+ * 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.search.Query;
+
+/**
+ * DocValues field for IntRange. This is a single valued field per document
+ * due to being an extension of BinaryDocValuesField.
+ */
+public class IntRangeDocValuesField extends BinaryRangeDocValuesField{
+  final String field;
+  final int[] min;
+  final int[] max;
+
+  /** Sole constructor. */
+  public IntRangeDocValuesField(String field, final int[] min, final int[] max) {
+    super(field, IntRange.encode(min, max), min.length, IntRange.BYTES);
+    checkArgs(min, max);
+    this.field = field;
+    this.min = min;
+    this.max = max;
+  }
+
+  /** Get the minimum value for the given dimension. */
+  public int getMin(int dimension) {
+    if (dimension > 4 || dimension > min.length) {
+      throw new IllegalArgumentException("Dimension out of valid range");
+    }
+
+    return min[dimension];
+  }
+
+  /** Get the maximum value for the given dimension. */
+  public int getMax(int dimension) {
+    if (dimension > 4 || dimension > min.length) {
+      throw new IllegalArgumentException("Dimension out of valid range");
+    }
+
+    return max[dimension];
+  }
+
+  private static Query newSlowRangeQuery(String field, final int[] min, final int[] max,
+                                        RangeFieldQuery.QueryType queryType) {
+    checkArgs(min, max);
+    return new IntRangeSlowRangeQuery(field, min, max, queryType);
+  }
+
+  /** Create a new range query that finds all ranges that intersect using doc values.
+   *  NOTE: This doesn't leverage indexing and may be slow.
+   *  @see IntRange#newIntersectsQuery */
+  public static Query newSlowIntersectsQuery(String field, final int[] min, final int[] max) {
+    return newSlowRangeQuery(field, min, max, RangeFieldQuery.QueryType.INTERSECTS);
+  }
+
+  /**
+   * 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");
+    }
+
+    for (int i = 0; i < min.length; i++) {
+      if (min[i] > max[i]) {
+        throw new IllegalArgumentException("min should be less than max but min = " + min[i] + " and max = " + max[i]);
+      }
+    }
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntRangeSlowRangeQuery.java b/lucene/core/src/java/org/apache/lucene/document/IntRangeSlowRangeQuery.java
new file mode 100644
index 0000000..ed62108
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/IntRangeSlowRangeQuery.java
@@ -0,0 +1,92 @@
+/*
+ * 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 org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+
+class IntRangeSlowRangeQuery extends BinaryRangeFieldRangeQuery {
+  private final String field;
+  private final int[] min;
+  private final int[] max;
+
+  IntRangeSlowRangeQuery(String field, int[] min, int[] max, RangeFieldQuery.QueryType queryType) {
+    super(field, encodeRanges(min, max), IntRange.BYTES, min.length, queryType);
+    this.field = field;
+    this.min = min;
+    this.max = max;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (sameClassAs(obj) == false) {
+      return false;
+    }
+    IntRangeSlowRangeQuery that = (IntRangeSlowRangeQuery) obj;
+    return Objects.equals(field, that.field)
+        && Arrays.equals(min, that.min)
+        && Arrays.equals(max, that.max);
+  }
+
+  @Override
+  public int hashCode() {
+    int h = classHash();
+    h = 31 * h + field.hashCode();
+    h = 31 * h + Arrays.hashCode(min);
+    h = 31 * h + Arrays.hashCode(max);
+    return h;
+  }
+
+  @Override
+  public void visit(QueryVisitor visitor) {
+    if (visitor.acceptField(field)) {
+      visitor.visitLeaf(this);
+    }
+  }
+
+  @Override
+  public String toString(String field) {
+    StringBuilder b = new StringBuilder();
+    if (this.field.equals(field) == false) {
+      b.append(this.field).append(":");
+    }
+    return b
+        .append("[")
+        .append(Arrays.toString(min))
+        .append(" TO ")
+        .append(Arrays.toString(max))
+        .append("]")
+        .toString();
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    return super.rewrite(reader);
+  }
+
+  private static byte[] encodeRanges(int[] min, int[] max) {
+    byte[] result = new byte[2 * IntRange.BYTES * min.length];
+
+    IntRange.verifyAndEncode(min, max, result);
+    return result;
+  }
+}
\ No newline at end of file
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongRange.java b/lucene/core/src/java/org/apache/lucene/document/LongRange.java
index 9056d2d..d29c0f9 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongRange.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongRange.java
@@ -106,7 +106,7 @@ public class LongRange extends Field {
   }
 
   /** Encodes the min, max ranges into a byte array */
-  private static byte[] encode(long[] min, long[] max) {
+  static byte[] encode(long[] min, long[] max) {
     checkArgs(min, max);
     byte[] b = new byte[BYTES*2*min.length];
     verifyAndEncode(min, max, b);
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongRangeDocValuesField.java b/lucene/core/src/java/org/apache/lucene/document/LongRangeDocValuesField.java
new file mode 100644
index 0000000..458635e
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/LongRangeDocValuesField.java
@@ -0,0 +1,88 @@
+/*
+ * 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.search.Query;
+
+/**
+ * DocValues field for LongRange. This is a single valued field per document
+ * due to being an extension of BinaryDocValuesField.
+ */
+public class LongRangeDocValuesField extends BinaryRangeDocValuesField{
+  final String field;
+  final long[] min;
+  final long[] max;
+
+  /** Sole constructor. */
+  public LongRangeDocValuesField(String field, final long[] min, final long[] max) {
+    super(field, LongRange.encode(min, max), min.length, LongRange.BYTES);
+    checkArgs(min, max);
+    this.field = field;
+    this.min = min;
+    this.max = max;
+  }
+
+  /** Get the minimum value for the given dimension. */
+  public long getMin(int dimension) {
+    if (dimension > 4 || dimension > min.length) {
+      throw new IllegalArgumentException("Dimension out of valid range");
+    }
+
+    return min[dimension];
+  }
+
+  /** Get the maximum value for the given dimension. */
+  public long getMax(int dimension) {
+    if (dimension > 4 || dimension > min.length) {
+      throw new IllegalArgumentException("Dimension out of valid range");
+    }
+
+    return max[dimension];
+  }
+
+  private static Query newSlowRangeQuery(String field, final long[] min, final long[] max,
+                                        RangeFieldQuery.QueryType queryType) {
+    checkArgs(min, max);
+    return new LongRangeSlowRangeQuery(field, min, max, queryType);
+  }
+
+  /** Create a new range query that finds all ranges that intersect using doc values.
+   *  NOTE: This doesn't leverage indexing and may be slow.
+   *  @see LongRange#newIntersectsQuery */
+  public static Query newSlowIntersectsQuery(String field, final long[] min, final long[] max) {
+    return newSlowRangeQuery(field, min, max, RangeFieldQuery.QueryType.INTERSECTS);
+  }
+
+  /**
+   * 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");
+    }
+
+    for (int i = 0; i < min.length; i++) {
+      if (min[i] > max[i]) {
+        throw new IllegalArgumentException("min should be less than max");
+      }
+    }
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongRangeSlowRangeQuery.java b/lucene/core/src/java/org/apache/lucene/document/LongRangeSlowRangeQuery.java
new file mode 100644
index 0000000..8564381
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/LongRangeSlowRangeQuery.java
@@ -0,0 +1,93 @@
+/*
+ * 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 org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+
+class LongRangeSlowRangeQuery extends BinaryRangeFieldRangeQuery {
+  private final String field;
+  private final long[] min;
+  private final long[] max;
+
+  LongRangeSlowRangeQuery(String field, long[] min, long[] max, RangeFieldQuery.QueryType queryType) {
+    super(field, encodeRanges(min, max), LongRange.BYTES, min.length, queryType);
+    this.field = field;
+    this.min = min;
+    this.max = max;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (sameClassAs(obj) == false) {
+      return false;
+    }
+    LongRangeSlowRangeQuery that = (LongRangeSlowRangeQuery) obj;
+    return Objects.equals(field, that.field)
+        && Arrays.equals(min, that.min)
+        && Arrays.equals(max, that.max);
+  }
+
+  @Override
+  public int hashCode() {
+    int h = classHash();
+    h = 31 * h + field.hashCode();
+    h = 31 * h + Arrays.hashCode(min);
+    h = 31 * h + Arrays.hashCode(max);
+    return h;
+  }
+
+  @Override
+  public void visit(QueryVisitor visitor) {
+    if (visitor.acceptField(field)) {
+      visitor.visitLeaf(this);
+    }
+  }
+
+  @Override
+  public String toString(String field) {
+    StringBuilder b = new StringBuilder();
+    if (this.field.equals(field) == false) {
+      b.append(this.field).append(":");
+    }
+    return b
+        .append("[")
+        .append(Arrays.toString(min))
+        .append(" TO ")
+        .append(Arrays.toString(max))
+        .append("]")
+        .toString();
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    return super.rewrite(reader);
+  }
+
+  private static byte[] encodeRanges(long[] min, long[] max) {
+    byte[] result = new byte[2 * LongRange.BYTES * min.length];
+
+    LongRange.verifyAndEncode(min, max, result);
+    return result;
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestDocValuesQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestDocValuesQueries.java
index 0664709..0f3a52f 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestDocValuesQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestDocValuesQueries.java
@@ -267,5 +267,4 @@ public class TestDocValuesQueries extends LuceneTestCase {
     reader.close();
     dir.close();
   }
-   
 }
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestRangeFieldsDocValuesQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestRangeFieldsDocValuesQuery.java
new file mode 100644
index 0000000..4e15086
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestRangeFieldsDocValuesQuery.java
@@ -0,0 +1,229 @@
+/*
+ * 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 org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleRangeDocValuesField;
+import org.apache.lucene.document.FloatRangeDocValuesField;
+import org.apache.lucene.document.IntRangeDocValuesField;
+import org.apache.lucene.document.LongRangeDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestRangeFieldsDocValuesQuery extends LuceneTestCase {
+  public void testDoubleRangeDocValuesIntersectsQuery() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int iters = atLeast(10);
+    double[] min = {112.7, 296.0, 512.4};
+    double[] max = {119.3, 314.8, 524.3};
+    for (int i = 0; i < iters; ++i) {
+      Document doc = new Document();
+      doc.add(new DoubleRangeDocValuesField("dv", min, max));
+      iw.addDocument(doc);
+    }
+    iw.commit();
+
+    double[] nonMatchingMin = {256.7, 296.0, 532.4};
+    double[] nonMatchingMax = {259.3, 364.8, 534.3};
+
+    Document doc = new Document();
+    doc.add(new DoubleRangeDocValuesField("dv", nonMatchingMin, nonMatchingMax));
+    iw.addDocument(doc);
+    iw.commit();
+
+    IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+
+    final double[] lowRange = {111.3, 294.4, 517.4};
+    final double[] highRange = {116.7, 319.4, 533.0};
+
+    Query query = DoubleRangeDocValuesField.newSlowIntersectsQuery("dv", lowRange, highRange);
+    assertEquals(searcher.count(query), iters);
+
+    double[] lowRange2 = {116.3, 299.3, 517.0};
+    double[] highRange2 = {121.0, 317.1, 531.2};
+
+    query = DoubleRangeDocValuesField.newSlowIntersectsQuery( "dv", lowRange2, highRange2);
+
+    assertEquals(searcher.count(query), iters);
+
+    reader.close();
+    dir.close();
+  }
+
+  public void testIntRangeDocValuesIntersectsQuery() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int iters = atLeast(10);
+    int[] min = {3, 11, 17};
+    int[] max = {27, 35, 49};
+    for (int i = 0; i < iters; ++i) {
+      Document doc = new Document();
+      doc.add(new IntRangeDocValuesField("dv", min, max));
+      iw.addDocument(doc);
+    }
+
+    int[] min2 = {11, 19, 27};
+    int[] max2 = {29, 38, 56};
+
+    Document doc = new Document();
+    doc.add(new IntRangeDocValuesField("dv", min2, max2));
+
+    iw.commit();
+
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+
+    final int[] lowRange = {6, 16, 19};
+    final int[] highRange = {29, 41, 42};
+
+    Query query = IntRangeDocValuesField.newSlowIntersectsQuery("dv", lowRange, highRange);
+
+    assertEquals(searcher.count(query), iters);
+
+    int[] lowRange2 = {2, 9, 18};
+    int[] highRange2 = {25, 34, 41};
+
+    query = IntRangeDocValuesField.newSlowIntersectsQuery("dv", lowRange2, highRange2);
+
+    assertEquals(searcher.count(query), iters);
+
+    int[] lowRange3 = {101, 121, 153};
+    int[] highRange3 = {156, 127, 176};
+
+    query = IntRangeDocValuesField.newSlowIntersectsQuery("dv", lowRange3, highRange3);
+
+    assertEquals(searcher.count(query), 0);
+
+    reader.close();
+    dir.close();
+  }
+
+  public void testLongRangeDocValuesIntersectQuery() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int iters = atLeast(10);
+    long[] min = {31, 15, 2};
+    long[] max = {95, 27, 4};
+    for (int i = 0; i < iters; ++i) {
+      Document doc = new Document();
+      doc.add(new LongRangeDocValuesField("dv", min, max));
+      iw.addDocument(doc);
+    }
+
+    long[] min2 = {101, 124, 137};
+    long[] max2 = {138, 145, 156};
+    Document doc = new Document();
+    doc.add(new LongRangeDocValuesField("dv", min2, max2));
+
+    iw.commit();
+
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+
+    final long[] lowRange = {6, 12, 1};
+    final long[] highRange = {34, 24, 3};
+
+    Query query = LongRangeDocValuesField.newSlowIntersectsQuery("dv", lowRange, highRange);
+
+    assertEquals(searcher.count(query), iters);
+
+    final long[] lowRange2 = {32, 18, 3};
+    final long[] highRange2 = {96, 29, 5};
+
+    query = LongRangeDocValuesField.newSlowIntersectsQuery("dv", lowRange2, highRange2);
+
+    assertEquals(searcher.count(query), iters);
+
+    reader.close();
+    dir.close();
+  }
+
+  public void testFloatRangeDocValuesIntersectQuery() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int iters = atLeast(10);
+    float[] min = {3.7f, 11.0f, 33.4f};
+    float[] max = {8.3f, 21.6f, 59.8f};
+    for (int i = 0; i < iters; ++i) {
+      Document doc = new Document();
+      doc.add(new FloatRangeDocValuesField("dv", min, max));
+      iw.addDocument(doc);
+    }
+
+
+    float[] nonMatchingMin = {11.4f, 29.7f, 102.4f};
+    float[] nonMatchingMax = {17.6f, 37.2f, 160.2f};
+    Document doc = new Document();
+    doc.add(new FloatRangeDocValuesField("dv", nonMatchingMin, nonMatchingMax));
+    iw.addDocument(doc);
+
+    iw.commit();
+
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+
+    final float[] lowRange = {1.2f, 8.3f, 21.4f};
+    final float[] highRange = {6.0f, 17.6f, 47.1f};
+
+    Query query = FloatRangeDocValuesField.newSlowIntersectsQuery("dv", lowRange, highRange);
+
+    assertEquals(searcher.count(query), iters);
+
+    final float[] lowRange2 = {6.1f, 17.0f, 31.3f};
+    final float[] highRange2 = {14.2f, 23.4f, 61.1f};
+
+    query = FloatRangeDocValuesField.newSlowIntersectsQuery("dv", lowRange2, highRange2);
+
+    assertEquals(searcher.count(query), iters);
+
+    reader.close();
+    dir.close();
+  }
+
+  public void testToString() {
+    double[] doubleMin = {112.7, 296.0, 512.4f};
+    double[] doubleMax = {119.3, 314.8, 524.3f};
+    Query q1 = DoubleRangeDocValuesField.newSlowIntersectsQuery("foo", doubleMin, doubleMax);
+    assertEquals("foo:[[112.7, 296.0, 512.4000244140625] TO [119.3, 314.8, 524.2999877929688]]", q1.toString());
+
+    int[] intMin = {3, 11, 17};
+    int[] intMax = {27, 35, 49};
+    Query q2 = IntRangeDocValuesField.newSlowIntersectsQuery("foo", intMin, intMax);
+    assertEquals("foo:[[3, 11, 17] TO [27, 35, 49]]", q2.toString());
+
+    float[] floatMin = {3.7f, 11.0f, 33.4f};
+    float[] floatMax = {8.3f, 21.6f, 59.8f};
+    Query q3 = FloatRangeDocValuesField.newSlowIntersectsQuery("foo", floatMin, floatMax);
+    assertEquals("foo:[[3.7, 11.0, 33.4] TO [8.3, 21.6, 59.8]]", q3.toString());
+
+    long[] longMin = {101, 124, 137};
+    long[] longMax = {138, 145, 156};
+    Query q4 = LongRangeDocValuesField.newSlowIntersectsQuery("foo", longMin, longMax);
+    assertEquals("foo:[[101, 124, 137] TO [138, 145, 156]]", q4.toString());
+  }
+}