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:47 UTC

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

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x a14948388 -> 974c03a6c
  refs/heads/master d33276461 -> 00eab54f9


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/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/00eab54f/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/00eab54f/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/00eab54f/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 },
+        {}
+    });
+  }
+
+}


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

Posted by ro...@apache.org.
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 },
+        {}
+    });
+  }
+
+}


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

Posted by ro...@apache.org.
LUCENE-8196: Add IntervalQuery and IntervalsSource to the sandbox


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/974c03a6
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/974c03a6
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/974c03a6

Branch: refs/heads/branch_7x
Commit: 974c03a6ca8eed3941e1414dd2ecb75132228d4f
Parents: a149483
Author: Alan Woodward <ro...@apache.org>
Authored: Tue Apr 3 08:32:17 2018 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Tue Apr 3 09:38:36 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +
 .../search/intervals/ConjunctionDISI.java       | 134 +++++++
 .../intervals/ConjunctionIntervalIterator.java  |  72 ++++
 .../intervals/ConjunctionIntervalsSource.java   |  77 ++++
 .../intervals/DifferenceIntervalFunction.java   | 326 ++++++++++++++++
 .../intervals/DifferenceIntervalsSource.java    |  74 ++++
 .../search/intervals/DisiPriorityQueue.java     | 171 +++++++++
 .../lucene/search/intervals/DisiWrapper.java    |  44 +++
 .../intervals/DisjunctionDISIApproximation.java |  76 ++++
 .../intervals/DisjunctionIntervalsSource.java   | 228 ++++++++++++
 .../lucene/search/intervals/IntervalFilter.java |  86 +++++
 .../search/intervals/IntervalFunction.java      | 368 +++++++++++++++++++
 .../search/intervals/IntervalIterator.java      |  77 ++++
 .../lucene/search/intervals/IntervalQuery.java  | 153 ++++++++
 .../lucene/search/intervals/IntervalScorer.java |  99 +++++
 .../lucene/search/intervals/Intervals.java      | 188 ++++++++++
 .../search/intervals/IntervalsSource.java       |  61 +++
 .../intervals/LowpassIntervalsSource.java       |  71 ++++
 .../search/intervals/TermIntervalsSource.java   | 182 +++++++++
 .../lucene/search/intervals/package-info.java   |  66 ++++
 .../search/intervals/TestIntervalQuery.java     | 164 +++++++++
 .../lucene/search/intervals/TestIntervals.java  | 197 ++++++++++
 22 files changed, 2918 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 49b366a..385345e 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -21,6 +21,10 @@ New Features
 
 * LUCENE-8125: ICUTokenizer support for emoji/emoji sequence tokens. (Robert Muir)
 
+* LUCENE-8196: A new IntervalQuery in the sandbox allows efficient proximity
+  searches based on minimum-interval semantics. (Alan Woodward, Adrien Grand,
+  Jim Ferenczi, Simon Willnauer)
+
 Bug Fixes
 
 * LUCENE-8234: Fixed bug in how spatial relationship is computed for

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionDISI.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionDISI.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionDISI.java
new file mode 100644
index 0000000..ea1e096
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionDISI.java
@@ -0,0 +1,134 @@
+/*
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.CollectionUtil;
+
+/**
+ * A conjunction of DocIdSetIterators.
+ * This iterates over the doc ids that are present in each given DocIdSetIterator.
+ *
+ * @lucene.internal
+ */
+final class ConjunctionDISI extends DocIdSetIterator {
+
+  /**
+   * Create a conjunction over the provided DocIdSetIterators.
+   */
+  public static DocIdSetIterator intersectIterators(List<? extends DocIdSetIterator> iterators) {
+    if (iterators.size() < 2) {
+      throw new IllegalArgumentException("Cannot make a ConjunctionDISI of less than 2 iterators");
+    }
+    final List<DocIdSetIterator> allIterators = new ArrayList<>();
+    for (DocIdSetIterator iterator : iterators) {
+      addIterator(iterator, allIterators);
+    }
+
+    return new ConjunctionDISI(allIterators);
+  }
+
+  private static void addIterator(DocIdSetIterator disi, List<DocIdSetIterator> allIterators) {
+    if (disi.getClass() == ConjunctionDISI.class) { // Check for exactly this class for collapsing
+      ConjunctionDISI conjunction = (ConjunctionDISI) disi;
+      // subconjuctions have already split themselves into two phase iterators and others, so we can take those
+      // iterators as they are and move them up to this conjunction
+      allIterators.add(conjunction.lead1);
+      allIterators.add(conjunction.lead2);
+      Collections.addAll(allIterators, conjunction.others);
+    } else {
+      allIterators.add(disi);
+    }
+  }
+
+  final DocIdSetIterator lead1, lead2;
+  final DocIdSetIterator[] others;
+
+  private ConjunctionDISI(List<? extends DocIdSetIterator> iterators) {
+    assert iterators.size() >= 2;
+    // Sort the array the first time to allow the least frequent DocsEnum to
+    // lead the matching.
+    CollectionUtil.timSort(iterators, Comparator.comparingLong(DocIdSetIterator::cost));
+    lead1 = iterators.get(0);
+    lead2 = iterators.get(1);
+    others = iterators.subList(2, iterators.size()).toArray(new DocIdSetIterator[0]);
+  }
+
+  private int doNext(int doc) throws IOException {
+    advanceHead:
+    for (; ; ) {
+      assert doc == lead1.docID();
+
+      // find agreement between the two iterators with the lower costs
+      // we special case them because they do not need the
+      // 'other.docID() < doc' check that the 'others' iterators need
+      final int next2 = lead2.advance(doc);
+      if (next2 != doc) {
+        doc = lead1.advance(next2);
+        if (next2 != doc) {
+          continue;
+        }
+      }
+
+      // then find agreement with other iterators
+      for (DocIdSetIterator other : others) {
+        // other.doc may already be equal to doc if we "continued advanceHead"
+        // on the previous iteration and the advance on the lead scorer exactly matched.
+        if (other.docID() < doc) {
+          final int next = other.advance(doc);
+
+          if (next > doc) {
+            // iterator beyond the current doc - advance lead and continue to the new highest doc.
+            doc = lead1.advance(next);
+            continue advanceHead;
+          }
+        }
+      }
+
+      // success - all iterators are on the same doc
+      return doc;
+    }
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    return doNext(lead1.advance(target));
+  }
+
+  @Override
+  public int docID() {
+    return lead1.docID();
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    return doNext(lead1.nextDoc());
+  }
+
+  @Override
+  public long cost() {
+    return lead1.cost(); // overestimate
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalIterator.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalIterator.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalIterator.java
new file mode 100644
index 0000000..8efe4e4
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalIterator.java
@@ -0,0 +1,72 @@
+/*
+ * 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.List;
+
+import org.apache.lucene.search.DocIdSetIterator;
+
+abstract class ConjunctionIntervalIterator extends IntervalIterator {
+
+  final DocIdSetIterator approximation;
+  final List<IntervalIterator> subIterators;
+  final float cost;
+
+  ConjunctionIntervalIterator(List<IntervalIterator> subIterators) {
+    this.approximation = ConjunctionDISI.intersectIterators(subIterators);
+    this.subIterators = subIterators;
+    float costsum = 0;
+    for (IntervalIterator it : subIterators) {
+      costsum += it.matchCost();
+    }
+    this.cost = costsum;
+  }
+
+  @Override
+  public int docID() {
+    return approximation.docID();
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    int doc = approximation.nextDoc();
+    reset();
+    return doc;
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    int doc = approximation.advance(target);
+    reset();
+    return doc;
+  }
+
+  protected abstract void reset() throws IOException;
+
+  @Override
+  public long cost() {
+    return approximation.cost();
+  }
+
+  @Override
+  public final float matchCost() {
+    return cost;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalsSource.java
new file mode 100644
index 0000000..d2805c9
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalsSource.java
@@ -0,0 +1,77 @@
+/*
+ * 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.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+
+class ConjunctionIntervalsSource extends IntervalsSource {
+
+  final List<IntervalsSource> subSources;
+  final IntervalFunction function;
+
+  ConjunctionIntervalsSource(List<IntervalsSource> subSources, IntervalFunction function) {
+    this.subSources = subSources;
+    this.function = function;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    ConjunctionIntervalsSource that = (ConjunctionIntervalsSource) o;
+    return Objects.equals(subSources, that.subSources) &&
+        Objects.equals(function, that.function);
+  }
+
+  @Override
+  public String toString() {
+    return function + subSources.stream().map(Object::toString).collect(Collectors.joining(",", "(", ")"));
+  }
+
+  @Override
+  public void extractTerms(String field, Set<Term> terms) {
+    for (IntervalsSource source : subSources) {
+      source.extractTerms(field, terms);
+    }
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    List<IntervalIterator> subIntervals = new ArrayList<>();
+    for (IntervalsSource source : subSources) {
+      IntervalIterator it = source.intervals(field, ctx);
+      if (it == null)
+        return null;
+      subIntervals.add(it);
+    }
+    return function.apply(subIntervals);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(subSources, function);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalFunction.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalFunction.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalFunction.java
new file mode 100644
index 0000000..18d4d67
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalFunction.java
@@ -0,0 +1,326 @@
+/*
+ * 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;
+
+/**
+ * A function that takes two interval iterators and combines them to produce a third,
+ * generally by computing a difference interval between them
+ */
+abstract class DifferenceIntervalFunction {
+
+  @Override
+  public abstract int hashCode();
+
+  @Override
+  public abstract boolean equals(Object obj);
+
+  @Override
+  public abstract String toString();
+
+  /**
+   * Combine two interval iterators into a third
+   */
+  public abstract IntervalIterator apply(IntervalIterator minuend, IntervalIterator subtrahend);
+
+  /**
+   * Filters the minuend iterator so that only intervals that do not overlap intervals from the
+   * subtrahend iterator are returned
+   */
+  static final DifferenceIntervalFunction NON_OVERLAPPING = new SingletonFunction("NON_OVERLAPPING") {
+    @Override
+    public IntervalIterator apply(IntervalIterator minuend, IntervalIterator subtrahend) {
+      return new NonOverlappingIterator(minuend, subtrahend);
+    }
+  };
+
+  /**
+   * Filters the minuend iterator so that only intervals that do not contain intervals from the
+   * subtrahend iterator are returned
+   */
+  static final DifferenceIntervalFunction NOT_CONTAINING = new SingletonFunction("NOT_CONTAINING") {
+    @Override
+    public IntervalIterator apply(IntervalIterator minuend, IntervalIterator subtrahend) {
+      return new NotContainingIterator(minuend, subtrahend);
+    }
+  };
+
+  /**
+   * Filters the minuend iterator so that only intervals that are not contained by intervals from
+   * the subtrahend iterator are returned
+   */
+  static final DifferenceIntervalFunction NOT_CONTAINED_BY = new SingletonFunction("NOT_CONTAINED_BY") {
+    @Override
+    public IntervalIterator apply(IntervalIterator minuend, IntervalIterator subtrahend) {
+      return new NotContainedByIterator(minuend, subtrahend);
+    }
+  };
+
+  private static abstract class RelativeIterator extends IntervalIterator {
+
+    final IntervalIterator a;
+    final IntervalIterator b;
+
+    boolean bpos;
+
+    RelativeIterator(IntervalIterator a, IntervalIterator b) {
+      this.a = a;
+      this.b = b;
+    }
+
+    @Override
+    public int docID() {
+      return a.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      int doc = a.nextDoc();
+      reset();
+      return doc;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      int doc = a.advance(target);
+      reset();
+      return doc;
+    }
+
+    @Override
+    public long cost() {
+      return a.cost();
+    }
+
+    protected void reset() throws IOException {
+      int doc = a.docID();
+      bpos = b.docID() == doc ||
+          (b.docID() < doc && b.advance(doc) == doc);
+    }
+
+    @Override
+    public int start() {
+      return a.start();
+    }
+
+    @Override
+    public int end() {
+      return a.end();
+    }
+
+    @Override
+    public float matchCost() {
+      return a.matchCost() + b.matchCost();
+    }
+  }
+
+  private static class NonOverlappingIterator extends RelativeIterator {
+
+    private NonOverlappingIterator(IntervalIterator minuend, IntervalIterator subtrahend) {
+      super(minuend, subtrahend);
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      if (bpos == false)
+        return a.nextInterval();
+      while (a.nextInterval() != NO_MORE_INTERVALS) {
+        while (b.end() < a.start()) {
+          if (b.nextInterval() == NO_MORE_INTERVALS) {
+            bpos = false;
+            return a.start();
+          }
+        }
+        if (b.start() > a.end())
+          return a.start();
+      }
+      return NO_MORE_INTERVALS;
+    }
+  }
+
+  /**
+   * Filters the minuend iterator so that only intervals that do not occur within a set number
+   * of positions of intervals from the subtrahend iterator are returned
+   */
+  static class NotWithinFunction extends DifferenceIntervalFunction {
+
+    private final int positions;
+
+    NotWithinFunction(int positions) {
+      this.positions = positions;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      NotWithinFunction that = (NotWithinFunction) o;
+      return positions == that.positions;
+    }
+
+    @Override
+    public String toString() {
+      return "NOTWITHIN/" + positions;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(positions);
+    }
+
+    @Override
+    public IntervalIterator apply(IntervalIterator minuend, IntervalIterator subtrahend) {
+      IntervalIterator notWithin = new IntervalIterator() {
+
+        @Override
+        public int docID() {
+          return subtrahend.docID();
+        }
+
+        @Override
+        public int nextDoc() throws IOException {
+          positioned = false;
+          return subtrahend.nextDoc();
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+          positioned = false;
+          return subtrahend.advance(target);
+        }
+
+        @Override
+        public long cost() {
+          return subtrahend.cost();
+        }
+
+        boolean positioned = false;
+
+        @Override
+        public int start() {
+          if (positioned == false)
+            return -1;
+          int start = subtrahend.start();
+          return Math.max(0, start - positions);
+        }
+
+        @Override
+        public int end() {
+          if (positioned == false)
+            return -1;
+          int end = subtrahend.end();
+          int newEnd = end + positions;
+          if (newEnd < 0) // check for overflow
+            return Integer.MAX_VALUE;
+          return newEnd;
+        }
+
+        @Override
+        public int nextInterval() throws IOException {
+          if (positioned == false) {
+            positioned = true;
+          }
+          return subtrahend.nextInterval();
+        }
+
+        @Override
+        public float matchCost() {
+          return subtrahend.matchCost();
+        }
+
+      };
+      return NON_OVERLAPPING.apply(minuend, notWithin);
+    }
+  }
+
+  private static class NotContainingIterator extends RelativeIterator {
+
+    private NotContainingIterator(IntervalIterator minuend, IntervalIterator subtrahend) {
+      super(minuend, subtrahend);
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      if (bpos == false)
+        return a.nextInterval();
+      while (a.nextInterval() != NO_MORE_INTERVALS) {
+        while (b.start() < a.start() && b.end() < a.end()) {
+          if (b.nextInterval() == NO_MORE_INTERVALS) {
+            bpos = false;
+            return a.start();
+          }
+        }
+        if (b.start() > a.end())
+          return a.start();
+      }
+      return NO_MORE_INTERVALS;
+    }
+
+  }
+
+  private static class NotContainedByIterator extends RelativeIterator {
+
+    NotContainedByIterator(IntervalIterator a, IntervalIterator b) {
+      super(a, b);
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      if (bpos == false)
+        return a.nextInterval();
+      while (a.nextInterval() != NO_MORE_INTERVALS) {
+        while (b.end() < a.end()) {
+          if (b.nextInterval() == NO_MORE_INTERVALS)
+            return a.start();
+        }
+        if (a.start() < b.start())
+          return a.start();
+      }
+      return NO_MORE_INTERVALS;
+    }
+  }
+
+  private static abstract class SingletonFunction extends DifferenceIntervalFunction {
+
+    private final String name;
+
+    SingletonFunction(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public int hashCode() {
+      return System.identityHashCode(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return obj == this;
+    }
+
+    @Override
+    public String toString() {
+      return name;
+    }
+
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalsSource.java
new file mode 100644
index 0000000..316b6ff
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalsSource.java
@@ -0,0 +1,74 @@
+/*
+ * 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.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+
+class DifferenceIntervalsSource extends IntervalsSource {
+
+  final IntervalsSource minuend;
+  final IntervalsSource subtrahend;
+  final DifferenceIntervalFunction function;
+
+  public DifferenceIntervalsSource(IntervalsSource minuend, IntervalsSource subtrahend, DifferenceIntervalFunction function) {
+    this.minuend = minuend;
+    this.subtrahend = subtrahend;
+    this.function = function;
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    IntervalIterator minIt = minuend.intervals(field, ctx);
+    if (minIt == null)
+      return null;
+    IntervalIterator subIt = subtrahend.intervals(field, ctx);
+    if (subIt == null)
+      return minIt;
+    return function.apply(minIt, subIt);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    DifferenceIntervalsSource that = (DifferenceIntervalsSource) o;
+    return Objects.equals(minuend, that.minuend) &&
+        Objects.equals(subtrahend, that.subtrahend) &&
+        Objects.equals(function, that.function);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(minuend, subtrahend, function);
+  }
+
+  @Override
+  public String toString() {
+    return function + "(" + minuend + ", " + subtrahend + ")";
+  }
+
+  @Override
+  public void extractTerms(String field, Set<Term> terms) {
+    minuend.extractTerms(field, terms);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiPriorityQueue.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiPriorityQueue.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiPriorityQueue.java
new file mode 100644
index 0000000..0be96ab
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiPriorityQueue.java
@@ -0,0 +1,171 @@
+/*
+ * 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.util.Arrays;
+import java.util.Iterator;
+
+import org.apache.lucene.util.PriorityQueue;
+
+/**
+ * A priority queue of DocIdSetIterators that orders by current doc ID.
+ * This specialization is needed over {@link PriorityQueue} because the
+ * pluggable comparison function makes the rebalancing quite slow.
+ * @lucene.internal
+ */
+final class DisiPriorityQueue implements Iterable<DisiWrapper> {
+
+  static int leftNode(int node) {
+    return ((node + 1) << 1) - 1;
+  }
+
+  static int rightNode(int leftNode) {
+    return leftNode + 1;
+  }
+
+  static int parentNode(int node) {
+    return ((node + 1) >>> 1) - 1;
+  }
+
+  private final DisiWrapper[] heap;
+  private int size;
+
+  public DisiPriorityQueue(int maxSize) {
+    heap = new DisiWrapper[maxSize];
+    size = 0;
+  }
+
+  public int size() {
+    return size;
+  }
+
+  public DisiWrapper top() {
+    return heap[0];
+  }
+
+  /** Get the list of scorers which are on the current doc. */
+  public DisiWrapper topList() {
+    final DisiWrapper[] heap = this.heap;
+    final int size = this.size;
+    DisiWrapper list = heap[0];
+    list.next = null;
+    if (size >= 3) {
+      list = topList(list, heap, size, 1);
+      list = topList(list, heap, size, 2);
+    } else if (size == 2 && heap[1].doc == list.doc) {
+      list = prepend(heap[1], list);
+    }
+    return list;
+  }
+
+  // prepend w1 (iterator) to w2 (list)
+  private DisiWrapper prepend(DisiWrapper w1, DisiWrapper w2) {
+    w1.next = w2;
+    return w1;
+  }
+
+  private DisiWrapper topList(DisiWrapper list, DisiWrapper[] heap,
+                              int size, int i) {
+    final DisiWrapper w = heap[i];
+    if (w.doc == list.doc) {
+      list = prepend(w, list);
+      final int left = leftNode(i);
+      final int right = left + 1;
+      if (right < size) {
+        list = topList(list, heap, size, left);
+        list = topList(list, heap, size, right);
+      } else if (left < size && heap[left].doc == list.doc) {
+        list = prepend(heap[left], list);
+      }
+    }
+    return list;
+  }
+
+  public DisiWrapper add(DisiWrapper entry) {
+    final DisiWrapper[] heap = this.heap;
+    final int size = this.size;
+    heap[size] = entry;
+    upHeap(size);
+    this.size = size + 1;
+    return heap[0];
+  }
+
+  public DisiWrapper pop() {
+    final DisiWrapper[] heap = this.heap;
+    final DisiWrapper result = heap[0];
+    final int i = --size;
+    heap[0] = heap[i];
+    heap[i] = null;
+    downHeap(i);
+    return result;
+  }
+
+  public DisiWrapper updateTop() {
+    downHeap(size);
+    return heap[0];
+  }
+
+  DisiWrapper updateTop(DisiWrapper topReplacement) {
+    heap[0] = topReplacement;
+    return updateTop();
+  }
+
+  void upHeap(int i) {
+    final DisiWrapper node = heap[i];
+    final int nodeDoc = node.doc;
+    int j = parentNode(i);
+    while (j >= 0 && nodeDoc < heap[j].doc) {
+      heap[i] = heap[j];
+      i = j;
+      j = parentNode(j);
+    }
+    heap[i] = node;
+  }
+
+  void downHeap(int size) {
+    int i = 0;
+    final DisiWrapper node = heap[0];
+    int j = leftNode(i);
+    if (j < size) {
+      int k = rightNode(j);
+      if (k < size && heap[k].doc < heap[j].doc) {
+        j = k;
+      }
+      if (heap[j].doc < node.doc) {
+        do {
+          heap[i] = heap[j];
+          i = j;
+          j = leftNode(i);
+          k = rightNode(j);
+          if (k < size && heap[k].doc < heap[j].doc) {
+            j = k;
+          }
+        } while (j < size && heap[j].doc < node.doc);
+        heap[i] = node;
+      }
+    }
+  }
+
+  @Override
+  public Iterator<DisiWrapper> iterator() {
+    return Arrays.asList(heap).subList(0, size).iterator();
+  }
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiWrapper.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiWrapper.java
new file mode 100644
index 0000000..483c626
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiWrapper.java
@@ -0,0 +1,44 @@
+/*
+ * 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 org.apache.lucene.search.DocIdSetIterator;
+
+class DisiWrapper {
+
+  public final DocIdSetIterator iterator;
+  public final IntervalIterator intervals;
+  public final long cost;
+  public final float matchCost; // the match cost for two-phase iterators, 0 otherwise
+  public int doc; // the current doc, used for comparison
+  public DisiWrapper next; // reference to a next element, see #topList
+
+  // An approximation of the iterator, or the iterator itself if it does not
+  // support two-phase iteration
+  public final DocIdSetIterator approximation;
+
+  public DisiWrapper(IntervalIterator iterator) {
+    this.intervals = iterator;
+    this.iterator = iterator;
+    this.cost = iterator.cost();
+    this.doc = -1;
+    this.approximation = iterator;
+    this.matchCost = iterator.matchCost();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionDISIApproximation.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionDISIApproximation.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionDISIApproximation.java
new file mode 100644
index 0000000..d4e8e79
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionDISIApproximation.java
@@ -0,0 +1,76 @@
+/*
+ * 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.search.DocIdSetIterator;
+
+/**
+ * A {@link DocIdSetIterator} which is a disjunction of the approximations of
+ * the provided iterators.
+ * @lucene.internal
+ */
+class DisjunctionDISIApproximation extends DocIdSetIterator {
+
+  final DisiPriorityQueue subIterators;
+  final long cost;
+
+  public DisjunctionDISIApproximation(DisiPriorityQueue subIterators) {
+    this.subIterators = subIterators;
+    long cost = 0;
+    for (DisiWrapper w : subIterators) {
+      cost += w.cost;
+    }
+    this.cost = cost;
+  }
+
+  @Override
+  public long cost() {
+    return cost;
+  }
+
+  @Override
+  public int docID() {
+    return subIterators.top().doc;
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    DisiWrapper top = subIterators.top();
+    final int doc = top.doc;
+    do {
+      top.doc = top.approximation.nextDoc();
+      top = subIterators.updateTop();
+    } while (top.doc == doc);
+
+    return top.doc;
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    DisiWrapper top = subIterators.top();
+    do {
+      top.doc = top.approximation.advance(target);
+      top = subIterators.updateTop();
+    } while (top.doc < target);
+
+    return top.doc;
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionIntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionIntervalsSource.java
new file mode 100644
index 0000000..c7c7166
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionIntervalsSource.java
@@ -0,0 +1,228 @@
+/*
+ * 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.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.PriorityQueue;
+
+class DisjunctionIntervalsSource extends IntervalsSource {
+
+  final List<IntervalsSource> subSources;
+
+  public DisjunctionIntervalsSource(List<IntervalsSource> subSources) {
+    this.subSources = subSources;
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    List<IntervalIterator> subIterators = new ArrayList<>();
+    for (IntervalsSource subSource : subSources) {
+      IntervalIterator it = subSource.intervals(field, ctx);
+      if (it != null) {
+        subIterators.add(it);
+      }
+    }
+    if (subIterators.size() == 0)
+      return null;
+    return new DisjunctionIntervalIterator(subIterators);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    DisjunctionIntervalsSource that = (DisjunctionIntervalsSource) o;
+    return Objects.equals(subSources, that.subSources);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(subSources);
+  }
+
+  @Override
+  public String toString() {
+    return subSources.stream().map(Object::toString).collect(Collectors.joining(",", "or(", ")"));
+  }
+
+  @Override
+  public void extractTerms(String field, Set<Term> terms) {
+    for (IntervalsSource source : subSources) {
+      source.extractTerms(field, terms);
+    }
+  }
+
+  private static class DisjunctionIntervalIterator extends IntervalIterator {
+
+    final DocIdSetIterator approximation;
+    final PriorityQueue<IntervalIterator> intervalQueue;
+    final DisiPriorityQueue disiQueue;
+    final List<IntervalIterator> iterators;
+    final float matchCost;
+
+    IntervalIterator current = EMPTY;
+
+    DisjunctionIntervalIterator(List<IntervalIterator> iterators) {
+      this.disiQueue = new DisiPriorityQueue(iterators.size());
+      for (IntervalIterator it : iterators) {
+        disiQueue.add(new DisiWrapper(it));
+      }
+      this.approximation = new DisjunctionDISIApproximation(disiQueue);
+      this.iterators = iterators;
+      this.intervalQueue = new PriorityQueue<IntervalIterator>(iterators.size()) {
+        @Override
+        protected boolean lessThan(IntervalIterator a, IntervalIterator b) {
+          return a.end() < b.end() || (a.end() == b.end() && a.start() >= b.start());
+        }
+      };
+      float costsum = 0;
+      for (IntervalIterator it : iterators) {
+        costsum += it.cost();
+      }
+      this.matchCost = costsum;
+    }
+
+    @Override
+    public float matchCost() {
+      return matchCost;
+    }
+
+    @Override
+    public int start() {
+      return current.start();
+    }
+
+    @Override
+    public int end() {
+      return current.end();
+    }
+
+    private void reset() throws IOException {
+      intervalQueue.clear();
+      for (DisiWrapper dw = disiQueue.topList(); dw != null; dw = dw.next) {
+        dw.intervals.nextInterval();
+        intervalQueue.add(dw.intervals);
+      }
+      current = EMPTY;
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      if (current == EMPTY) {
+        if (intervalQueue.size() > 0) {
+          current = intervalQueue.top();
+        }
+        return current.start();
+      }
+      int start = current.start(), end = current.end();
+      while (intervalQueue.size() > 0 && contains(intervalQueue.top(), start, end)) {
+        IntervalIterator it = intervalQueue.pop();
+        if (it != null && it.nextInterval() != NO_MORE_INTERVALS) {
+          intervalQueue.add(it);
+        }
+      }
+      if (intervalQueue.size() == 0) {
+        current = EMPTY;
+        return NO_MORE_INTERVALS;
+      }
+      current = intervalQueue.top();
+      return current.start();
+    }
+
+    private boolean contains(IntervalIterator it, int start, int end) {
+      return start >= it.start() && start <= it.end() && end >= it.start() && end <= it.end();
+    }
+
+    @Override
+    public int docID() {
+      return approximation.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      int doc = approximation.nextDoc();
+      reset();
+      return doc;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      int doc = approximation.advance(target);
+      reset();
+      return doc;
+    }
+
+    @Override
+    public long cost() {
+      return approximation.cost();
+    }
+  }
+
+  private static final IntervalIterator EMPTY = new IntervalIterator() {
+
+    @Override
+    public int docID() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long cost() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int start() {
+      return -1;
+    }
+
+    @Override
+    public int end() {
+      return -1;
+    }
+
+    @Override
+    public int nextInterval() {
+      return NO_MORE_INTERVALS;
+    }
+
+    @Override
+    public float matchCost() {
+      return 0;
+    }
+  };
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFilter.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFilter.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFilter.java
new file mode 100644
index 0000000..d1d2fcf
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFilter.java
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+/**
+ * Wraps an {@link IntervalIterator} and passes through those intervals that match the {@link #accept()} function
+ */
+public abstract class IntervalFilter extends IntervalIterator {
+
+  private final IntervalIterator in;
+
+  /**
+   * Create a new filter
+   */
+  public IntervalFilter(IntervalIterator in) {
+    this.in = in;
+  }
+
+  @Override
+  public int docID() {
+    return in.docID();
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    return in.nextDoc();
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    return in.advance(target);
+  }
+
+  @Override
+  public long cost() {
+    return in.cost();
+  }
+
+  @Override
+  public int start() {
+    return in.start();
+  }
+
+  @Override
+  public int end() {
+    return in.end();
+  }
+
+  @Override
+  public float matchCost() {
+    return in.matchCost();
+  }
+
+  /**
+   * @return {@code true} if the wrapped iterator's interval should be passed on
+   */
+  protected abstract boolean accept();
+
+  @Override
+  public final int nextInterval() throws IOException {
+    int next;
+    do {
+      next = in.nextInterval();
+    }
+    while (accept() == false && next != IntervalIterator.NO_MORE_INTERVALS);
+    return next;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFunction.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFunction.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFunction.java
new file mode 100644
index 0000000..c550a72
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFunction.java
@@ -0,0 +1,368 @@
+/*
+ * 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.List;
+
+import org.apache.lucene.util.PriorityQueue;
+
+/**
+ * Combine a list of {@link IntervalIterator}s into another
+ */
+abstract class IntervalFunction {
+
+  @Override
+  public abstract int hashCode();
+
+  @Override
+  public abstract boolean equals(Object obj);
+
+  @Override
+  public abstract String toString();
+
+  /**
+   * Combine the iterators into another iterator
+   */
+  public abstract IntervalIterator apply(List<IntervalIterator> iterators);
+
+  static final IntervalFunction BLOCK = new SingletonFunction("BLOCK") {
+    @Override
+    public IntervalIterator apply(List<IntervalIterator> iterators) {
+      return new BlockIntervalIterator(iterators);
+    }
+  };
+
+  private static class BlockIntervalIterator extends ConjunctionIntervalIterator {
+
+    int start = -1, end = -1;
+
+    BlockIntervalIterator(List<IntervalIterator> subIterators) {
+      super(subIterators);
+    }
+
+    @Override
+    public int start() {
+      return start;
+    }
+
+    @Override
+    public int end() {
+      return end;
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      if (subIterators.get(0).nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
+        return IntervalIterator.NO_MORE_INTERVALS;
+      int i = 1;
+      while (i < subIterators.size()) {
+        while (subIterators.get(i).start() <= subIterators.get(i - 1).end()) {
+          if (subIterators.get(i).nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
+            return IntervalIterator.NO_MORE_INTERVALS;
+        }
+        if (subIterators.get(i).start() == subIterators.get(i - 1).end() + 1) {
+          i = i + 1;
+        }
+        else {
+          if (subIterators.get(0).nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
+            return IntervalIterator.NO_MORE_INTERVALS;
+          i = 1;
+        }
+      }
+      start = subIterators.get(0).start();
+      end = subIterators.get(subIterators.size() - 1).end();
+      return start;
+    }
+
+    @Override
+    protected void reset() {
+      start = end = -1;
+    }
+  }
+
+  /**
+   * Return an iterator over intervals where the subiterators appear in a given order
+   */
+  static final IntervalFunction ORDERED = new SingletonFunction("ORDERED") {
+    @Override
+    public IntervalIterator apply(List<IntervalIterator> intervalIterators) {
+      return new OrderedIntervalIterator(intervalIterators);
+    }
+  };
+
+  private static class OrderedIntervalIterator extends ConjunctionIntervalIterator {
+
+    int start = -1, end = -1, i;
+
+    private OrderedIntervalIterator(List<IntervalIterator> subIntervals) {
+      super(subIntervals);
+    }
+
+    @Override
+    public int start() {
+      return start;
+    }
+
+    @Override
+    public int end() {
+      return end;
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      start = end = IntervalIterator.NO_MORE_INTERVALS;
+      int b = Integer.MAX_VALUE;
+      i = 1;
+      while (true) {
+        while (true) {
+          if (subIterators.get(i - 1).end() >= b)
+            return start;
+          if (i == subIterators.size() || subIterators.get(i).start() > subIterators.get(i - 1).end())
+            break;
+          do {
+            if (subIterators.get(i).end() >= b || subIterators.get(i).nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
+              return start;
+          }
+          while (subIterators.get(i).start() <= subIterators.get(i - 1).end());
+          i++;
+        }
+        start = subIterators.get(0).start();
+        end = subIterators.get(subIterators.size() - 1).end();
+        b = subIterators.get(subIterators.size() - 1).start();
+        i = 1;
+        if (subIterators.get(0).nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
+          return start;
+      }
+    }
+
+    @Override
+    protected void reset() throws IOException {
+      subIterators.get(0).nextInterval();
+      i = 1;
+      start = end = -1;
+    }
+  }
+
+  /**
+   * Return an iterator over intervals where the subiterators appear in any order
+   */
+  static final IntervalFunction UNORDERED = new SingletonFunction("UNORDERED") {
+    @Override
+    public IntervalIterator apply(List<IntervalIterator> intervalIterators) {
+      return new UnorderedIntervalIterator(intervalIterators);
+    }
+  };
+
+  private static class UnorderedIntervalIterator extends ConjunctionIntervalIterator {
+
+    private final PriorityQueue<IntervalIterator> queue;
+    private final IntervalIterator[] subIterators;
+
+    int start = -1, end = -1, queueEnd;
+
+    UnorderedIntervalIterator(List<IntervalIterator> subIterators) {
+      super(subIterators);
+      this.queue = new PriorityQueue<IntervalIterator>(subIterators.size()) {
+        @Override
+        protected boolean lessThan(IntervalIterator a, IntervalIterator b) {
+          return a.start() < b.start() || (a.start() == b.start() && a.end() >= b.end());
+        }
+      };
+      this.subIterators = new IntervalIterator[subIterators.size()];
+
+      for (int i = 0; i < subIterators.size(); i++) {
+        this.subIterators[i] = subIterators.get(i);
+      }
+    }
+
+    @Override
+    public int start() {
+      return start;
+    }
+
+    @Override
+    public int end() {
+      return end;
+    }
+
+    void updateRightExtreme(IntervalIterator it) {
+      int itEnd = it.end();
+      if (itEnd > queueEnd) {
+        queueEnd = itEnd;
+      }
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      while (this.queue.size() == subIterators.length && queue.top().start() == start) {
+        IntervalIterator it = queue.pop();
+        if (it != null && it.nextInterval() != IntervalIterator.NO_MORE_INTERVALS) {
+          queue.add(it);
+          updateRightExtreme(it);
+        }
+      }
+      if (this.queue.size() < subIterators.length)
+        return IntervalIterator.NO_MORE_INTERVALS;
+      do {
+        start = queue.top().start();
+        end = queueEnd;
+        if (queue.top().end() == end)
+          return start;
+        IntervalIterator it = queue.pop();
+        if (it != null && it.nextInterval() != IntervalIterator.NO_MORE_INTERVALS) {
+          queue.add(it);
+          updateRightExtreme(it);
+        }
+      } while (this.queue.size() == subIterators.length && end == queueEnd);
+      return start;
+    }
+
+    @Override
+    protected void reset() throws IOException {
+      queueEnd = start = end = -1;
+      this.queue.clear();
+      for (IntervalIterator it : subIterators) {
+        it.nextInterval();
+        queue.add(it);
+        updateRightExtreme(it);
+      }
+    }
+
+  }
+
+  /**
+   * Returns an interval over iterators where the first iterator contains intervals from the second
+   */
+  static final IntervalFunction CONTAINING = new SingletonFunction("CONTAINING") {
+    @Override
+    public IntervalIterator apply(List<IntervalIterator> iterators) {
+      if (iterators.size() != 2)
+        throw new IllegalStateException("CONTAINING function requires two iterators");
+      IntervalIterator a = iterators.get(0);
+      IntervalIterator b = iterators.get(1);
+      return new ConjunctionIntervalIterator(iterators) {
+
+        boolean bpos;
+
+        @Override
+        public int start() {
+          return a.start();
+        }
+
+        @Override
+        public int end() {
+          return a.end();
+        }
+
+        @Override
+        public int nextInterval() throws IOException {
+          if (bpos == false)
+            return IntervalIterator.NO_MORE_INTERVALS;
+          while (a.nextInterval() != IntervalIterator.NO_MORE_INTERVALS) {
+            while (b.start() < a.start() && b.end() < a.end()) {
+              if (b.nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
+                return IntervalIterator.NO_MORE_INTERVALS;
+            }
+            if (a.start() <= b.start() && a.end() >= b.end())
+              return a.start();
+          }
+          return IntervalIterator.NO_MORE_INTERVALS;
+        }
+
+        @Override
+        protected void reset() throws IOException {
+          bpos = true;
+        }
+      };
+    }
+  };
+
+  /**
+   * Return an iterator over intervals where the first iterator is contained by intervals from the second
+   */
+  static final IntervalFunction CONTAINED_BY = new SingletonFunction("CONTAINED_BY") {
+    @Override
+    public IntervalIterator apply(List<IntervalIterator> iterators) {
+      if (iterators.size() != 2)
+        throw new IllegalStateException("CONTAINED_BY function requires two iterators");
+      IntervalIterator a = iterators.get(0);
+      IntervalIterator b = iterators.get(1);
+      return new ConjunctionIntervalIterator(iterators) {
+
+        boolean bpos;
+
+        @Override
+        public int start() {
+          return a.start();
+        }
+
+        @Override
+        public int end() {
+          return a.end();
+        }
+
+        @Override
+        public int nextInterval() throws IOException {
+          if (bpos == false)
+            return IntervalIterator.NO_MORE_INTERVALS;
+          while (a.nextInterval() != IntervalIterator.NO_MORE_INTERVALS) {
+            while (b.end() < a.end()) {
+              if (b.nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
+                return IntervalIterator.NO_MORE_INTERVALS;
+            }
+            if (b.start() <= a.start())
+              return a.start();
+          }
+          return IntervalIterator.NO_MORE_INTERVALS;
+        }
+
+        @Override
+        protected void reset() throws IOException {
+          bpos = true;
+        }
+      };
+    }
+  };
+
+  private static abstract class SingletonFunction extends IntervalFunction {
+
+    private final String name;
+
+    protected SingletonFunction(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public int hashCode() {
+      return System.identityHashCode(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return obj == this;
+    }
+
+    @Override
+    public String toString() {
+      return name;
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalIterator.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalIterator.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalIterator.java
new file mode 100644
index 0000000..2428720
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalIterator.java
@@ -0,0 +1,77 @@
+/*
+ * 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.search.DocIdSetIterator;
+import org.apache.lucene.search.TwoPhaseIterator;
+
+/**
+ * A {@link DocIdSetIterator} that also allows iteration over matching
+ * intervals in a document.
+ *
+ * Once the iterator is positioned on a document by calling {@link #advance(int)}
+ * or {@link #nextDoc()}, intervals may be retrieved by calling {@link #nextInterval()}
+ * until {@link #NO_MORE_INTERVALS} is returned.
+ *
+ * The limits of the current interval are returned by {@link #start()} and {@link #end()}.
+ * When the iterator has been moved to a new document, but before {@link #nextInterval()}
+ * has been called, both these methods return {@code -1}.
+ *
+ * Note that it is possible for a document to return {@link #NO_MORE_INTERVALS}
+ * on the first call to {@link #nextInterval()}
+ */
+public abstract class IntervalIterator extends DocIdSetIterator {
+
+  /**
+   * When returned from {@link #nextInterval()}, indicates that there are no more
+   * matching intervals on the current document
+   */
+  public static final int NO_MORE_INTERVALS = Integer.MAX_VALUE;
+
+  /**
+   * The start of the current interval
+   *
+   * Returns -1 if {@link #nextInterval()} has not yet been called
+   */
+  public abstract int start();
+
+  /**
+   * The end of the current interval
+   *
+   * Returns -1 if {@link #nextInterval()} has not yet been called
+   */
+  public abstract int end();
+
+  /**
+   * Advance the iterator to the next interval
+   *
+   * @return the start of the next interval, or {@link IntervalIterator#NO_MORE_INTERVALS} if
+   *         there are no more intervals on the current document
+   */
+  public abstract int nextInterval() throws IOException;
+
+  /**
+   * An indication of the average cost of iterating over all intervals in a document
+   *
+   * @see TwoPhaseIterator#matchCost()
+   */
+  public abstract float matchCost();
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalQuery.java
new file mode 100644
index 0000000..cd1dca1
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalQuery.java
@@ -0,0 +1,153 @@
+/*
+ * 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.Arrays;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermContext;
+import org.apache.lucene.search.CollectionStatistics;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TermStatistics;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.search.similarities.Similarity;
+
+/**
+ * A query that retrieves documents containing intervals returned from an
+ * {@link IntervalsSource}
+ *
+ * Static constructor functions for various different sources can be found in the
+ * {@link Intervals} class
+ */
+public final class IntervalQuery extends Query {
+
+  private final String field;
+  private final IntervalsSource intervalsSource;
+
+  /**
+   * Create a new IntervalQuery
+   * @param field             the field to query
+   * @param intervalsSource   an {@link IntervalsSource} to retrieve intervals from
+   */
+  public IntervalQuery(String field, IntervalsSource intervalsSource) {
+    this.field = field;
+    this.intervalsSource = intervalsSource;
+  }
+
+  public String getField() {
+    return field;
+  }
+
+  @Override
+  public String toString(String field) {
+    return intervalsSource.toString();
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
+    return new IntervalWeight(this, needsScores ? buildSimScorer(searcher, needsScores, boost) : null,
+        searcher.getSimilarity(needsScores), needsScores);
+  }
+
+  private Similarity.SimWeight buildSimScorer(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
+    Set<Term> terms = new HashSet<>();
+    intervalsSource.extractTerms(field, terms);
+    TermStatistics[] termStats = new TermStatistics[terms.size()];
+    int termUpTo = 0;
+    for (Term term : terms) {
+      TermStatistics termStatistics = searcher.termStatistics(term, TermContext.build(searcher.getTopReaderContext(), term));
+      if (termStatistics != null) {
+        termStats[termUpTo++] = termStatistics;
+      }
+    }
+    if (termUpTo == 0) {
+      return null;
+    }
+    CollectionStatistics collectionStats = searcher.collectionStatistics(field);
+    return searcher.getSimilarity(needsScores).computeWeight(boost, collectionStats, Arrays.copyOf(termStats, termUpTo));
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    IntervalQuery that = (IntervalQuery) o;
+    return Objects.equals(field, that.field) &&
+        Objects.equals(intervalsSource, that.intervalsSource);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(field, intervalsSource);
+  }
+
+  private class IntervalWeight extends Weight {
+
+    final Similarity.SimWeight simWeight;
+    final Similarity similarity;
+    final boolean needsScores;
+
+    public IntervalWeight(Query query, Similarity.SimWeight simWeight, Similarity similarity, boolean needsScores) {
+      super(query);
+      this.simWeight = simWeight;
+      this.similarity = similarity;
+      this.needsScores = needsScores;
+    }
+
+    @Override
+    public void extractTerms(Set<Term> terms) {
+      intervalsSource.extractTerms(field, terms);
+    }
+
+    @Override
+    public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+      IntervalScorer scorer = (IntervalScorer) scorer(context);
+      if (scorer != null) {
+        int newDoc = scorer.iterator().advance(doc);
+        if (newDoc == doc) {
+          return scorer.explain("weight("+getQuery()+" in "+doc+") [" + similarity.getClass().getSimpleName() + "]");
+        }
+      }
+      return Explanation.noMatch("no matching intervals");
+    }
+
+    @Override
+    public Scorer scorer(LeafReaderContext context) throws IOException {
+      IntervalIterator intervals = intervalsSource.intervals(field, context);
+      if (intervals == null)
+        return null;
+      Similarity.SimScorer leafScorer = simWeight == null ? null
+          : similarity.simScorer(simWeight, context);
+      return new IntervalScorer(this, intervals, leafScorer);
+    }
+
+    @Override
+    public boolean isCacheable(LeafReaderContext ctx) {
+      return true;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalScorer.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalScorer.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalScorer.java
new file mode 100644
index 0000000..8c3c5f2
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalScorer.java
@@ -0,0 +1,99 @@
+/*
+ * 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.search.DocIdSetIterator;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TwoPhaseIterator;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.search.similarities.Similarity;
+
+class IntervalScorer extends Scorer {
+
+  private final IntervalIterator intervals;
+  private final Similarity.SimScorer simScorer;
+
+  private float freq = -1;
+  private int lastScoredDoc = -1;
+
+  protected IntervalScorer(Weight weight, IntervalIterator intervals, Similarity.SimScorer simScorer) {
+    super(weight);
+    this.intervals = intervals;
+    this.simScorer = simScorer;
+  }
+
+  @Override
+  public int docID() {
+    return intervals.docID();
+  }
+
+  @Override
+  public float score() throws IOException {
+    ensureFreq();
+    return simScorer.score(docID(), freq);
+  }
+
+  public Explanation explain(String topLevel) throws IOException {
+    ensureFreq();
+    Explanation freqExplanation = Explanation.match(freq, "intervalFreq=" + freq);
+    Explanation scoreExplanation = simScorer.explain(docID(), freqExplanation);
+    return Explanation.match(scoreExplanation.getValue(),
+        topLevel + ", result of:",
+        scoreExplanation);
+  }
+
+  public float freq() throws IOException {
+    ensureFreq();
+    return freq;
+  }
+
+  private void ensureFreq() throws IOException {
+    if (lastScoredDoc != docID()) {
+      lastScoredDoc = docID();
+      freq = 0;
+      do {
+        freq += (1.0 / (intervals.end() - intervals.start() + 1));
+      }
+      while (intervals.nextInterval() != IntervalIterator.NO_MORE_INTERVALS);
+    }
+  }
+
+  @Override
+  public DocIdSetIterator iterator() {
+    return TwoPhaseIterator.asDocIdSetIterator(twoPhaseIterator());
+  }
+
+  @Override
+  public TwoPhaseIterator twoPhaseIterator() {
+    return new TwoPhaseIterator(intervals) {
+      @Override
+      public boolean matches() throws IOException {
+        return intervals.nextInterval() != IntervalIterator.NO_MORE_INTERVALS;
+      }
+
+      @Override
+      public float matchCost() {
+        return intervals.matchCost();
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/Intervals.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/Intervals.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/Intervals.java
new file mode 100644
index 0000000..b360919
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/Intervals.java
@@ -0,0 +1,188 @@
+/*
+ * 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.util.Arrays;
+
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Constructor functions for {@link IntervalsSource} types
+ *
+ * These sources implement minimum-interval algorithms taken from the paper
+ * <a href="http://vigna.di.unimi.it/ftp/papers/EfficientAlgorithmsMinimalIntervalSemantics.pdf">
+ * Efficient Optimally Lazy Algorithms for Minimal-Interval Semantics</a>
+ */
+public final class Intervals {
+
+  private Intervals() {}
+
+  /**
+   * Return an {@link IntervalsSource} exposing intervals for a term
+   */
+  public static IntervalsSource term(BytesRef term) {
+    return new TermIntervalsSource(term);
+  }
+
+  /**
+   * Return an {@link IntervalsSource} exposing intervals for a term
+   */
+  public static IntervalsSource term(String term) {
+    return new TermIntervalsSource(new BytesRef(term));
+  }
+
+  /**
+   * Return an {@link IntervalsSource} exposing intervals for a phrase consisting of a list of terms
+   */
+  public static IntervalsSource phrase(String... terms) {
+    IntervalsSource[] sources = new IntervalsSource[terms.length];
+    int i = 0;
+    for (String term : terms) {
+      sources[i] = term(term);
+      i++;
+    }
+    return phrase(sources);
+  }
+
+  /**
+   * Return an {@link IntervalsSource} exposing intervals for a phrase consisting of a list of IntervalsSources
+   */
+  public static IntervalsSource phrase(IntervalsSource... subSources) {
+    return new ConjunctionIntervalsSource(Arrays.asList(subSources), IntervalFunction.BLOCK);
+  }
+
+  /**
+   * Return an {@link IntervalsSource} over the disjunction of a set of sub-sources
+   */
+  public static IntervalsSource or(IntervalsSource... subSources) {
+    if (subSources.length == 1)
+      return subSources[0];
+    return new DisjunctionIntervalsSource(Arrays.asList(subSources));
+  }
+
+  /**
+   * Create an {@link IntervalsSource} that filters a sub-source by the width of its intervals
+   * @param width       the maximum width of intervals in the sub-source ot return
+   * @param subSource   the sub-source to filter
+   */
+  public static IntervalsSource maxwidth(int width, IntervalsSource subSource) {
+    return new LowpassIntervalsSource(subSource, width);
+  }
+
+  /**
+   * Create an ordered {@link IntervalsSource} with an unbounded width range
+   *
+   * Returns intervals in which the subsources all appear in the given order
+   *
+   * @param subSources  an ordered set of {@link IntervalsSource} objects
+   */
+  public static IntervalsSource ordered(IntervalsSource... subSources) {
+    return new ConjunctionIntervalsSource(Arrays.asList(subSources), IntervalFunction.ORDERED);
+  }
+
+  /**
+   * Create an unordered {@link IntervalsSource} with an unbounded width range
+   *
+   * Returns intervals in which all the subsources appear.
+   *
+   * @param subSources  an unordered set of queries
+   */
+  public static IntervalsSource unordered(IntervalsSource... subSources) {
+    return new ConjunctionIntervalsSource(Arrays.asList(subSources), IntervalFunction.UNORDERED);
+  }
+
+  /**
+   * Create a non-overlapping IntervalsSource
+   *
+   * Returns intervals of the minuend that do not overlap with intervals from the subtrahend
+
+   * @param minuend     the {@link IntervalsSource} to filter
+   * @param subtrahend  the {@link IntervalsSource} to filter by
+   */
+  public static IntervalsSource nonOverlapping(IntervalsSource minuend, IntervalsSource subtrahend) {
+    return new DifferenceIntervalsSource(minuend, subtrahend, DifferenceIntervalFunction.NON_OVERLAPPING);
+  }
+
+  /**
+   * Create a not-within {@link IntervalsSource}
+   *
+   * Returns intervals of the minuend that do not appear within a set number of positions of
+   * intervals from the subtrahend query
+   *
+   * @param minuend     the {@link IntervalsSource} to filter
+   * @param positions   the maximum distance that intervals from the minuend may occur from intervals
+   *                    of the subtrahend
+   * @param subtrahend  the {@link IntervalsSource} to filter by
+   */
+  public static IntervalsSource notWithin(IntervalsSource minuend, int positions, IntervalsSource subtrahend) {
+    return new DifferenceIntervalsSource(minuend, subtrahend, new DifferenceIntervalFunction.NotWithinFunction(positions));
+  }
+
+  /**
+   * Create a not-containing {@link IntervalsSource}
+   *
+   * Returns intervals from the minuend that do not contain intervals of the subtrahend
+   *
+   * @param minuend     the {@link IntervalsSource} to filter
+   * @param subtrahend  the {@link IntervalsSource} to filter by
+   */
+  public static IntervalsSource notContaining(IntervalsSource minuend, IntervalsSource subtrahend) {
+    return new DifferenceIntervalsSource(minuend, subtrahend, DifferenceIntervalFunction.NOT_CONTAINING);
+  }
+
+  /**
+   * Create a containing {@link IntervalsSource}
+   *
+   * Returns intervals from the big source that contain one or more intervals from
+   * the small source
+   *
+   * @param big     the {@link IntervalsSource} to filter
+   * @param small   the {@link IntervalsSource} to filter by
+   */
+  public static IntervalsSource containing(IntervalsSource big, IntervalsSource small) {
+    return new ConjunctionIntervalsSource(Arrays.asList(big, small), IntervalFunction.CONTAINING);
+  }
+
+  /**
+   * Create a not-contained-by {@link IntervalsSource}
+   *
+   * Returns intervals from the small {@link IntervalsSource} that do not appear within
+   * intervals from the big {@link IntervalsSource}.
+   *
+   * @param small   the {@link IntervalsSource} to filter
+   * @param big     the {@link IntervalsSource} to filter by
+   */
+  public static IntervalsSource notContainedBy(IntervalsSource small, IntervalsSource big) {
+    return new DifferenceIntervalsSource(small, big, DifferenceIntervalFunction.NOT_CONTAINED_BY);
+  }
+
+  /**
+   * Create a contained-by {@link IntervalsSource}
+   *
+   * Returns intervals from the small query that appear within intervals of the big query
+   *
+   * @param small     the {@link IntervalsSource} to filter
+   * @param big       the {@link IntervalsSource} to filter by
+   */
+  public static IntervalsSource containedBy(IntervalsSource small, IntervalsSource big) {
+    return new ConjunctionIntervalsSource(Arrays.asList(small, big), IntervalFunction.CONTAINED_BY);
+  }
+
+  // TODO: beforeQuery, afterQuery, arbitrary IntervalFunctions
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalsSource.java
new file mode 100644
index 0000000..9791ff8
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalsSource.java
@@ -0,0 +1,61 @@
+/*
+ * 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.Set;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+
+/**
+ * A helper class for {@link IntervalQuery} that provides an {@link IntervalIterator}
+ * for a given field and segment
+ *
+ * Static constructor functions for various different sources can be found in the
+ * {@link Intervals} class
+ */
+public abstract class IntervalsSource {
+
+  /**
+   * Create an {@link IntervalIterator} exposing the minimum intervals defined by this {@link IntervalsSource}
+   *
+   * Returns {@code null} if no intervals for this field exist in this segment
+   *
+   * @param field the field to read positions from
+   * @param ctx   the context for which to return the iterator
+   */
+  public abstract IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException;
+
+  /**
+   * Expert: collect {@link Term} objects from this source, to be used for top-level term scoring
+   * @param field the field to be scored
+   * @param terms a {@link Set} which terms should be added to
+   */
+  public abstract void extractTerms(String field, Set<Term> terms);
+
+  @Override
+  public abstract int hashCode();
+
+  @Override
+  public abstract boolean equals(Object other);
+
+  @Override
+  public abstract String toString();
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/974c03a6/lucene/sandbox/src/java/org/apache/lucene/search/intervals/LowpassIntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/LowpassIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/LowpassIntervalsSource.java
new file mode 100644
index 0000000..3bb469e
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/LowpassIntervalsSource.java
@@ -0,0 +1,71 @@
+/*
+ * 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.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+
+class LowpassIntervalsSource extends IntervalsSource {
+
+  final IntervalsSource in;
+  private final int maxWidth;
+
+  LowpassIntervalsSource(IntervalsSource in, int maxWidth) {
+    this.in = in;
+    this.maxWidth = maxWidth;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    LowpassIntervalsSource that = (LowpassIntervalsSource) o;
+    return maxWidth == that.maxWidth &&
+        Objects.equals(in, that.in);
+  }
+
+  @Override
+  public String toString() {
+    return "MAXWIDTH/" + maxWidth + "(" + in + ")";
+  }
+
+  @Override
+  public void extractTerms(String field, Set<Term> terms) {
+    in.extractTerms(field, terms);
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    IntervalIterator i = in.intervals(field, ctx);
+    return new IntervalFilter(i) {
+      @Override
+      protected boolean accept() {
+        return (i.end() - i.start()) + 1 <= maxWidth;
+      }
+    };
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(in, maxWidth);
+  }
+}


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

Posted by ro...@apache.org.
LUCENE-8196: Add IntervalQuery and IntervalsSource to the sandbox


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/00eab54f
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/00eab54f
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/00eab54f

Branch: refs/heads/master
Commit: 00eab54f9d6232c68a93f10ff20e3a724ffeca14
Parents: d332764
Author: Alan Woodward <ro...@apache.org>
Authored: Tue Apr 3 08:32:17 2018 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Tue Apr 3 09:37:12 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +
 .../search/intervals/ConjunctionDISI.java       | 129 +++++++
 .../intervals/ConjunctionIntervalIterator.java  |  72 ++++
 .../intervals/ConjunctionIntervalsSource.java   |  77 ++++
 .../intervals/DifferenceIntervalFunction.java   | 326 ++++++++++++++++
 .../intervals/DifferenceIntervalsSource.java    |  74 ++++
 .../search/intervals/DisiPriorityQueue.java     | 171 +++++++++
 .../lucene/search/intervals/DisiWrapper.java    |  44 +++
 .../intervals/DisjunctionDISIApproximation.java |  76 ++++
 .../intervals/DisjunctionIntervalsSource.java   | 228 ++++++++++++
 .../lucene/search/intervals/IntervalFilter.java |  86 +++++
 .../search/intervals/IntervalFunction.java      | 368 +++++++++++++++++++
 .../search/intervals/IntervalIterator.java      |  77 ++++
 .../lucene/search/intervals/IntervalQuery.java  | 155 ++++++++
 .../lucene/search/intervals/IntervalScorer.java | 105 ++++++
 .../lucene/search/intervals/Intervals.java      | 188 ++++++++++
 .../search/intervals/IntervalsSource.java       |  61 +++
 .../intervals/LowpassIntervalsSource.java       |  71 ++++
 .../search/intervals/TermIntervalsSource.java   | 182 +++++++++
 .../lucene/search/intervals/package-info.java   |  66 ++++
 .../search/intervals/TestIntervalQuery.java     | 164 +++++++++
 .../lucene/search/intervals/TestIntervals.java  | 197 ++++++++++
 22 files changed, 2921 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 6111554..1f83980 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -111,6 +111,10 @@ New Features
 
 * LUCENE-8125: ICUTokenizer support for emoji/emoji sequence tokens. (Robert Muir)
 
+* LUCENE-8196: A new IntervalQuery in the sandbox allows efficient proximity
+  searches based on minimum-interval semantics. (Alan Woodward, Adrien Grand,
+  Jim Ferenczi, Simon Willnauer)
+
 Bug Fixes
 
 * LUCENE-8234: Fixed bug in how spatial relationship is computed for

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionDISI.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionDISI.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionDISI.java
new file mode 100644
index 0000000..d290e52
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionDISI.java
@@ -0,0 +1,129 @@
+/*
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.CollectionUtil;
+
+/** A conjunction of DocIdSetIterators.
+ * This iterates over the doc ids that are present in each given DocIdSetIterator.
+ * @lucene.internal
+ */
+final class ConjunctionDISI extends DocIdSetIterator {
+
+  /** Create a conjunction over the provided DocIdSetIterators. */
+  public static DocIdSetIterator intersectIterators(List<? extends DocIdSetIterator> iterators) {
+    if (iterators.size() < 2) {
+      throw new IllegalArgumentException("Cannot make a ConjunctionDISI of less than 2 iterators");
+    }
+    final List<DocIdSetIterator> allIterators = new ArrayList<>();
+    for (DocIdSetIterator iterator : iterators) {
+      addIterator(iterator, allIterators);
+    }
+
+    return new ConjunctionDISI(allIterators);
+  }
+
+  private static void addIterator(DocIdSetIterator disi, List<DocIdSetIterator> allIterators) {
+    if (disi.getClass() == ConjunctionDISI.class) { // Check for exactly this class for collapsing
+      ConjunctionDISI conjunction = (ConjunctionDISI) disi;
+      // subconjuctions have already split themselves into two phase iterators and others, so we can take those
+      // iterators as they are and move them up to this conjunction
+      allIterators.add(conjunction.lead1);
+      allIterators.add(conjunction.lead2);
+      Collections.addAll(allIterators, conjunction.others);
+    } else {
+      allIterators.add(disi);
+    }
+  }
+
+  final DocIdSetIterator lead1, lead2;
+  final DocIdSetIterator[] others;
+
+  private ConjunctionDISI(List<? extends DocIdSetIterator> iterators) {
+    assert iterators.size() >= 2;
+    // Sort the array the first time to allow the least frequent DocsEnum to
+    // lead the matching.
+    CollectionUtil.timSort(iterators, Comparator.comparingLong(DocIdSetIterator::cost));
+    lead1 = iterators.get(0);
+    lead2 = iterators.get(1);
+    others = iterators.subList(2, iterators.size()).toArray(new DocIdSetIterator[0]);
+  }
+
+  private int doNext(int doc) throws IOException {
+    advanceHead: for(;;) {
+      assert doc == lead1.docID();
+
+      // find agreement between the two iterators with the lower costs
+      // we special case them because they do not need the
+      // 'other.docID() < doc' check that the 'others' iterators need
+      final int next2 = lead2.advance(doc);
+      if (next2 != doc) {
+        doc = lead1.advance(next2);
+        if (next2 != doc) {
+          continue;
+        }
+      }
+
+      // then find agreement with other iterators
+      for (DocIdSetIterator other : others) {
+        // other.doc may already be equal to doc if we "continued advanceHead"
+        // on the previous iteration and the advance on the lead scorer exactly matched.
+        if (other.docID() < doc) {
+          final int next = other.advance(doc);
+
+          if (next > doc) {
+            // iterator beyond the current doc - advance lead and continue to the new highest doc.
+            doc = lead1.advance(next);
+            continue advanceHead;
+          }
+        }
+      }
+
+      // success - all iterators are on the same doc
+      return doc;
+    }
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    return doNext(lead1.advance(target));
+  }
+
+  @Override
+  public int docID() {
+    return lead1.docID();
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    return doNext(lead1.nextDoc());
+  }
+
+  @Override
+  public long cost() {
+    return lead1.cost(); // overestimate
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalIterator.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalIterator.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalIterator.java
new file mode 100644
index 0000000..8efe4e4
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalIterator.java
@@ -0,0 +1,72 @@
+/*
+ * 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.List;
+
+import org.apache.lucene.search.DocIdSetIterator;
+
+abstract class ConjunctionIntervalIterator extends IntervalIterator {
+
+  final DocIdSetIterator approximation;
+  final List<IntervalIterator> subIterators;
+  final float cost;
+
+  ConjunctionIntervalIterator(List<IntervalIterator> subIterators) {
+    this.approximation = ConjunctionDISI.intersectIterators(subIterators);
+    this.subIterators = subIterators;
+    float costsum = 0;
+    for (IntervalIterator it : subIterators) {
+      costsum += it.matchCost();
+    }
+    this.cost = costsum;
+  }
+
+  @Override
+  public int docID() {
+    return approximation.docID();
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    int doc = approximation.nextDoc();
+    reset();
+    return doc;
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    int doc = approximation.advance(target);
+    reset();
+    return doc;
+  }
+
+  protected abstract void reset() throws IOException;
+
+  @Override
+  public long cost() {
+    return approximation.cost();
+  }
+
+  @Override
+  public final float matchCost() {
+    return cost;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalsSource.java
new file mode 100644
index 0000000..d2805c9
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalsSource.java
@@ -0,0 +1,77 @@
+/*
+ * 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.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+
+class ConjunctionIntervalsSource extends IntervalsSource {
+
+  final List<IntervalsSource> subSources;
+  final IntervalFunction function;
+
+  ConjunctionIntervalsSource(List<IntervalsSource> subSources, IntervalFunction function) {
+    this.subSources = subSources;
+    this.function = function;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    ConjunctionIntervalsSource that = (ConjunctionIntervalsSource) o;
+    return Objects.equals(subSources, that.subSources) &&
+        Objects.equals(function, that.function);
+  }
+
+  @Override
+  public String toString() {
+    return function + subSources.stream().map(Object::toString).collect(Collectors.joining(",", "(", ")"));
+  }
+
+  @Override
+  public void extractTerms(String field, Set<Term> terms) {
+    for (IntervalsSource source : subSources) {
+      source.extractTerms(field, terms);
+    }
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    List<IntervalIterator> subIntervals = new ArrayList<>();
+    for (IntervalsSource source : subSources) {
+      IntervalIterator it = source.intervals(field, ctx);
+      if (it == null)
+        return null;
+      subIntervals.add(it);
+    }
+    return function.apply(subIntervals);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(subSources, function);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalFunction.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalFunction.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalFunction.java
new file mode 100644
index 0000000..18d4d67
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalFunction.java
@@ -0,0 +1,326 @@
+/*
+ * 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;
+
+/**
+ * A function that takes two interval iterators and combines them to produce a third,
+ * generally by computing a difference interval between them
+ */
+abstract class DifferenceIntervalFunction {
+
+  @Override
+  public abstract int hashCode();
+
+  @Override
+  public abstract boolean equals(Object obj);
+
+  @Override
+  public abstract String toString();
+
+  /**
+   * Combine two interval iterators into a third
+   */
+  public abstract IntervalIterator apply(IntervalIterator minuend, IntervalIterator subtrahend);
+
+  /**
+   * Filters the minuend iterator so that only intervals that do not overlap intervals from the
+   * subtrahend iterator are returned
+   */
+  static final DifferenceIntervalFunction NON_OVERLAPPING = new SingletonFunction("NON_OVERLAPPING") {
+    @Override
+    public IntervalIterator apply(IntervalIterator minuend, IntervalIterator subtrahend) {
+      return new NonOverlappingIterator(minuend, subtrahend);
+    }
+  };
+
+  /**
+   * Filters the minuend iterator so that only intervals that do not contain intervals from the
+   * subtrahend iterator are returned
+   */
+  static final DifferenceIntervalFunction NOT_CONTAINING = new SingletonFunction("NOT_CONTAINING") {
+    @Override
+    public IntervalIterator apply(IntervalIterator minuend, IntervalIterator subtrahend) {
+      return new NotContainingIterator(minuend, subtrahend);
+    }
+  };
+
+  /**
+   * Filters the minuend iterator so that only intervals that are not contained by intervals from
+   * the subtrahend iterator are returned
+   */
+  static final DifferenceIntervalFunction NOT_CONTAINED_BY = new SingletonFunction("NOT_CONTAINED_BY") {
+    @Override
+    public IntervalIterator apply(IntervalIterator minuend, IntervalIterator subtrahend) {
+      return new NotContainedByIterator(minuend, subtrahend);
+    }
+  };
+
+  private static abstract class RelativeIterator extends IntervalIterator {
+
+    final IntervalIterator a;
+    final IntervalIterator b;
+
+    boolean bpos;
+
+    RelativeIterator(IntervalIterator a, IntervalIterator b) {
+      this.a = a;
+      this.b = b;
+    }
+
+    @Override
+    public int docID() {
+      return a.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      int doc = a.nextDoc();
+      reset();
+      return doc;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      int doc = a.advance(target);
+      reset();
+      return doc;
+    }
+
+    @Override
+    public long cost() {
+      return a.cost();
+    }
+
+    protected void reset() throws IOException {
+      int doc = a.docID();
+      bpos = b.docID() == doc ||
+          (b.docID() < doc && b.advance(doc) == doc);
+    }
+
+    @Override
+    public int start() {
+      return a.start();
+    }
+
+    @Override
+    public int end() {
+      return a.end();
+    }
+
+    @Override
+    public float matchCost() {
+      return a.matchCost() + b.matchCost();
+    }
+  }
+
+  private static class NonOverlappingIterator extends RelativeIterator {
+
+    private NonOverlappingIterator(IntervalIterator minuend, IntervalIterator subtrahend) {
+      super(minuend, subtrahend);
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      if (bpos == false)
+        return a.nextInterval();
+      while (a.nextInterval() != NO_MORE_INTERVALS) {
+        while (b.end() < a.start()) {
+          if (b.nextInterval() == NO_MORE_INTERVALS) {
+            bpos = false;
+            return a.start();
+          }
+        }
+        if (b.start() > a.end())
+          return a.start();
+      }
+      return NO_MORE_INTERVALS;
+    }
+  }
+
+  /**
+   * Filters the minuend iterator so that only intervals that do not occur within a set number
+   * of positions of intervals from the subtrahend iterator are returned
+   */
+  static class NotWithinFunction extends DifferenceIntervalFunction {
+
+    private final int positions;
+
+    NotWithinFunction(int positions) {
+      this.positions = positions;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      NotWithinFunction that = (NotWithinFunction) o;
+      return positions == that.positions;
+    }
+
+    @Override
+    public String toString() {
+      return "NOTWITHIN/" + positions;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(positions);
+    }
+
+    @Override
+    public IntervalIterator apply(IntervalIterator minuend, IntervalIterator subtrahend) {
+      IntervalIterator notWithin = new IntervalIterator() {
+
+        @Override
+        public int docID() {
+          return subtrahend.docID();
+        }
+
+        @Override
+        public int nextDoc() throws IOException {
+          positioned = false;
+          return subtrahend.nextDoc();
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+          positioned = false;
+          return subtrahend.advance(target);
+        }
+
+        @Override
+        public long cost() {
+          return subtrahend.cost();
+        }
+
+        boolean positioned = false;
+
+        @Override
+        public int start() {
+          if (positioned == false)
+            return -1;
+          int start = subtrahend.start();
+          return Math.max(0, start - positions);
+        }
+
+        @Override
+        public int end() {
+          if (positioned == false)
+            return -1;
+          int end = subtrahend.end();
+          int newEnd = end + positions;
+          if (newEnd < 0) // check for overflow
+            return Integer.MAX_VALUE;
+          return newEnd;
+        }
+
+        @Override
+        public int nextInterval() throws IOException {
+          if (positioned == false) {
+            positioned = true;
+          }
+          return subtrahend.nextInterval();
+        }
+
+        @Override
+        public float matchCost() {
+          return subtrahend.matchCost();
+        }
+
+      };
+      return NON_OVERLAPPING.apply(minuend, notWithin);
+    }
+  }
+
+  private static class NotContainingIterator extends RelativeIterator {
+
+    private NotContainingIterator(IntervalIterator minuend, IntervalIterator subtrahend) {
+      super(minuend, subtrahend);
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      if (bpos == false)
+        return a.nextInterval();
+      while (a.nextInterval() != NO_MORE_INTERVALS) {
+        while (b.start() < a.start() && b.end() < a.end()) {
+          if (b.nextInterval() == NO_MORE_INTERVALS) {
+            bpos = false;
+            return a.start();
+          }
+        }
+        if (b.start() > a.end())
+          return a.start();
+      }
+      return NO_MORE_INTERVALS;
+    }
+
+  }
+
+  private static class NotContainedByIterator extends RelativeIterator {
+
+    NotContainedByIterator(IntervalIterator a, IntervalIterator b) {
+      super(a, b);
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      if (bpos == false)
+        return a.nextInterval();
+      while (a.nextInterval() != NO_MORE_INTERVALS) {
+        while (b.end() < a.end()) {
+          if (b.nextInterval() == NO_MORE_INTERVALS)
+            return a.start();
+        }
+        if (a.start() < b.start())
+          return a.start();
+      }
+      return NO_MORE_INTERVALS;
+    }
+  }
+
+  private static abstract class SingletonFunction extends DifferenceIntervalFunction {
+
+    private final String name;
+
+    SingletonFunction(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public int hashCode() {
+      return System.identityHashCode(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return obj == this;
+    }
+
+    @Override
+    public String toString() {
+      return name;
+    }
+
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalsSource.java
new file mode 100644
index 0000000..316b6ff
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalsSource.java
@@ -0,0 +1,74 @@
+/*
+ * 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.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+
+class DifferenceIntervalsSource extends IntervalsSource {
+
+  final IntervalsSource minuend;
+  final IntervalsSource subtrahend;
+  final DifferenceIntervalFunction function;
+
+  public DifferenceIntervalsSource(IntervalsSource minuend, IntervalsSource subtrahend, DifferenceIntervalFunction function) {
+    this.minuend = minuend;
+    this.subtrahend = subtrahend;
+    this.function = function;
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    IntervalIterator minIt = minuend.intervals(field, ctx);
+    if (minIt == null)
+      return null;
+    IntervalIterator subIt = subtrahend.intervals(field, ctx);
+    if (subIt == null)
+      return minIt;
+    return function.apply(minIt, subIt);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    DifferenceIntervalsSource that = (DifferenceIntervalsSource) o;
+    return Objects.equals(minuend, that.minuend) &&
+        Objects.equals(subtrahend, that.subtrahend) &&
+        Objects.equals(function, that.function);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(minuend, subtrahend, function);
+  }
+
+  @Override
+  public String toString() {
+    return function + "(" + minuend + ", " + subtrahend + ")";
+  }
+
+  @Override
+  public void extractTerms(String field, Set<Term> terms) {
+    minuend.extractTerms(field, terms);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiPriorityQueue.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiPriorityQueue.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiPriorityQueue.java
new file mode 100644
index 0000000..0be96ab
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiPriorityQueue.java
@@ -0,0 +1,171 @@
+/*
+ * 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.util.Arrays;
+import java.util.Iterator;
+
+import org.apache.lucene.util.PriorityQueue;
+
+/**
+ * A priority queue of DocIdSetIterators that orders by current doc ID.
+ * This specialization is needed over {@link PriorityQueue} because the
+ * pluggable comparison function makes the rebalancing quite slow.
+ * @lucene.internal
+ */
+final class DisiPriorityQueue implements Iterable<DisiWrapper> {
+
+  static int leftNode(int node) {
+    return ((node + 1) << 1) - 1;
+  }
+
+  static int rightNode(int leftNode) {
+    return leftNode + 1;
+  }
+
+  static int parentNode(int node) {
+    return ((node + 1) >>> 1) - 1;
+  }
+
+  private final DisiWrapper[] heap;
+  private int size;
+
+  public DisiPriorityQueue(int maxSize) {
+    heap = new DisiWrapper[maxSize];
+    size = 0;
+  }
+
+  public int size() {
+    return size;
+  }
+
+  public DisiWrapper top() {
+    return heap[0];
+  }
+
+  /** Get the list of scorers which are on the current doc. */
+  public DisiWrapper topList() {
+    final DisiWrapper[] heap = this.heap;
+    final int size = this.size;
+    DisiWrapper list = heap[0];
+    list.next = null;
+    if (size >= 3) {
+      list = topList(list, heap, size, 1);
+      list = topList(list, heap, size, 2);
+    } else if (size == 2 && heap[1].doc == list.doc) {
+      list = prepend(heap[1], list);
+    }
+    return list;
+  }
+
+  // prepend w1 (iterator) to w2 (list)
+  private DisiWrapper prepend(DisiWrapper w1, DisiWrapper w2) {
+    w1.next = w2;
+    return w1;
+  }
+
+  private DisiWrapper topList(DisiWrapper list, DisiWrapper[] heap,
+                              int size, int i) {
+    final DisiWrapper w = heap[i];
+    if (w.doc == list.doc) {
+      list = prepend(w, list);
+      final int left = leftNode(i);
+      final int right = left + 1;
+      if (right < size) {
+        list = topList(list, heap, size, left);
+        list = topList(list, heap, size, right);
+      } else if (left < size && heap[left].doc == list.doc) {
+        list = prepend(heap[left], list);
+      }
+    }
+    return list;
+  }
+
+  public DisiWrapper add(DisiWrapper entry) {
+    final DisiWrapper[] heap = this.heap;
+    final int size = this.size;
+    heap[size] = entry;
+    upHeap(size);
+    this.size = size + 1;
+    return heap[0];
+  }
+
+  public DisiWrapper pop() {
+    final DisiWrapper[] heap = this.heap;
+    final DisiWrapper result = heap[0];
+    final int i = --size;
+    heap[0] = heap[i];
+    heap[i] = null;
+    downHeap(i);
+    return result;
+  }
+
+  public DisiWrapper updateTop() {
+    downHeap(size);
+    return heap[0];
+  }
+
+  DisiWrapper updateTop(DisiWrapper topReplacement) {
+    heap[0] = topReplacement;
+    return updateTop();
+  }
+
+  void upHeap(int i) {
+    final DisiWrapper node = heap[i];
+    final int nodeDoc = node.doc;
+    int j = parentNode(i);
+    while (j >= 0 && nodeDoc < heap[j].doc) {
+      heap[i] = heap[j];
+      i = j;
+      j = parentNode(j);
+    }
+    heap[i] = node;
+  }
+
+  void downHeap(int size) {
+    int i = 0;
+    final DisiWrapper node = heap[0];
+    int j = leftNode(i);
+    if (j < size) {
+      int k = rightNode(j);
+      if (k < size && heap[k].doc < heap[j].doc) {
+        j = k;
+      }
+      if (heap[j].doc < node.doc) {
+        do {
+          heap[i] = heap[j];
+          i = j;
+          j = leftNode(i);
+          k = rightNode(j);
+          if (k < size && heap[k].doc < heap[j].doc) {
+            j = k;
+          }
+        } while (j < size && heap[j].doc < node.doc);
+        heap[i] = node;
+      }
+    }
+  }
+
+  @Override
+  public Iterator<DisiWrapper> iterator() {
+    return Arrays.asList(heap).subList(0, size).iterator();
+  }
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiWrapper.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiWrapper.java
new file mode 100644
index 0000000..483c626
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisiWrapper.java
@@ -0,0 +1,44 @@
+/*
+ * 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 org.apache.lucene.search.DocIdSetIterator;
+
+class DisiWrapper {
+
+  public final DocIdSetIterator iterator;
+  public final IntervalIterator intervals;
+  public final long cost;
+  public final float matchCost; // the match cost for two-phase iterators, 0 otherwise
+  public int doc; // the current doc, used for comparison
+  public DisiWrapper next; // reference to a next element, see #topList
+
+  // An approximation of the iterator, or the iterator itself if it does not
+  // support two-phase iteration
+  public final DocIdSetIterator approximation;
+
+  public DisiWrapper(IntervalIterator iterator) {
+    this.intervals = iterator;
+    this.iterator = iterator;
+    this.cost = iterator.cost();
+    this.doc = -1;
+    this.approximation = iterator;
+    this.matchCost = iterator.matchCost();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionDISIApproximation.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionDISIApproximation.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionDISIApproximation.java
new file mode 100644
index 0000000..d4e8e79
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionDISIApproximation.java
@@ -0,0 +1,76 @@
+/*
+ * 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.search.DocIdSetIterator;
+
+/**
+ * A {@link DocIdSetIterator} which is a disjunction of the approximations of
+ * the provided iterators.
+ * @lucene.internal
+ */
+class DisjunctionDISIApproximation extends DocIdSetIterator {
+
+  final DisiPriorityQueue subIterators;
+  final long cost;
+
+  public DisjunctionDISIApproximation(DisiPriorityQueue subIterators) {
+    this.subIterators = subIterators;
+    long cost = 0;
+    for (DisiWrapper w : subIterators) {
+      cost += w.cost;
+    }
+    this.cost = cost;
+  }
+
+  @Override
+  public long cost() {
+    return cost;
+  }
+
+  @Override
+  public int docID() {
+    return subIterators.top().doc;
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    DisiWrapper top = subIterators.top();
+    final int doc = top.doc;
+    do {
+      top.doc = top.approximation.nextDoc();
+      top = subIterators.updateTop();
+    } while (top.doc == doc);
+
+    return top.doc;
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    DisiWrapper top = subIterators.top();
+    do {
+      top.doc = top.approximation.advance(target);
+      top = subIterators.updateTop();
+    } while (top.doc < target);
+
+    return top.doc;
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionIntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionIntervalsSource.java
new file mode 100644
index 0000000..c7c7166
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionIntervalsSource.java
@@ -0,0 +1,228 @@
+/*
+ * 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.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.PriorityQueue;
+
+class DisjunctionIntervalsSource extends IntervalsSource {
+
+  final List<IntervalsSource> subSources;
+
+  public DisjunctionIntervalsSource(List<IntervalsSource> subSources) {
+    this.subSources = subSources;
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    List<IntervalIterator> subIterators = new ArrayList<>();
+    for (IntervalsSource subSource : subSources) {
+      IntervalIterator it = subSource.intervals(field, ctx);
+      if (it != null) {
+        subIterators.add(it);
+      }
+    }
+    if (subIterators.size() == 0)
+      return null;
+    return new DisjunctionIntervalIterator(subIterators);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    DisjunctionIntervalsSource that = (DisjunctionIntervalsSource) o;
+    return Objects.equals(subSources, that.subSources);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(subSources);
+  }
+
+  @Override
+  public String toString() {
+    return subSources.stream().map(Object::toString).collect(Collectors.joining(",", "or(", ")"));
+  }
+
+  @Override
+  public void extractTerms(String field, Set<Term> terms) {
+    for (IntervalsSource source : subSources) {
+      source.extractTerms(field, terms);
+    }
+  }
+
+  private static class DisjunctionIntervalIterator extends IntervalIterator {
+
+    final DocIdSetIterator approximation;
+    final PriorityQueue<IntervalIterator> intervalQueue;
+    final DisiPriorityQueue disiQueue;
+    final List<IntervalIterator> iterators;
+    final float matchCost;
+
+    IntervalIterator current = EMPTY;
+
+    DisjunctionIntervalIterator(List<IntervalIterator> iterators) {
+      this.disiQueue = new DisiPriorityQueue(iterators.size());
+      for (IntervalIterator it : iterators) {
+        disiQueue.add(new DisiWrapper(it));
+      }
+      this.approximation = new DisjunctionDISIApproximation(disiQueue);
+      this.iterators = iterators;
+      this.intervalQueue = new PriorityQueue<IntervalIterator>(iterators.size()) {
+        @Override
+        protected boolean lessThan(IntervalIterator a, IntervalIterator b) {
+          return a.end() < b.end() || (a.end() == b.end() && a.start() >= b.start());
+        }
+      };
+      float costsum = 0;
+      for (IntervalIterator it : iterators) {
+        costsum += it.cost();
+      }
+      this.matchCost = costsum;
+    }
+
+    @Override
+    public float matchCost() {
+      return matchCost;
+    }
+
+    @Override
+    public int start() {
+      return current.start();
+    }
+
+    @Override
+    public int end() {
+      return current.end();
+    }
+
+    private void reset() throws IOException {
+      intervalQueue.clear();
+      for (DisiWrapper dw = disiQueue.topList(); dw != null; dw = dw.next) {
+        dw.intervals.nextInterval();
+        intervalQueue.add(dw.intervals);
+      }
+      current = EMPTY;
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      if (current == EMPTY) {
+        if (intervalQueue.size() > 0) {
+          current = intervalQueue.top();
+        }
+        return current.start();
+      }
+      int start = current.start(), end = current.end();
+      while (intervalQueue.size() > 0 && contains(intervalQueue.top(), start, end)) {
+        IntervalIterator it = intervalQueue.pop();
+        if (it != null && it.nextInterval() != NO_MORE_INTERVALS) {
+          intervalQueue.add(it);
+        }
+      }
+      if (intervalQueue.size() == 0) {
+        current = EMPTY;
+        return NO_MORE_INTERVALS;
+      }
+      current = intervalQueue.top();
+      return current.start();
+    }
+
+    private boolean contains(IntervalIterator it, int start, int end) {
+      return start >= it.start() && start <= it.end() && end >= it.start() && end <= it.end();
+    }
+
+    @Override
+    public int docID() {
+      return approximation.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      int doc = approximation.nextDoc();
+      reset();
+      return doc;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      int doc = approximation.advance(target);
+      reset();
+      return doc;
+    }
+
+    @Override
+    public long cost() {
+      return approximation.cost();
+    }
+  }
+
+  private static final IntervalIterator EMPTY = new IntervalIterator() {
+
+    @Override
+    public int docID() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long cost() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int start() {
+      return -1;
+    }
+
+    @Override
+    public int end() {
+      return -1;
+    }
+
+    @Override
+    public int nextInterval() {
+      return NO_MORE_INTERVALS;
+    }
+
+    @Override
+    public float matchCost() {
+      return 0;
+    }
+  };
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFilter.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFilter.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFilter.java
new file mode 100644
index 0000000..d1d2fcf
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFilter.java
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+/**
+ * Wraps an {@link IntervalIterator} and passes through those intervals that match the {@link #accept()} function
+ */
+public abstract class IntervalFilter extends IntervalIterator {
+
+  private final IntervalIterator in;
+
+  /**
+   * Create a new filter
+   */
+  public IntervalFilter(IntervalIterator in) {
+    this.in = in;
+  }
+
+  @Override
+  public int docID() {
+    return in.docID();
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    return in.nextDoc();
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    return in.advance(target);
+  }
+
+  @Override
+  public long cost() {
+    return in.cost();
+  }
+
+  @Override
+  public int start() {
+    return in.start();
+  }
+
+  @Override
+  public int end() {
+    return in.end();
+  }
+
+  @Override
+  public float matchCost() {
+    return in.matchCost();
+  }
+
+  /**
+   * @return {@code true} if the wrapped iterator's interval should be passed on
+   */
+  protected abstract boolean accept();
+
+  @Override
+  public final int nextInterval() throws IOException {
+    int next;
+    do {
+      next = in.nextInterval();
+    }
+    while (accept() == false && next != IntervalIterator.NO_MORE_INTERVALS);
+    return next;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFunction.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFunction.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFunction.java
new file mode 100644
index 0000000..c550a72
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFunction.java
@@ -0,0 +1,368 @@
+/*
+ * 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.List;
+
+import org.apache.lucene.util.PriorityQueue;
+
+/**
+ * Combine a list of {@link IntervalIterator}s into another
+ */
+abstract class IntervalFunction {
+
+  @Override
+  public abstract int hashCode();
+
+  @Override
+  public abstract boolean equals(Object obj);
+
+  @Override
+  public abstract String toString();
+
+  /**
+   * Combine the iterators into another iterator
+   */
+  public abstract IntervalIterator apply(List<IntervalIterator> iterators);
+
+  static final IntervalFunction BLOCK = new SingletonFunction("BLOCK") {
+    @Override
+    public IntervalIterator apply(List<IntervalIterator> iterators) {
+      return new BlockIntervalIterator(iterators);
+    }
+  };
+
+  private static class BlockIntervalIterator extends ConjunctionIntervalIterator {
+
+    int start = -1, end = -1;
+
+    BlockIntervalIterator(List<IntervalIterator> subIterators) {
+      super(subIterators);
+    }
+
+    @Override
+    public int start() {
+      return start;
+    }
+
+    @Override
+    public int end() {
+      return end;
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      if (subIterators.get(0).nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
+        return IntervalIterator.NO_MORE_INTERVALS;
+      int i = 1;
+      while (i < subIterators.size()) {
+        while (subIterators.get(i).start() <= subIterators.get(i - 1).end()) {
+          if (subIterators.get(i).nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
+            return IntervalIterator.NO_MORE_INTERVALS;
+        }
+        if (subIterators.get(i).start() == subIterators.get(i - 1).end() + 1) {
+          i = i + 1;
+        }
+        else {
+          if (subIterators.get(0).nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
+            return IntervalIterator.NO_MORE_INTERVALS;
+          i = 1;
+        }
+      }
+      start = subIterators.get(0).start();
+      end = subIterators.get(subIterators.size() - 1).end();
+      return start;
+    }
+
+    @Override
+    protected void reset() {
+      start = end = -1;
+    }
+  }
+
+  /**
+   * Return an iterator over intervals where the subiterators appear in a given order
+   */
+  static final IntervalFunction ORDERED = new SingletonFunction("ORDERED") {
+    @Override
+    public IntervalIterator apply(List<IntervalIterator> intervalIterators) {
+      return new OrderedIntervalIterator(intervalIterators);
+    }
+  };
+
+  private static class OrderedIntervalIterator extends ConjunctionIntervalIterator {
+
+    int start = -1, end = -1, i;
+
+    private OrderedIntervalIterator(List<IntervalIterator> subIntervals) {
+      super(subIntervals);
+    }
+
+    @Override
+    public int start() {
+      return start;
+    }
+
+    @Override
+    public int end() {
+      return end;
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      start = end = IntervalIterator.NO_MORE_INTERVALS;
+      int b = Integer.MAX_VALUE;
+      i = 1;
+      while (true) {
+        while (true) {
+          if (subIterators.get(i - 1).end() >= b)
+            return start;
+          if (i == subIterators.size() || subIterators.get(i).start() > subIterators.get(i - 1).end())
+            break;
+          do {
+            if (subIterators.get(i).end() >= b || subIterators.get(i).nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
+              return start;
+          }
+          while (subIterators.get(i).start() <= subIterators.get(i - 1).end());
+          i++;
+        }
+        start = subIterators.get(0).start();
+        end = subIterators.get(subIterators.size() - 1).end();
+        b = subIterators.get(subIterators.size() - 1).start();
+        i = 1;
+        if (subIterators.get(0).nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
+          return start;
+      }
+    }
+
+    @Override
+    protected void reset() throws IOException {
+      subIterators.get(0).nextInterval();
+      i = 1;
+      start = end = -1;
+    }
+  }
+
+  /**
+   * Return an iterator over intervals where the subiterators appear in any order
+   */
+  static final IntervalFunction UNORDERED = new SingletonFunction("UNORDERED") {
+    @Override
+    public IntervalIterator apply(List<IntervalIterator> intervalIterators) {
+      return new UnorderedIntervalIterator(intervalIterators);
+    }
+  };
+
+  private static class UnorderedIntervalIterator extends ConjunctionIntervalIterator {
+
+    private final PriorityQueue<IntervalIterator> queue;
+    private final IntervalIterator[] subIterators;
+
+    int start = -1, end = -1, queueEnd;
+
+    UnorderedIntervalIterator(List<IntervalIterator> subIterators) {
+      super(subIterators);
+      this.queue = new PriorityQueue<IntervalIterator>(subIterators.size()) {
+        @Override
+        protected boolean lessThan(IntervalIterator a, IntervalIterator b) {
+          return a.start() < b.start() || (a.start() == b.start() && a.end() >= b.end());
+        }
+      };
+      this.subIterators = new IntervalIterator[subIterators.size()];
+
+      for (int i = 0; i < subIterators.size(); i++) {
+        this.subIterators[i] = subIterators.get(i);
+      }
+    }
+
+    @Override
+    public int start() {
+      return start;
+    }
+
+    @Override
+    public int end() {
+      return end;
+    }
+
+    void updateRightExtreme(IntervalIterator it) {
+      int itEnd = it.end();
+      if (itEnd > queueEnd) {
+        queueEnd = itEnd;
+      }
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      while (this.queue.size() == subIterators.length && queue.top().start() == start) {
+        IntervalIterator it = queue.pop();
+        if (it != null && it.nextInterval() != IntervalIterator.NO_MORE_INTERVALS) {
+          queue.add(it);
+          updateRightExtreme(it);
+        }
+      }
+      if (this.queue.size() < subIterators.length)
+        return IntervalIterator.NO_MORE_INTERVALS;
+      do {
+        start = queue.top().start();
+        end = queueEnd;
+        if (queue.top().end() == end)
+          return start;
+        IntervalIterator it = queue.pop();
+        if (it != null && it.nextInterval() != IntervalIterator.NO_MORE_INTERVALS) {
+          queue.add(it);
+          updateRightExtreme(it);
+        }
+      } while (this.queue.size() == subIterators.length && end == queueEnd);
+      return start;
+    }
+
+    @Override
+    protected void reset() throws IOException {
+      queueEnd = start = end = -1;
+      this.queue.clear();
+      for (IntervalIterator it : subIterators) {
+        it.nextInterval();
+        queue.add(it);
+        updateRightExtreme(it);
+      }
+    }
+
+  }
+
+  /**
+   * Returns an interval over iterators where the first iterator contains intervals from the second
+   */
+  static final IntervalFunction CONTAINING = new SingletonFunction("CONTAINING") {
+    @Override
+    public IntervalIterator apply(List<IntervalIterator> iterators) {
+      if (iterators.size() != 2)
+        throw new IllegalStateException("CONTAINING function requires two iterators");
+      IntervalIterator a = iterators.get(0);
+      IntervalIterator b = iterators.get(1);
+      return new ConjunctionIntervalIterator(iterators) {
+
+        boolean bpos;
+
+        @Override
+        public int start() {
+          return a.start();
+        }
+
+        @Override
+        public int end() {
+          return a.end();
+        }
+
+        @Override
+        public int nextInterval() throws IOException {
+          if (bpos == false)
+            return IntervalIterator.NO_MORE_INTERVALS;
+          while (a.nextInterval() != IntervalIterator.NO_MORE_INTERVALS) {
+            while (b.start() < a.start() && b.end() < a.end()) {
+              if (b.nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
+                return IntervalIterator.NO_MORE_INTERVALS;
+            }
+            if (a.start() <= b.start() && a.end() >= b.end())
+              return a.start();
+          }
+          return IntervalIterator.NO_MORE_INTERVALS;
+        }
+
+        @Override
+        protected void reset() throws IOException {
+          bpos = true;
+        }
+      };
+    }
+  };
+
+  /**
+   * Return an iterator over intervals where the first iterator is contained by intervals from the second
+   */
+  static final IntervalFunction CONTAINED_BY = new SingletonFunction("CONTAINED_BY") {
+    @Override
+    public IntervalIterator apply(List<IntervalIterator> iterators) {
+      if (iterators.size() != 2)
+        throw new IllegalStateException("CONTAINED_BY function requires two iterators");
+      IntervalIterator a = iterators.get(0);
+      IntervalIterator b = iterators.get(1);
+      return new ConjunctionIntervalIterator(iterators) {
+
+        boolean bpos;
+
+        @Override
+        public int start() {
+          return a.start();
+        }
+
+        @Override
+        public int end() {
+          return a.end();
+        }
+
+        @Override
+        public int nextInterval() throws IOException {
+          if (bpos == false)
+            return IntervalIterator.NO_MORE_INTERVALS;
+          while (a.nextInterval() != IntervalIterator.NO_MORE_INTERVALS) {
+            while (b.end() < a.end()) {
+              if (b.nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
+                return IntervalIterator.NO_MORE_INTERVALS;
+            }
+            if (b.start() <= a.start())
+              return a.start();
+          }
+          return IntervalIterator.NO_MORE_INTERVALS;
+        }
+
+        @Override
+        protected void reset() throws IOException {
+          bpos = true;
+        }
+      };
+    }
+  };
+
+  private static abstract class SingletonFunction extends IntervalFunction {
+
+    private final String name;
+
+    protected SingletonFunction(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public int hashCode() {
+      return System.identityHashCode(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return obj == this;
+    }
+
+    @Override
+    public String toString() {
+      return name;
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalIterator.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalIterator.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalIterator.java
new file mode 100644
index 0000000..2428720
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalIterator.java
@@ -0,0 +1,77 @@
+/*
+ * 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.search.DocIdSetIterator;
+import org.apache.lucene.search.TwoPhaseIterator;
+
+/**
+ * A {@link DocIdSetIterator} that also allows iteration over matching
+ * intervals in a document.
+ *
+ * Once the iterator is positioned on a document by calling {@link #advance(int)}
+ * or {@link #nextDoc()}, intervals may be retrieved by calling {@link #nextInterval()}
+ * until {@link #NO_MORE_INTERVALS} is returned.
+ *
+ * The limits of the current interval are returned by {@link #start()} and {@link #end()}.
+ * When the iterator has been moved to a new document, but before {@link #nextInterval()}
+ * has been called, both these methods return {@code -1}.
+ *
+ * Note that it is possible for a document to return {@link #NO_MORE_INTERVALS}
+ * on the first call to {@link #nextInterval()}
+ */
+public abstract class IntervalIterator extends DocIdSetIterator {
+
+  /**
+   * When returned from {@link #nextInterval()}, indicates that there are no more
+   * matching intervals on the current document
+   */
+  public static final int NO_MORE_INTERVALS = Integer.MAX_VALUE;
+
+  /**
+   * The start of the current interval
+   *
+   * Returns -1 if {@link #nextInterval()} has not yet been called
+   */
+  public abstract int start();
+
+  /**
+   * The end of the current interval
+   *
+   * Returns -1 if {@link #nextInterval()} has not yet been called
+   */
+  public abstract int end();
+
+  /**
+   * Advance the iterator to the next interval
+   *
+   * @return the start of the next interval, or {@link IntervalIterator#NO_MORE_INTERVALS} if
+   *         there are no more intervals on the current document
+   */
+  public abstract int nextInterval() throws IOException;
+
+  /**
+   * An indication of the average cost of iterating over all intervals in a document
+   *
+   * @see TwoPhaseIterator#matchCost()
+   */
+  public abstract float matchCost();
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalQuery.java
new file mode 100644
index 0000000..934d553
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalQuery.java
@@ -0,0 +1,155 @@
+/*
+ * 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.Arrays;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermStates;
+import org.apache.lucene.search.CollectionStatistics;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LeafSimScorer;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TermStatistics;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.search.similarities.Similarity;
+
+/**
+ * A query that retrieves documents containing intervals returned from an
+ * {@link IntervalsSource}
+ *
+ * Static constructor functions for various different sources can be found in the
+ * {@link Intervals} class
+ */
+public final class IntervalQuery extends Query {
+
+  private final String field;
+  private final IntervalsSource intervalsSource;
+
+  /**
+   * Create a new IntervalQuery
+   * @param field             the field to query
+   * @param intervalsSource   an {@link IntervalsSource} to retrieve intervals from
+   */
+  public IntervalQuery(String field, IntervalsSource intervalsSource) {
+    this.field = field;
+    this.intervalsSource = intervalsSource;
+  }
+
+  public String getField() {
+    return field;
+  }
+
+  @Override
+  public String toString(String field) {
+    return intervalsSource.toString();
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+    return new IntervalWeight(this, scoreMode.needsScores() ? buildSimScorer(searcher, boost) : null,
+        searcher.getSimilarity(), scoreMode);
+  }
+
+  private Similarity.SimScorer buildSimScorer(IndexSearcher searcher, float boost) throws IOException {
+    Set<Term> terms = new HashSet<>();
+    intervalsSource.extractTerms(field, terms);
+    TermStatistics[] termStats = new TermStatistics[terms.size()];
+    int termUpTo = 0;
+    for (Term term : terms) {
+      TermStatistics termStatistics = searcher.termStatistics(term, TermStates.build(searcher.getTopReaderContext(), term, true));
+      if (termStatistics != null) {
+        termStats[termUpTo++] = termStatistics;
+      }
+    }
+    if (termUpTo == 0) {
+      return null;
+    }
+    CollectionStatistics collectionStats = searcher.collectionStatistics(field);
+    return searcher.getSimilarity().scorer(boost, collectionStats, Arrays.copyOf(termStats, termUpTo));
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    IntervalQuery that = (IntervalQuery) o;
+    return Objects.equals(field, that.field) &&
+        Objects.equals(intervalsSource, that.intervalsSource);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(field, intervalsSource);
+  }
+
+  private class IntervalWeight extends Weight {
+
+    final Similarity.SimScorer simScorer;
+    final Similarity similarity;
+    final ScoreMode scoreMode;
+
+    public IntervalWeight(Query query, Similarity.SimScorer simScorer, Similarity similarity, ScoreMode scoreMode) {
+      super(query);
+      this.simScorer = simScorer;
+      this.similarity = similarity;
+      this.scoreMode = scoreMode;
+    }
+
+    @Override
+    public void extractTerms(Set<Term> terms) {
+      intervalsSource.extractTerms(field, terms);
+    }
+
+    @Override
+    public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+      IntervalScorer scorer = (IntervalScorer) scorer(context);
+      if (scorer != null) {
+        int newDoc = scorer.iterator().advance(doc);
+        if (newDoc == doc) {
+          return scorer.explain("weight("+getQuery()+" in "+doc+") [" + similarity.getClass().getSimpleName() + "]");
+        }
+      }
+      return Explanation.noMatch("no matching intervals");
+    }
+
+    @Override
+    public Scorer scorer(LeafReaderContext context) throws IOException {
+      IntervalIterator intervals = intervalsSource.intervals(field, context);
+      if (intervals == null)
+        return null;
+      LeafSimScorer leafScorer = simScorer == null ? null
+          : new LeafSimScorer(simScorer, context.reader(), scoreMode.needsScores(), Float.MAX_VALUE);
+      return new IntervalScorer(this, intervals, leafScorer);
+    }
+
+    @Override
+    public boolean isCacheable(LeafReaderContext ctx) {
+      return true;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalScorer.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalScorer.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalScorer.java
new file mode 100644
index 0000000..6672905
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalScorer.java
@@ -0,0 +1,105 @@
+/*
+ * 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.search.DocIdSetIterator;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.LeafSimScorer;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TwoPhaseIterator;
+import org.apache.lucene.search.Weight;
+
+class IntervalScorer extends Scorer {
+
+  private final IntervalIterator intervals;
+  private final LeafSimScorer simScorer;
+
+  private float freq = -1;
+  private int lastScoredDoc = -1;
+
+  protected IntervalScorer(Weight weight, IntervalIterator intervals, LeafSimScorer simScorer) {
+    super(weight);
+    this.intervals = intervals;
+    this.simScorer = simScorer;
+  }
+
+  @Override
+  public int docID() {
+    return intervals.docID();
+  }
+
+  @Override
+  public float score() throws IOException {
+    ensureFreq();
+    return simScorer.score(docID(), freq);
+  }
+
+  public Explanation explain(String topLevel) throws IOException {
+    ensureFreq();
+    Explanation freqExplanation = Explanation.match(freq, "intervalFreq=" + freq);
+    Explanation scoreExplanation = simScorer.explain(docID(), freqExplanation);
+    return Explanation.match(scoreExplanation.getValue(),
+        topLevel + ", result of:",
+        scoreExplanation);
+  }
+
+  public float freq() throws IOException {
+    ensureFreq();
+    return freq;
+  }
+
+  private void ensureFreq() throws IOException {
+    if (lastScoredDoc != docID()) {
+      lastScoredDoc = docID();
+      freq = 0;
+      do {
+        freq += (1.0 / (intervals.end() - intervals.start() + 1));
+      }
+      while (intervals.nextInterval() != IntervalIterator.NO_MORE_INTERVALS);
+    }
+  }
+
+  @Override
+  public DocIdSetIterator iterator() {
+    return TwoPhaseIterator.asDocIdSetIterator(twoPhaseIterator());
+  }
+
+  @Override
+  public TwoPhaseIterator twoPhaseIterator() {
+    return new TwoPhaseIterator(intervals) {
+      @Override
+      public boolean matches() throws IOException {
+        return intervals.nextInterval() != IntervalIterator.NO_MORE_INTERVALS;
+      }
+
+      @Override
+      public float matchCost() {
+        return intervals.matchCost();
+      }
+    };
+  }
+
+  @Override
+  public float getMaxScore(int upTo) throws IOException {
+    return Float.POSITIVE_INFINITY;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/Intervals.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/Intervals.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/Intervals.java
new file mode 100644
index 0000000..b360919
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/Intervals.java
@@ -0,0 +1,188 @@
+/*
+ * 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.util.Arrays;
+
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Constructor functions for {@link IntervalsSource} types
+ *
+ * These sources implement minimum-interval algorithms taken from the paper
+ * <a href="http://vigna.di.unimi.it/ftp/papers/EfficientAlgorithmsMinimalIntervalSemantics.pdf">
+ * Efficient Optimally Lazy Algorithms for Minimal-Interval Semantics</a>
+ */
+public final class Intervals {
+
+  private Intervals() {}
+
+  /**
+   * Return an {@link IntervalsSource} exposing intervals for a term
+   */
+  public static IntervalsSource term(BytesRef term) {
+    return new TermIntervalsSource(term);
+  }
+
+  /**
+   * Return an {@link IntervalsSource} exposing intervals for a term
+   */
+  public static IntervalsSource term(String term) {
+    return new TermIntervalsSource(new BytesRef(term));
+  }
+
+  /**
+   * Return an {@link IntervalsSource} exposing intervals for a phrase consisting of a list of terms
+   */
+  public static IntervalsSource phrase(String... terms) {
+    IntervalsSource[] sources = new IntervalsSource[terms.length];
+    int i = 0;
+    for (String term : terms) {
+      sources[i] = term(term);
+      i++;
+    }
+    return phrase(sources);
+  }
+
+  /**
+   * Return an {@link IntervalsSource} exposing intervals for a phrase consisting of a list of IntervalsSources
+   */
+  public static IntervalsSource phrase(IntervalsSource... subSources) {
+    return new ConjunctionIntervalsSource(Arrays.asList(subSources), IntervalFunction.BLOCK);
+  }
+
+  /**
+   * Return an {@link IntervalsSource} over the disjunction of a set of sub-sources
+   */
+  public static IntervalsSource or(IntervalsSource... subSources) {
+    if (subSources.length == 1)
+      return subSources[0];
+    return new DisjunctionIntervalsSource(Arrays.asList(subSources));
+  }
+
+  /**
+   * Create an {@link IntervalsSource} that filters a sub-source by the width of its intervals
+   * @param width       the maximum width of intervals in the sub-source ot return
+   * @param subSource   the sub-source to filter
+   */
+  public static IntervalsSource maxwidth(int width, IntervalsSource subSource) {
+    return new LowpassIntervalsSource(subSource, width);
+  }
+
+  /**
+   * Create an ordered {@link IntervalsSource} with an unbounded width range
+   *
+   * Returns intervals in which the subsources all appear in the given order
+   *
+   * @param subSources  an ordered set of {@link IntervalsSource} objects
+   */
+  public static IntervalsSource ordered(IntervalsSource... subSources) {
+    return new ConjunctionIntervalsSource(Arrays.asList(subSources), IntervalFunction.ORDERED);
+  }
+
+  /**
+   * Create an unordered {@link IntervalsSource} with an unbounded width range
+   *
+   * Returns intervals in which all the subsources appear.
+   *
+   * @param subSources  an unordered set of queries
+   */
+  public static IntervalsSource unordered(IntervalsSource... subSources) {
+    return new ConjunctionIntervalsSource(Arrays.asList(subSources), IntervalFunction.UNORDERED);
+  }
+
+  /**
+   * Create a non-overlapping IntervalsSource
+   *
+   * Returns intervals of the minuend that do not overlap with intervals from the subtrahend
+
+   * @param minuend     the {@link IntervalsSource} to filter
+   * @param subtrahend  the {@link IntervalsSource} to filter by
+   */
+  public static IntervalsSource nonOverlapping(IntervalsSource minuend, IntervalsSource subtrahend) {
+    return new DifferenceIntervalsSource(minuend, subtrahend, DifferenceIntervalFunction.NON_OVERLAPPING);
+  }
+
+  /**
+   * Create a not-within {@link IntervalsSource}
+   *
+   * Returns intervals of the minuend that do not appear within a set number of positions of
+   * intervals from the subtrahend query
+   *
+   * @param minuend     the {@link IntervalsSource} to filter
+   * @param positions   the maximum distance that intervals from the minuend may occur from intervals
+   *                    of the subtrahend
+   * @param subtrahend  the {@link IntervalsSource} to filter by
+   */
+  public static IntervalsSource notWithin(IntervalsSource minuend, int positions, IntervalsSource subtrahend) {
+    return new DifferenceIntervalsSource(minuend, subtrahend, new DifferenceIntervalFunction.NotWithinFunction(positions));
+  }
+
+  /**
+   * Create a not-containing {@link IntervalsSource}
+   *
+   * Returns intervals from the minuend that do not contain intervals of the subtrahend
+   *
+   * @param minuend     the {@link IntervalsSource} to filter
+   * @param subtrahend  the {@link IntervalsSource} to filter by
+   */
+  public static IntervalsSource notContaining(IntervalsSource minuend, IntervalsSource subtrahend) {
+    return new DifferenceIntervalsSource(minuend, subtrahend, DifferenceIntervalFunction.NOT_CONTAINING);
+  }
+
+  /**
+   * Create a containing {@link IntervalsSource}
+   *
+   * Returns intervals from the big source that contain one or more intervals from
+   * the small source
+   *
+   * @param big     the {@link IntervalsSource} to filter
+   * @param small   the {@link IntervalsSource} to filter by
+   */
+  public static IntervalsSource containing(IntervalsSource big, IntervalsSource small) {
+    return new ConjunctionIntervalsSource(Arrays.asList(big, small), IntervalFunction.CONTAINING);
+  }
+
+  /**
+   * Create a not-contained-by {@link IntervalsSource}
+   *
+   * Returns intervals from the small {@link IntervalsSource} that do not appear within
+   * intervals from the big {@link IntervalsSource}.
+   *
+   * @param small   the {@link IntervalsSource} to filter
+   * @param big     the {@link IntervalsSource} to filter by
+   */
+  public static IntervalsSource notContainedBy(IntervalsSource small, IntervalsSource big) {
+    return new DifferenceIntervalsSource(small, big, DifferenceIntervalFunction.NOT_CONTAINED_BY);
+  }
+
+  /**
+   * Create a contained-by {@link IntervalsSource}
+   *
+   * Returns intervals from the small query that appear within intervals of the big query
+   *
+   * @param small     the {@link IntervalsSource} to filter
+   * @param big       the {@link IntervalsSource} to filter by
+   */
+  public static IntervalsSource containedBy(IntervalsSource small, IntervalsSource big) {
+    return new ConjunctionIntervalsSource(Arrays.asList(small, big), IntervalFunction.CONTAINED_BY);
+  }
+
+  // TODO: beforeQuery, afterQuery, arbitrary IntervalFunctions
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalsSource.java
new file mode 100644
index 0000000..9791ff8
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalsSource.java
@@ -0,0 +1,61 @@
+/*
+ * 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.Set;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+
+/**
+ * A helper class for {@link IntervalQuery} that provides an {@link IntervalIterator}
+ * for a given field and segment
+ *
+ * Static constructor functions for various different sources can be found in the
+ * {@link Intervals} class
+ */
+public abstract class IntervalsSource {
+
+  /**
+   * Create an {@link IntervalIterator} exposing the minimum intervals defined by this {@link IntervalsSource}
+   *
+   * Returns {@code null} if no intervals for this field exist in this segment
+   *
+   * @param field the field to read positions from
+   * @param ctx   the context for which to return the iterator
+   */
+  public abstract IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException;
+
+  /**
+   * Expert: collect {@link Term} objects from this source, to be used for top-level term scoring
+   * @param field the field to be scored
+   * @param terms a {@link Set} which terms should be added to
+   */
+  public abstract void extractTerms(String field, Set<Term> terms);
+
+  @Override
+  public abstract int hashCode();
+
+  @Override
+  public abstract boolean equals(Object other);
+
+  @Override
+  public abstract String toString();
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/00eab54f/lucene/sandbox/src/java/org/apache/lucene/search/intervals/LowpassIntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/LowpassIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/LowpassIntervalsSource.java
new file mode 100644
index 0000000..3bb469e
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/LowpassIntervalsSource.java
@@ -0,0 +1,71 @@
+/*
+ * 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.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+
+class LowpassIntervalsSource extends IntervalsSource {
+
+  final IntervalsSource in;
+  private final int maxWidth;
+
+  LowpassIntervalsSource(IntervalsSource in, int maxWidth) {
+    this.in = in;
+    this.maxWidth = maxWidth;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    LowpassIntervalsSource that = (LowpassIntervalsSource) o;
+    return maxWidth == that.maxWidth &&
+        Objects.equals(in, that.in);
+  }
+
+  @Override
+  public String toString() {
+    return "MAXWIDTH/" + maxWidth + "(" + in + ")";
+  }
+
+  @Override
+  public void extractTerms(String field, Set<Term> terms) {
+    in.extractTerms(field, terms);
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    IntervalIterator i = in.intervals(field, ctx);
+    return new IntervalFilter(i) {
+      @Override
+      protected boolean accept() {
+        return (i.end() - i.start()) + 1 <= maxWidth;
+      }
+    };
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(in, maxWidth);
+  }
+}