You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ji...@apache.org on 2019/06/26 07:42:48 UTC

[lucene-solr] branch master updated: LUCENE-7714: Add a range query in sandbox that takes advantage of index sorting.

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 5bf023c  LUCENE-7714: Add a range query in sandbox that takes advantage of index sorting.
5bf023c is described below

commit 5bf023cf19424a7a265e2290b7ea06e594cbeddd
Author: Julie Tibshirani <ju...@elastic.co>
AuthorDate: Fri May 10 13:05:27 2019 -0700

    LUCENE-7714: Add a range query in sandbox that takes advantage of index sorting.
---
 lucene/CHANGES.txt                                 |   5 +-
 .../IndexSortSortedNumericDocValuesRangeQuery.java | 301 ++++++++++++++
 ...tIndexSortSortedNumericDocValuesRangeQuery.java | 446 +++++++++++++++++++++
 3 files changed, 751 insertions(+), 1 deletion(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index f283e14..c557fa2 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -65,6 +65,9 @@ New Features
   in an efficient manner, for use as a monitoring or classification tool.
   (Alan Woodward)
 
+* LUCENE-7714: Add a numeric range query in sandbox that takes advantage of index sorting.
+  (Julie Tibshirani via Jim Ferenczi)
+
 Bug Fixes
 
 * LUCENE-8831: Fixed LatLonShapeBoundingBoxQuery .hashCode methods. (Ignacio Vera)
@@ -445,7 +448,7 @@ Changes in Runtime Behavior
 
 * LUCENE-8535: Highlighter and FVH doesn't support ToParent and ToChildBlockJoinQuery out of the
   box anymore. In order to highlight on Block-Join Queries a custom WeightedSpanTermExtractor / FieldQuery
-  should be used. (Simon Willnauer, Jim Ferenczi, Julie Tibshiran)
+  should be used. (Simon Willnauer, Jim Ferenczi, Julie Tibshirani)
 
 * LUCENE-8563: BM25 scores don't include the (k1+1) factor in their numerator
   anymore. This doesn't affect ordering as this is a constant factor which is
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/IndexSortSortedNumericDocValuesRangeQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/IndexSortSortedNumericDocValuesRangeQuery.java
new file mode 100644
index 0000000..9e22117
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/IndexSortSortedNumericDocValuesRangeQuery.java
@@ -0,0 +1,301 @@
+/*
+ * 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.Objects;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
+
+/**
+ * A range query that can take advantage of the fact that the index is sorted to speed up
+ * execution. If the index is sorted on the same field as the query, it performs binary
+ * search on the field's numeric doc values to find the documents at the lower and upper
+ * ends of the range.
+ *
+ * This optimized execution strategy is only used if the following conditions hold:
+ * <ul>
+ *   <li> The index is sorted, and its primary sort is on the same field as the query.
+ *   <li> The query field has either {@link SortedNumericDocValues} or {@link NumericDocValues}.
+ *   <li> The segments must have at most one field value per document (otherwise we cannot easily
+ * determine the matching document IDs through a binary search).
+ * </ul>
+ *
+ * If any of these conditions isn't met, the search is delegated to {@code fallbackQuery}.
+ *
+ * This fallback must be an equivalent range query -- it should produce the same documents and give
+ * constant scores. As an example, an {@link IndexSortSortedNumericDocValuesRangeQuery} might be
+ * constructed as follows:
+ * <pre class="prettyprint">
+ *   String field = "field";
+ *   long lowerValue = 0, long upperValue = 10;
+ *   Query fallbackQuery = LongPoint.newRangeQuery(field, lowerValue, upperValue);
+ *   Query rangeQuery = new IndexSortSortedNumericDocValuesRangeQuery(
+ *       field, lowerValue, upperValue, fallbackQuery);
+ * </pre>
+ *
+ * @lucene.experimental
+ */
+public class IndexSortSortedNumericDocValuesRangeQuery extends Query {
+
+  private final String field;
+  private final long lowerValue;
+  private final long upperValue;
+  private final Query fallbackQuery;
+
+  /**
+   * Creates a new {@link IndexSortSortedNumericDocValuesRangeQuery}.
+   *
+   * @param field The field name.
+   * @param lowerValue The lower end of the range (inclusive).
+   * @param upperValue The upper end of the range (exclusive).
+   * @param fallbackQuery A query to fall back to if the optimization cannot be applied.
+      */
+  public IndexSortSortedNumericDocValuesRangeQuery(String field,
+                                                   long lowerValue,
+                                                   long upperValue,
+                                                   Query fallbackQuery) {
+    this.field = Objects.requireNonNull(field);
+    this.lowerValue = lowerValue;
+    this.upperValue = upperValue;
+    this.fallbackQuery = fallbackQuery;
+  }
+
+  public Query getFallbackQuery() {
+    return fallbackQuery;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    IndexSortSortedNumericDocValuesRangeQuery that = (IndexSortSortedNumericDocValuesRangeQuery) o;
+    return lowerValue == that.lowerValue &&
+        upperValue == that.upperValue &&
+        Objects.equals(field, that.field) &&
+        Objects.equals(fallbackQuery, that.fallbackQuery);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(field, lowerValue, upperValue, fallbackQuery);
+  }
+
+  @Override
+  public void visit(QueryVisitor visitor) {
+    if (visitor.acceptField(field)) {
+      visitor.visitLeaf(this);
+      fallbackQuery.visit(visitor);
+    }
+  }
+
+  @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(lowerValue)
+        .append(" TO ")
+        .append(upperValue)
+        .append("]")
+        .toString();
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    if (lowerValue == Long.MIN_VALUE && upperValue == Long.MAX_VALUE) {
+      return new DocValuesFieldExistsQuery(field);
+    }
+
+    Query rewrittenFallback = fallbackQuery.rewrite(reader);
+    if (rewrittenFallback == fallbackQuery) {
+      return this;
+    } else {
+      return new IndexSortSortedNumericDocValuesRangeQuery(
+          field, lowerValue, upperValue, rewrittenFallback);
+    }
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+    Weight fallbackWeight = fallbackQuery.createWeight(searcher, scoreMode, boost);
+
+    return new ConstantScoreWeight(this, boost) {
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        SortedNumericDocValues sortedNumericValues = DocValues.getSortedNumeric(context.reader(), field);
+        NumericDocValues numericValues = DocValues.unwrapSingleton(sortedNumericValues);
+
+        if (numericValues != null) {
+          Sort indexSort = context.reader().getMetaData().getSort();
+          if (indexSort != null
+              && indexSort.getSort().length > 0
+              && indexSort.getSort()[0].getField().equals(field)) {
+
+            SortField sortField = indexSort.getSort()[0];
+            DocIdSetIterator disi = getDocIdSetIterator(sortField, context, numericValues);
+            return new ConstantScoreScorer(this, score(), scoreMode, disi);
+          }
+        }
+        return fallbackWeight.scorer(context);
+      }
+
+      @Override
+      public boolean isCacheable(LeafReaderContext ctx) {
+        // Both queries should always return the same values, so we can just check
+        // if the fallback query is cacheable.
+        return fallbackWeight.isCacheable(ctx);
+      }
+    };
+  }
+
+  /**
+   * Computes the document IDs that lie within the range [lowerValue, upperValue] by
+   * performing binary search on the field's doc values.
+   *
+   * Because doc values only allow forward iteration, we need to reload the field comparator
+   * every time the binary search accesses an earlier element.
+   *
+   * We must also account for missing values when performing the binary search. For this
+   * reason, we load the {@link FieldComparator} instead of checking the docvalues directly.
+   * The returned {@link DocIdSetIterator} makes sure to wrap the original docvalues to skip
+   * over documents with no value.
+   */
+  private DocIdSetIterator getDocIdSetIterator(SortField sortField,
+                                               LeafReaderContext context,
+                                               DocIdSetIterator delegate) throws IOException {
+    long lower = sortField.getReverse() ? upperValue : lowerValue;
+    long upper = sortField.getReverse() ? lowerValue : upperValue;
+    int maxDoc = context.reader().maxDoc();
+
+    // Perform a binary search to find the first document with value >= lower.
+    ValueComparator comparator = loadComparator(sortField, lower, context);
+    int low = 0;
+    int high = maxDoc - 1;
+
+    while (low <= high) {
+      int mid = (low + high) >>> 1;
+      if (comparator.compare(mid) <= 0) {
+        high = mid - 1;
+        comparator = loadComparator(sortField, lower, context);
+      } else {
+        low = mid + 1;
+      }
+    }
+    int firstDocIdInclusive = high + 1;
+
+    // Perform a binary search to find the first document with value > upper.
+    // Since we know that upper >= lower, we can initialize the lower bound
+    // of the binary search to the result of the previous search.
+    comparator = loadComparator(sortField, upper, context);
+    low = firstDocIdInclusive;
+    high = maxDoc - 1;
+
+    while (low <= high) {
+      int mid = (low + high) >>> 1;
+      if (comparator.compare(mid) < 0) {
+        high = mid - 1;
+        comparator = loadComparator(sortField, upper, context);
+      } else {
+        low = mid + 1;
+      }
+    }
+
+    int lastDocIdExclusive = high + 1;
+    return new BoundedDocSetIdIterator(firstDocIdInclusive, lastDocIdExclusive, delegate);
+  }
+
+  /**
+   * Compares the given document's value with a stored reference value.
+   */
+  private interface ValueComparator {
+    int compare(int docID) throws IOException;
+  }
+
+  private static ValueComparator loadComparator(SortField sortField,
+                                                long topValue,
+                                                LeafReaderContext context) throws IOException {
+    @SuppressWarnings("unchecked")
+    FieldComparator<Long> fieldComparator = (FieldComparator<Long>) sortField.getComparator(1, 0);
+    fieldComparator.setTopValue(topValue);
+
+    LeafFieldComparator leafFieldComparator = fieldComparator.getLeafComparator(context);
+    int direction = sortField.getReverse() ? -1 : 1;
+
+    return doc -> {
+      int value = leafFieldComparator.compareTop(doc);
+      return direction * value;
+    };
+  }
+
+  /**
+   * A doc ID set iterator that wraps a delegate iterator and only returns doc IDs in
+   * the range [firstDocInclusive, lastDoc).
+   */
+  private static class BoundedDocSetIdIterator extends DocIdSetIterator {
+    private final int firstDoc;
+    private final int lastDoc;
+    private final DocIdSetIterator delegate;
+
+    private int docID = -1;
+
+    BoundedDocSetIdIterator(int firstDoc,
+                            int lastDoc,
+                            DocIdSetIterator delegate) {
+      this.firstDoc = firstDoc;
+      this.lastDoc = lastDoc;
+      this.delegate = delegate;
+    }
+
+    @Override
+    public int docID() {
+      return docID;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return advance(docID + 1);
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      if (target < firstDoc) {
+        target = firstDoc;
+      }
+
+      int result = delegate.advance(target);
+      if (result < lastDoc) {
+        docID = result;
+        return docID;
+      } else {
+        return NO_MORE_DOCS;
+      }
+    }
+
+    @Override
+    public long cost() {
+      return lastDoc - firstDoc;
+    }
+  }
+}
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestIndexSortSortedNumericDocValuesRangeQuery.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestIndexSortSortedNumericDocValuesRangeQuery.java
new file mode 100644
index 0000000..da6bcf1b
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestIndexSortSortedNumericDocValuesRangeQuery.java
@@ -0,0 +1,446 @@
+/*
+ * 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.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.document.SortedSetDocValuesField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+
+public class TestIndexSortSortedNumericDocValuesRangeQuery extends LuceneTestCase {
+
+  public void testSameHitsAsPointRangeQuery() throws IOException {
+    final int iters = atLeast(10);
+    for (int iter = 0; iter < iters; ++iter) {
+      Directory dir = newDirectory();
+
+      IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+      boolean reverse = random().nextBoolean();
+      SortField sortField = new SortedNumericSortField("dv", SortField.Type.LONG, reverse);
+      sortField.setMissingValue(random().nextLong());
+      iwc.setIndexSort(new Sort(sortField));
+
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final int numValues = TestUtil.nextInt(random(), 0, 1);
+        for (int j = 0; j < numValues; ++j) {
+          final long value = TestUtil.nextLong(random(), -100, 10000);
+          doc.add(new SortedNumericDocValuesField("dv", value));
+          doc.add(new LongPoint("idx", value));
+        }
+        iw.addDocument(doc);
+      }
+      if (random().nextBoolean()) {
+        iw.deleteDocuments(LongPoint.newRangeQuery("idx", 0L, 10L));
+      }
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader, false);
+      iw.close();
+
+      for (int i = 0; i < 100; ++i) {
+        final long min = random().nextBoolean() ? Long.MIN_VALUE : TestUtil.nextLong(random(), -100, 10000);
+        final long max = random().nextBoolean() ? Long.MAX_VALUE : TestUtil.nextLong(random(), -100, 10000);
+        final Query q1 = LongPoint.newRangeQuery("idx", min, max);
+        final Query q2 = createQuery("dv", min, max);
+        assertSameHits(searcher, q1, q2, false);
+      }
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  private void assertSameHits(IndexSearcher searcher, Query q1, Query q2, boolean scores) throws IOException {
+    final int maxDoc = searcher.getIndexReader().maxDoc();
+    final TopDocs td1 = searcher.search(q1, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
+    final TopDocs td2 = searcher.search(q2, maxDoc, scores ? Sort.RELEVANCE : Sort.INDEXORDER);
+    assertEquals(td1.totalHits.value, td2.totalHits.value);
+    for (int i = 0; i < td1.scoreDocs.length; ++i) {
+      assertEquals(td1.scoreDocs[i].doc, td2.scoreDocs[i].doc);
+      if (scores) {
+        assertEquals(td1.scoreDocs[i].score, td2.scoreDocs[i].score, 10e-7);
+      }
+    }
+  }
+
+  public void testEquals() {
+    Query q1 = createQuery("foo", 3, 5);
+    QueryUtils.checkEqual(q1, createQuery("foo", 3, 5));
+    QueryUtils.checkUnequal(q1, createQuery("foo", 3, 6));
+    QueryUtils.checkUnequal(q1, createQuery("foo", 4, 5));
+    QueryUtils.checkUnequal(q1, createQuery("bar", 3, 5));
+  }
+
+  public void testToString() {
+    Query q1 = createQuery("foo", 3, 5);
+    assertEquals("foo:[3 TO 5]", q1.toString());
+    assertEquals("[3 TO 5]", q1.toString("foo"));
+    assertEquals("foo:[3 TO 5]", q1.toString("bar"));
+
+    Query q2 = SortedSetDocValuesField.newSlowRangeQuery("foo", new BytesRef("bar"), new BytesRef("baz"), true, true);
+    assertEquals("foo:[[62 61 72] TO [62 61 7a]]", q2.toString());
+    q2 = SortedSetDocValuesField.newSlowRangeQuery("foo", new BytesRef("bar"), new BytesRef("baz"), false, true);
+    assertEquals("foo:{[62 61 72] TO [62 61 7a]]", q2.toString());
+    q2 = SortedSetDocValuesField.newSlowRangeQuery("foo", new BytesRef("bar"), new BytesRef("baz"), false, false);
+    assertEquals("foo:{[62 61 72] TO [62 61 7a]}", q2.toString());
+    q2 = SortedSetDocValuesField.newSlowRangeQuery("foo", new BytesRef("bar"), null, true, true);
+    assertEquals("foo:[[62 61 72] TO *}", q2.toString());
+    q2 = SortedSetDocValuesField.newSlowRangeQuery("foo", null, new BytesRef("baz"), true, true);
+    assertEquals("foo:{* TO [62 61 7a]]", q2.toString());
+    assertEquals("{* TO [62 61 7a]]", q2.toString("foo"));
+    assertEquals("foo:{* TO [62 61 7a]]", q2.toString("bar"));
+  }
+
+  public void testIndexSortDocValuesWithEvenLength() throws Exception {
+    testIndexSortDocValuesWithEvenLength(false);
+    testIndexSortDocValuesWithEvenLength(true);
+  }
+
+  public void testIndexSortDocValuesWithEvenLength(boolean reverse) throws Exception {
+    Directory dir = newDirectory();
+
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    Sort indexSort = new Sort(new SortedNumericSortField("field", SortField.Type.LONG, reverse));
+    iwc.setIndexSort(indexSort);
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+
+    writer.addDocument(createDocument("field", -80));
+    writer.addDocument(createDocument("field", -5));
+    writer.addDocument(createDocument("field", 0));
+    writer.addDocument(createDocument("field", 0));
+    writer.addDocument(createDocument("field", 30));
+    writer.addDocument(createDocument("field", 35));
+
+    DirectoryReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+
+    // Test ranges consisting of one value.
+    assertEquals(1, searcher.count(createQuery("field", -80, -80)));
+    assertEquals(1, searcher.count(createQuery("field", -5, -5)));
+    assertEquals(2, searcher.count(createQuery("field", 0, 0)));
+    assertEquals(1, searcher.count(createQuery("field", 30, 30)));
+    assertEquals(1, searcher.count(createQuery("field", 35, 35)));
+
+    assertEquals(0, searcher.count(createQuery("field", -90, -90)));
+    assertEquals(0, searcher.count(createQuery("field", 5, 5)));
+    assertEquals(0, searcher.count(createQuery("field", 40, 40)));
+
+    // Test the lower end of the document value range.
+    assertEquals(2, searcher.count(createQuery("field", -90, -4)));
+    assertEquals(2, searcher.count(createQuery("field", -80, -4)));
+    assertEquals(1, searcher.count(createQuery("field", -70, -4)));
+    assertEquals(2, searcher.count(createQuery("field", -80, -5)));
+
+    // Test the upper end of the document value range.
+    assertEquals(1, searcher.count(createQuery("field", 25, 34)));
+    assertEquals(2, searcher.count(createQuery("field", 25, 35)));
+    assertEquals(2, searcher.count(createQuery("field", 25, 36)));
+    assertEquals(2, searcher.count(createQuery("field", 30, 35)));
+
+    // Test multiple occurrences of the same value.
+    assertEquals(2, searcher.count(createQuery("field", -4, 4)));
+    assertEquals(2, searcher.count(createQuery("field", -4, 0)));
+    assertEquals(2, searcher.count(createQuery("field", 0, 4)));
+    assertEquals(3, searcher.count(createQuery("field", 0, 30)));
+
+    // Test ranges that span all documents.
+    assertEquals(6, searcher.count(createQuery("field", -80, 35)));
+    assertEquals(6, searcher.count(createQuery("field", -90, 40)));
+
+    writer.close();
+    reader.close();
+    dir.close();
+  }
+
+  public void testIndexSortDocValuesWithOddLength() throws Exception {
+    testIndexSortDocValuesWithOddLength(false);
+    testIndexSortDocValuesWithOddLength(true);
+  }
+
+  public void testIndexSortDocValuesWithOddLength(boolean reverse) throws Exception {
+    Directory dir = newDirectory();
+
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    Sort indexSort = new Sort(new SortedNumericSortField("field", SortField.Type.LONG, reverse));
+    iwc.setIndexSort(indexSort);
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+
+    writer.addDocument(createDocument("field", -80));
+    writer.addDocument(createDocument("field", -5));
+    writer.addDocument(createDocument("field", 0));
+    writer.addDocument(createDocument("field", 0));
+    writer.addDocument(createDocument("field", 5));
+    writer.addDocument(createDocument("field", 30));
+    writer.addDocument(createDocument("field", 35));
+
+    DirectoryReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+
+    // Test ranges consisting of one value.
+    assertEquals(1, searcher.count(createQuery("field", -80, -80)));
+    assertEquals(1, searcher.count(createQuery("field", -5, -5)));
+    assertEquals(2, searcher.count(createQuery("field", 0, 0)));
+    assertEquals(1, searcher.count(createQuery("field", 5, 5)));
+    assertEquals(1, searcher.count(createQuery("field", 30, 30)));
+    assertEquals(1, searcher.count(createQuery("field", 35, 35)));
+
+    assertEquals(0, searcher.count(createQuery("field", -90, -90)));
+    assertEquals(0, searcher.count(createQuery("field", 6, 6)));
+    assertEquals(0, searcher.count(createQuery("field", 40, 40)));
+
+    // Test the lower end of the document value range.
+    assertEquals(2, searcher.count(createQuery("field", -90, -4)));
+    assertEquals(2, searcher.count(createQuery("field", -80, -4)));
+    assertEquals(1, searcher.count(createQuery("field", -70, -4)));
+    assertEquals(2, searcher.count(createQuery("field", -80, -5)));
+
+    // Test the upper end of the document value range.
+    assertEquals(1, searcher.count(createQuery("field", 25, 34)));
+    assertEquals(2, searcher.count(createQuery("field", 25, 35)));
+    assertEquals(2, searcher.count(createQuery("field", 25, 36)));
+    assertEquals(2, searcher.count(createQuery("field", 30, 35)));
+
+    // Test multiple occurrences of the same value.
+    assertEquals(2, searcher.count(createQuery("field", -4, 4)));
+    assertEquals(2, searcher.count(createQuery("field", -4, 0)));
+    assertEquals(2, searcher.count(createQuery("field", 0, 4)));
+    assertEquals(4, searcher.count(createQuery("field", 0, 30)));
+
+    // Test ranges that span all documents.
+    assertEquals(7, searcher.count(createQuery("field", -80, 35)));
+    assertEquals(7, searcher.count(createQuery("field", -90, 40)));
+
+    writer.close();
+    reader.close();
+    dir.close();
+  }
+
+  public void testIndexSortDocValuesWithSingleValue() throws Exception {
+    testIndexSortDocValuesWithSingleValue(false);
+    testIndexSortDocValuesWithSingleValue(true);
+  }
+
+  private void testIndexSortDocValuesWithSingleValue(boolean reverse) throws IOException{
+    Directory dir = newDirectory();
+
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    Sort indexSort = new Sort(new SortedNumericSortField("field", SortField.Type.LONG, reverse));
+    iwc.setIndexSort(indexSort);
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+
+    writer.addDocument(createDocument("field", 42));
+
+    DirectoryReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+
+    assertEquals(1, searcher.count(createQuery("field", 42, 43)));
+    assertEquals(1, searcher.count(createQuery("field", 42, 42)));
+    assertEquals(0, searcher.count(createQuery("field", 41, 41)));
+    assertEquals(0, searcher.count(createQuery("field", 43, 43)));
+
+    writer.close();
+    reader.close();
+    dir.close();
+  }
+
+  public void testIndexSortMissingValues() throws Exception {
+    Directory dir = newDirectory();
+
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    SortField sortField = new SortedNumericSortField("field", SortField.Type.LONG);
+    sortField.setMissingValue(random().nextLong());
+    iwc.setIndexSort(new Sort(sortField));
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+
+    writer.addDocument(createDocument("field", -80));
+    writer.addDocument(createDocument("field", -5));
+    writer.addDocument(createDocument("field", 0));
+    writer.addDocument(createDocument("field", 35));
+
+    writer.addDocument(createDocument("other-field", 0));
+    writer.addDocument(createDocument("other-field", 10));
+    writer.addDocument(createDocument("other-field", 20));
+
+    DirectoryReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+
+    assertEquals(2, searcher.count(createQuery("field", -70, 0)));
+    assertEquals(2, searcher.count(createQuery("field", -2, 35)));
+
+    assertEquals(4, searcher.count(createQuery("field", -80, 35)));
+    assertEquals(4, searcher.count(createQuery("field", Long.MIN_VALUE, Long.MAX_VALUE)));
+
+    writer.close();
+    reader.close();
+    dir.close();
+  }
+
+  public void testNoDocuments() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    writer.addDocument(new Document());
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+    Query query = createQuery("foo", 2, 4);
+    Weight w = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1);
+    assertNull(w.scorer(searcher.getIndexReader().leaves().get(0)));
+
+    writer.close();
+    reader.close();
+    dir.close();
+  }
+
+  public void testRewriteExhaustiveRange() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    writer.addDocument(new Document());
+    IndexReader reader = writer.getReader();
+
+    Query query = createQuery("field", Long.MIN_VALUE, Long.MAX_VALUE);
+    Query rewrittenQuery = query.rewrite(reader);
+    assertEquals(new DocValuesFieldExistsQuery("field"), rewrittenQuery);
+
+    writer.close();
+    reader.close();
+    dir.close();
+  }
+
+  public void testRewriteFallbackQuery() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    writer.addDocument(new Document());
+    IndexReader reader = writer.getReader();
+
+    // Create an (unrealistic) fallback query that is sure to be rewritten.
+    Query fallbackQuery = new BooleanQuery.Builder().build();
+    Query query = new IndexSortSortedNumericDocValuesRangeQuery("field", 1, 42, fallbackQuery);
+
+    Query rewrittenQuery = query.rewrite(reader);
+    assertNotEquals(query, rewrittenQuery);
+    assertThat(rewrittenQuery, instanceOf(IndexSortSortedNumericDocValuesRangeQuery.class));
+
+    IndexSortSortedNumericDocValuesRangeQuery rangeQuery = (IndexSortSortedNumericDocValuesRangeQuery) rewrittenQuery;
+    assertEquals(new MatchNoDocsQuery(), rangeQuery.getFallbackQuery());
+
+    writer.close();
+    reader.close();
+    dir.close();
+  }
+
+  /**
+   * Test that the index sort optimization not activated if there is no index sort.
+   */
+  public void testNoIndexSort() throws Exception {
+    Directory dir = newDirectory();
+
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    writer.addDocument(createDocument("field", 0));
+
+    testIndexSortOptimizationDeactivated(writer);
+
+    writer.close();
+    dir.close();
+  }
+
+  /**
+   * Test that the index sort optimization is not activated when the sort is
+   * on the wrong field.
+   */
+  public void testIndexSortOnWrongField() throws Exception {
+    Directory dir = newDirectory();
+
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    Sort indexSort = new Sort(new SortedNumericSortField("other-field", SortField.Type.LONG));
+    iwc.setIndexSort(indexSort);
+
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+    writer.addDocument(createDocument("field", 0));
+
+    testIndexSortOptimizationDeactivated(writer);
+
+    writer.close();
+    dir.close();
+  }
+
+  /**
+   * Test that the index sort optimization is not activated when some documents
+   * have multiple values.
+   */
+  public void testMultiDocValues() throws Exception {
+    Directory dir = newDirectory();
+
+    IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
+    Sort indexSort = new Sort(new SortedNumericSortField("field", SortField.Type.LONG));
+    iwc.setIndexSort(indexSort);
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
+
+    Document doc = new Document();
+    doc.add(new SortedNumericDocValuesField("field", 0));
+    doc.add(new SortedNumericDocValuesField("field", 10));
+    writer.addDocument(doc);
+
+    testIndexSortOptimizationDeactivated(writer);
+
+    writer.close();
+    dir.close();
+  }
+
+  public void testIndexSortOptimizationDeactivated(RandomIndexWriter writer) throws IOException {
+    DirectoryReader reader = writer.getReader();
+    IndexSearcher searcher = newSearcher(reader);
+
+    Query query = createQuery("field", 0, 0);
+    Weight weight = query.createWeight(searcher, ScoreMode.TOP_SCORES, 1.0F);
+
+    // Check that the two-phase iterator is not null, indicating that we've fallen
+    // back to SortedNumericDocValuesField.newSlowRangeQuery.
+    for (LeafReaderContext context : searcher.getIndexReader().leaves()) {
+      Scorer scorer = weight.scorer(context);
+      assertNotNull(scorer.twoPhaseIterator());
+    }
+
+    reader.close();
+  }
+
+  private Document createDocument(String field, long value) {
+    Document doc = new Document();
+    doc.add(new SortedNumericDocValuesField(field, value));
+    return doc;
+  }
+
+  private Query createQuery(String field, long lowerValue, long upperValue) {
+    Query fallbackQuery = SortedNumericDocValuesField.newSlowRangeQuery(field, lowerValue, upperValue);
+    return new IndexSortSortedNumericDocValuesRangeQuery(field, lowerValue, upperValue, fallbackQuery);
+  }
+}