You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2018/04/03 08:38:49 UTC

[3/4] lucene-solr:branch_7x: LUCENE-8196: Add IntervalQuery and IntervalsSource to the sandbox

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/TermIntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/TermIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/TermIntervalsSource.java
new file mode 100644
index 0000000..b375a01
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/TermIntervalsSource.java
@@ -0,0 +1,182 @@
+/*
+ * 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.intervals;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
+import org.apache.lucene.codecs.lucene50.Lucene50PostingsReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.TwoPhaseIterator;
+import org.apache.lucene.util.BytesRef;
+
+class TermIntervalsSource extends IntervalsSource {
+
+  final BytesRef term;
+
+  TermIntervalsSource(BytesRef term) {
+    this.term = term;
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    Terms terms = ctx.reader().terms(field);
+    if (terms == null)
+      return null;
+    if (terms.hasPositions() == false) {
+      throw new IllegalArgumentException("Cannot create an IntervalIterator over field " + field + " because it has no indexed positions");
+    }
+    TermsEnum te = terms.iterator();
+    te.seekExact(term);
+    PostingsEnum pe = te.postings(null, PostingsEnum.POSITIONS);
+    float cost = termPositionsCost(te);
+    return new IntervalIterator() {
+
+      @Override
+      public int docID() {
+        return pe.docID();
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        int doc = pe.nextDoc();
+        reset();
+        return doc;
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        int doc = pe.advance(target);
+        reset();
+        return doc;
+      }
+
+      @Override
+      public long cost() {
+        return pe.cost();
+      }
+
+      int pos = -1, upto;
+
+      @Override
+      public int start() {
+        return pos;
+      }
+
+      @Override
+      public int end() {
+        return pos;
+      }
+
+      @Override
+      public int nextInterval() throws IOException {
+        if (upto <= 0)
+          return pos = NO_MORE_INTERVALS;
+        upto--;
+        return pos = pe.nextPosition();
+      }
+
+      @Override
+      public float matchCost() {
+        return cost;
+      }
+
+      private void reset() throws IOException {
+        if (pe.docID() == NO_MORE_DOCS) {
+          upto = -1;
+          pos = NO_MORE_INTERVALS;
+        }
+        else {
+          upto = pe.freq();
+          pos = -1;
+        }
+      }
+
+      @Override
+      public String toString() {
+        return term.utf8ToString() + ":" + super.toString();
+      }
+    };
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(term);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    TermIntervalsSource that = (TermIntervalsSource) o;
+    return Objects.equals(term, that.term);
+  }
+
+  @Override
+  public String toString() {
+    return term.utf8ToString();
+  }
+
+  @Override
+  public void extractTerms(String field, Set<Term> terms) {
+    terms.add(new Term(field, term));
+  }
+
+  /** A guess of
+   * the average number of simple operations for the initial seek and buffer refill
+   * per document for the positions of a term.
+   * See also {@link Lucene50PostingsReader.BlockPostingsEnum#nextPosition()}.
+   * <p>
+   * Aside: Instead of being constant this could depend among others on
+   * {@link Lucene50PostingsFormat#BLOCK_SIZE},
+   * {@link TermsEnum#docFreq()},
+   * {@link TermsEnum#totalTermFreq()},
+   * {@link DocIdSetIterator#cost()} (expected number of matching docs),
+   * {@link LeafReader#maxDoc()} (total number of docs in the segment),
+   * and the seek time and block size of the device storing the index.
+   */
+  private static final int TERM_POSNS_SEEK_OPS_PER_DOC = 128;
+
+  /** Number of simple operations in {@link Lucene50PostingsReader.BlockPostingsEnum#nextPosition()}
+   *  when no seek or buffer refill is done.
+   */
+  private static final int TERM_OPS_PER_POS = 7;
+
+  /** Returns an expected cost in simple operations
+   *  of processing the occurrences of a term
+   *  in a document that contains the term.
+   *  This is for use by {@link TwoPhaseIterator#matchCost} implementations.
+   *  @param termsEnum The term is the term at which this TermsEnum is positioned.
+   */
+  static float termPositionsCost(TermsEnum termsEnum) throws IOException {
+    // TODO: When intervals move to core, refactor to use the copy of this in PhraseQuery
+    int docFreq = termsEnum.docFreq();
+    assert docFreq > 0;
+    long totalTermFreq = termsEnum.totalTermFreq();
+    float expOccurrencesInMatchingDoc = totalTermFreq / (float) docFreq;
+    return TERM_POSNS_SEEK_OPS_PER_DOC + expOccurrencesInMatchingDoc * TERM_OPS_PER_POS;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/package-info.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/package-info.java
new file mode 100644
index 0000000..88d93ea
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/package-info.java
@@ -0,0 +1,66 @@
+/*
+ * 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.
+ */
+
+/**
+ * <h2>Intervals queries</h2>
+ *
+ * This package contains experimental classes to search over intervals within fields
+ *
+ * <h2>IntervalsSource</h2>
+ *
+ * The {@link org.apache.lucene.search.intervals.IntervalsSource} class can be used to construct proximity
+ * relationships between terms and intervals.  They can be built using static methods
+ * in the {@link org.apache.lucene.search.intervals.Intervals} class
+ *
+ * <h3>Basic intervals</h3>
+ *
+ * <ul>
+ *   <li>{@link org.apache.lucene.search.intervals.Intervals#term(String)} &mdash; Represents a single term</li>
+ *   <li>{@link org.apache.lucene.search.intervals.Intervals#phrase(java.lang.String...)} &mdash; Represents a phrase</li>
+ *   <li>{@link org.apache.lucene.search.intervals.Intervals#ordered(org.apache.lucene.search.intervals.IntervalsSource...)}
+ *        &mdash; Represents an interval over an ordered set of terms or intervals</li>
+ *   <li>{@link org.apache.lucene.search.intervals.Intervals#unordered(org.apache.lucene.search.intervals.IntervalsSource...)}
+ *        &mdash; Represents an interval over an unordered set of terms or intervals</li>
+ *   <li>{@link org.apache.lucene.search.intervals.Intervals#or(org.apache.lucene.search.intervals.IntervalsSource...)}
+ *        &mdash; Represents the disjunction of a set of terms or intervals</li>
+ * </ul>
+ *
+ * <h3>Filters</h3>
+ *
+ * <ul>
+ *   <li>{@link org.apache.lucene.search.intervals.Intervals#maxwidth(int, org.apache.lucene.search.intervals.IntervalsSource)}
+ *          &mdash; Filters out intervals that are larger than a set width</li>
+ *   <li>{@link org.apache.lucene.search.intervals.Intervals#containedBy(org.apache.lucene.search.intervals.IntervalsSource, org.apache.lucene.search.intervals.IntervalsSource)}
+ *          &mdash; Returns intervals that are contained by another interval</li>
+ *   <li>{@link org.apache.lucene.search.intervals.Intervals#notContainedBy(org.apache.lucene.search.intervals.IntervalsSource, org.apache.lucene.search.intervals.IntervalsSource)}
+ *          &mdash; Returns intervals that are *not* contained by another interval</li>
+ *   <li>{@link org.apache.lucene.search.intervals.Intervals#containing(org.apache.lucene.search.intervals.IntervalsSource, org.apache.lucene.search.intervals.IntervalsSource)}
+ *          &mdash; Returns intervals that contain another interval</li>
+ *   <li>{@link org.apache.lucene.search.intervals.Intervals#notContaining(org.apache.lucene.search.intervals.IntervalsSource, org.apache.lucene.search.intervals.IntervalsSource)}
+ *          &mdash; Returns intervals that do not contain another interval</li>
+ *   <li>{@link org.apache.lucene.search.intervals.Intervals#nonOverlapping(org.apache.lucene.search.intervals.IntervalsSource, org.apache.lucene.search.intervals.IntervalsSource)}
+ *          &mdash; Returns intervals that do not overlap with another interval</li>
+ *   <li>{@link org.apache.lucene.search.intervals.Intervals#notWithin(org.apache.lucene.search.intervals.IntervalsSource, int, org.apache.lucene.search.intervals.IntervalsSource)}
+ *          &mdash; Returns intervals that do not appear within a set number of positions of another interval</li>
+ * </ul>
+ *
+ * <h2>IntervalQuery</h2>
+ *
+ * An {@link org.apache.lucene.search.intervals.IntervalQuery} takes a field name and an {@link org.apache.lucene.search.intervals.IntervalsSource},
+ * and matches all documents that contain intervals defined by the source in that field.
+ */
+package org.apache.lucene.search.intervals;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervalQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervalQuery.java b/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervalQuery.java
new file mode 100644
index 0000000..a81ab76
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervalQuery.java
@@ -0,0 +1,164 @@
+/*
+ * 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.intervals;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.CheckHits;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.Ignore;
+
+public class TestIntervalQuery extends LuceneTestCase {
+
+  private IndexSearcher searcher;
+  private IndexReader reader;
+  private Directory directory;
+
+  public static final String field = "field";
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    directory = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), directory, newIndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
+    for (int i = 0; i < docFields.length; i++) {
+      Document doc = new Document();
+      doc.add(newTextField(field, docFields[i], Field.Store.YES));
+      writer.addDocument(doc);
+    }
+    reader = writer.getReader();
+    writer.close();
+    searcher = newSearcher(reader);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    reader.close();
+    directory.close();
+    super.tearDown();
+  }
+
+  private String[] docFields = {
+      "w1 w2 w3 w4 w5",
+      "w1 w3 w2 w3",
+      "w1 xx w2 w4 yy w3",
+      "w1 w3 xx w2 yy w3",
+      "w2 w1",
+      "w2 w1 w3 w2 w4",
+      "coordinate genome mapping research",
+      "coordinate genome research"
+  };
+
+  private void checkHits(Query query, int[] results) throws IOException {
+    CheckHits.checkHits(random(), query, field, searcher, results);
+  }
+
+  public void testPhraseQuery() throws IOException {
+    checkHits(new IntervalQuery(field, Intervals.phrase(Intervals.term("w1"), Intervals.term("w2"))),
+        new int[]{0});
+  }
+
+  public void testOrderedNearQueryWidth3() throws IOException {
+    checkHits(new IntervalQuery(field, Intervals.maxwidth(3, Intervals.ordered(Intervals.term("w1"), Intervals.term("w2")))),
+        new int[]{0, 1, 2, 5});
+  }
+
+  public void testOrderedNearQueryWidth4() throws IOException {
+    checkHits(new IntervalQuery(field, Intervals.maxwidth(4, Intervals.ordered(Intervals.term("w1"), Intervals.term("w2")))),
+        new int[]{0, 1, 2, 3, 5});
+  }
+
+  public void testNestedOrderedNearQuery() throws IOException {
+    // onear/1(w1, onear/2(w2, w3))
+    Query q = new IntervalQuery(field,
+        Intervals.ordered(
+            Intervals.term("w1"),
+            Intervals.maxwidth(3, Intervals.ordered(Intervals.term("w2"), Intervals.term("w3")))));
+
+    checkHits(q, new int[]{0, 1, 3});
+  }
+
+  public void testUnorderedQuery() throws IOException {
+    Query q = new IntervalQuery(field, Intervals.unordered(Intervals.term("w1"), Intervals.term("w3")));
+    checkHits(q, new int[]{0, 1, 2, 3, 5});
+  }
+
+  public void testNonOverlappingQuery() throws IOException {
+    Query q = new IntervalQuery(field, Intervals.nonOverlapping(
+        Intervals.unordered(Intervals.term("w1"), Intervals.term("w3")),
+        Intervals.unordered(Intervals.term("w2"), Intervals.term("w4"))));
+    checkHits(q, new int[]{1, 3, 5});
+  }
+
+  public void testNotWithinQuery() throws IOException {
+    Query q = new IntervalQuery(field, Intervals.notWithin(Intervals.term("w1"), 1, Intervals.term("w2")));
+    checkHits(q, new int[]{ 1, 2, 3 });
+  }
+
+  public void testNotContainingQuery() throws IOException {
+    Query q = new IntervalQuery(field, Intervals.notContaining(
+        Intervals.unordered(Intervals.term("w1"), Intervals.term("w2")),
+        Intervals.term("w3")
+    ));
+    checkHits(q, new int[]{ 0, 2, 4, 5 });
+  }
+
+  public void testContainingQuery() throws IOException {
+    Query q = new IntervalQuery(field, Intervals.containing(
+        Intervals.unordered(Intervals.term("w1"), Intervals.term("w2")),
+        Intervals.term("w3")
+    ));
+    checkHits(q, new int[]{ 1, 3, 5 });
+  }
+
+  public void testContainedByQuery() throws IOException {
+    Query q = new IntervalQuery(field, Intervals.containedBy(
+        Intervals.term("w3"),
+        Intervals.unordered(Intervals.term("w1"), Intervals.term("w2"))));
+    checkHits(q, new int[]{ 1, 3, 5 });
+  }
+
+  public void testNotContainedByQuery() throws IOException {
+    Query q = new IntervalQuery(field, Intervals.notContainedBy(
+        Intervals.term("w2"),
+        Intervals.unordered(Intervals.term("w1"), Intervals.term("w4"))
+    ));
+    checkHits(q, new int[]{ 1, 3, 4, 5 });
+  }
+
+  // The Vigna paper doesn't deal with prefix disjunctions.  For now, we keep the same
+  // logic as detailed in the paper, but we may want to address it in future so that tests
+  // like the one below will pass
+  @Ignore
+  public void testNestedOr() throws IOException {
+    Query q = new IntervalQuery(field, Intervals.phrase(
+        Intervals.term("coordinate"),
+        Intervals.or(Intervals.phrase("genome", "mapping"), Intervals.term("genome")),
+        Intervals.term("research")));
+    checkHits(q, new int[]{ 6, 7 });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervals.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervals.java b/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervals.java
new file mode 100644
index 0000000..8f91a7f
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervals.java
@@ -0,0 +1,197 @@
+/*
+ * 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.intervals;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class TestIntervals extends LuceneTestCase {
+
+  private static String field1_docs[] = {
+      "Nothing of interest to anyone here",
+      "Pease porridge hot, pease porridge cold, pease porridge in the pot nine days old.  Some like it hot, some like it cold, some like it in the pot nine days old",
+      "Pease porridge cold, pease porridge hot, pease porridge in the pot twelve days old.  Some like it cold, some like it hot, some like it in the fraggle",
+      "Nor here, nowt hot going on in pease this one",
+      "Pease porridge hot, pease porridge cold, pease porridge in the pot nine years old.  Some like it hot, some like it twelve",
+      "Porridge is great"
+  };
+
+  private static String field2_docs[] = {
+      "In Xanadu did Kubla Khan a stately pleasure dome decree",
+      "Where Alph the sacred river ran through caverns measureless to man",
+      "Down to a sunless sea",
+      "So thrice five miles of fertile ground",
+      "With walls and towers were girdled round",
+      "Which was nice"
+  };
+
+  private static Directory directory;
+  private static IndexSearcher searcher;
+  private static Analyzer analyzer = new StandardAnalyzer();
+
+  @BeforeClass
+  public static void setupIndex() throws IOException {
+    directory = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), directory, newIndexWriterConfig(analyzer));
+    for (int i = 0; i < field1_docs.length; i++) {
+      Document doc = new Document();
+      doc.add(new TextField("field1", field1_docs[i], Field.Store.NO));
+      doc.add(new TextField("field2", field2_docs[i], Field.Store.NO));
+      doc.add(new StringField("id", Integer.toString(i), Field.Store.NO));
+      doc.add(new NumericDocValuesField("id", i));
+      writer.addDocument(doc);
+    }
+    writer.close();
+    searcher = new IndexSearcher(DirectoryReader.open(directory));
+  }
+
+  @AfterClass
+  public static void teardownIndex() throws IOException {
+    IOUtils.close(searcher.getIndexReader(), directory);
+  }
+
+  private void checkIntervals(IntervalsSource source, String field, int expectedMatchCount, int[][] expected) throws IOException {
+    int matchedDocs = 0;
+    for (LeafReaderContext ctx : searcher.getIndexReader().leaves()) {
+      assertNull(source.intervals(field + "fake", ctx));
+      NumericDocValues ids = DocValues.getNumeric(ctx.reader(), "id");
+      IntervalIterator intervals = source.intervals(field, ctx);
+      if (intervals == null)
+        continue;
+      for (int doc = 0; doc < ctx.reader().maxDoc(); doc++) {
+        ids.advance(doc);
+        int id = (int) ids.longValue();
+        if (intervals.docID() == doc ||
+            (intervals.docID() < doc && intervals.advance(doc) == doc)) {
+          int i = 0, pos;
+          assertEquals(-1, intervals.start());
+          assertEquals(-1, intervals.end());
+          while ((pos = intervals.nextInterval()) != IntervalIterator.NO_MORE_INTERVALS) {
+            //System.out.println(doc + ": " + intervals);
+            assertEquals(expected[id][i], pos);
+            assertEquals(expected[id][i], intervals.start());
+            assertEquals(expected[id][i + 1], intervals.end());
+            i += 2;
+          }
+          assertEquals(expected[id].length, i);
+          if (i > 0)
+            matchedDocs++;
+        }
+        else {
+          assertEquals(0, expected[id].length);
+        }
+      }
+    }
+    assertEquals(expectedMatchCount, matchedDocs);
+  }
+
+  public void testIntervalsOnFieldWithNoPositions() throws IOException {
+    IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> {
+      Intervals.term("wibble").intervals("id", searcher.getIndexReader().leaves().get(0));
+    });
+    assertEquals("Cannot create an IntervalIterator over field id because it has no indexed positions", e.getMessage());
+  }
+
+  public void testTermQueryIntervals() throws IOException {
+    checkIntervals(Intervals.term("porridge"), "field1", 4, new int[][]{
+        {},
+        { 1, 1, 4, 4, 7, 7 },
+        { 1, 1, 4, 4, 7, 7 },
+        {},
+        { 1, 1, 4, 4, 7, 7 },
+        { 0, 0 }
+    });
+  }
+
+  public void testOrderedNearIntervals() throws IOException {
+    checkIntervals(Intervals.ordered(Intervals.term("pease"), Intervals.term("hot")),
+        "field1", 3, new int[][]{
+        {},
+        { 0, 2, 6, 17 },
+        { 3, 5, 6, 21 },
+        {},
+        { 0, 2, 6, 17 },
+        { }
+    });
+  }
+
+  public void testPhraseIntervals() throws IOException {
+    checkIntervals(Intervals.phrase("pease", "porridge"), "field1", 3, new int[][]{
+        {},
+        { 0, 1, 3, 4, 6, 7 },
+        { 0, 1, 3, 4, 6, 7 },
+        {},
+        { 0, 1, 3, 4, 6, 7 },
+        {}
+    });
+  }
+
+  public void testUnorderedNearIntervals() throws IOException {
+    checkIntervals(Intervals.unordered(Intervals.term("pease"), Intervals.term("hot")),
+        "field1", 4, new int[][]{
+            {},
+            { 0, 2, 2, 3, 6, 17 },
+            { 3, 5, 5, 6, 6, 21 },
+            { 3, 7 },
+            { 0, 2, 2, 3, 6, 17 },
+            {}
+        });
+  }
+
+  public void testIntervalDisjunction() throws IOException {
+    checkIntervals(Intervals.or(Intervals.term("pease"), Intervals.term("hot")), "field1", 4, new int[][]{
+        {},
+        { 0, 0, 2, 2, 3, 3, 6, 6, 17, 17},
+        { 0, 0, 3, 3, 5, 5, 6, 6, 21, 21},
+        { 3, 3, 7, 7 },
+        { 0, 0, 2, 2, 3, 3, 6, 6, 17, 17},
+        {}
+    });
+  }
+
+  public void testNesting() throws IOException {
+    checkIntervals(Intervals.unordered(Intervals.term("pease"), Intervals.term("porridge"), Intervals.or(Intervals.term("hot"), Intervals.term("cold"))),
+        "field1", 3, new int[][]{
+        {},
+        { 0, 2, 1, 3, 2, 4, 3, 5, 4, 6, 5, 7, 6, 17 },
+        { 0, 2, 1, 3, 2, 4, 3, 5, 4, 6, 5, 7, 6, 17 },
+        {},
+        { 0, 2, 1, 3, 2, 4, 3, 5, 4, 6, 5, 7, 6, 17 },
+        {}
+    });
+  }
+
+}