You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/09/07 07:49:12 UTC

[03/50] [abbrv] lucene-solr:jira/http2: LUCENE-8422: Add matches to IntervalQuery

LUCENE-8422: Add matches to IntervalQuery


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

Branch: refs/heads/jira/http2
Commit: 1acfca5ebcc4eb8600fe0fc0def160f610866f72
Parents: 74b53b1
Author: Alan Woodward <ro...@apache.org>
Authored: Fri Aug 31 08:43:20 2018 +0100
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Sep 3 21:12:01 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   5 +
 .../org/apache/lucene/search/BooleanWeight.java |   2 +-
 .../lucene/search/DisjunctionMaxQuery.java      |   2 +-
 .../lucene/search/DocValuesRewriteMethod.java   |   2 +-
 .../lucene/search/FilterMatchesIterator.java    |  74 ++++++
 .../java/org/apache/lucene/search/Matches.java  |  96 --------
 .../org/apache/lucene/search/MatchesUtils.java  | 132 +++++++++++
 .../MultiTermQueryConstantScoreWrapper.java     |   2 +-
 .../org/apache/lucene/search/PhraseWeight.java  |   2 +-
 .../org/apache/lucene/search/SynonymQuery.java  |   2 +-
 .../apache/lucene/search/TermInSetQuery.java    |   2 +-
 .../org/apache/lucene/search/TermQuery.java     |   2 +-
 .../java/org/apache/lucene/search/Weight.java   |   4 +-
 .../apache/lucene/search/spans/SpanWeight.java  |   3 +-
 .../search/join/ToParentBlockJoinQuery.java     |   3 +-
 .../org/apache/lucene/search/CoveringQuery.java |   2 +-
 .../intervals/ConjunctionIntervalsSource.java   | 113 ++++++++-
 .../intervals/DifferenceIntervalsSource.java    |  23 +-
 .../intervals/DisjunctionIntervalsSource.java   |  14 ++
 .../lucene/search/intervals/IntervalFilter.java |   2 +-
 .../search/intervals/IntervalFunction.java      |   4 +-
 .../search/intervals/IntervalIterator.java      |   5 +
 .../search/intervals/IntervalMatches.java       | 156 ++++++++++++
 .../lucene/search/intervals/IntervalQuery.java  |  20 ++
 .../lucene/search/intervals/Intervals.java      |   4 +-
 .../search/intervals/IntervalsSource.java       |  13 +
 .../intervals/LowpassIntervalsSource.java       |  16 ++
 .../MinimizingConjunctionIntervalsSource.java   | 236 +++++++++++++++++++
 .../search/intervals/TermIntervalsSource.java   |  66 ++++++
 .../lucene/search/intervals/TestIntervals.java  | 232 ++++++++++++++++--
 30 files changed, 1095 insertions(+), 144 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index a2d88cd..c2a9e2e 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -198,6 +198,9 @@ API Changes:
 * LUCENE-8471: IndexWriter.getFlushingBytes() returns how many bytes are currently
   being flushed to disk. (Alan Woodward)
 
+* LUCENE-8422: Static helper functions for Matches and MatchesIterator implementations
+  have been moved from Matches to MatchesUtils (Alan Woodward)
+
 Bug Fixes:
 
 * LUCENE-8445: Tighten condition when two planes are identical to prevent constructing
@@ -278,6 +281,8 @@ Improvements
   sorted, even if hits still need to be visited to compute the hit count.
   (Nikolay Khitrin)
 
+* LUCENE-8422: IntervalQuery now returns useful Matches (Alan Woodward)
+
 Other:
 
 * LUCENE-8366: Upgrade to ICU 62.1. Emoji handling now uses Unicode 11's

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
index 5f20a6b..e0ea044 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
@@ -151,7 +151,7 @@ final class BooleanWeight extends Weight {
     if (shouldMatchCount < minShouldMatch) {
       return null;
     }
-    return Matches.fromSubMatches(matches);
+    return MatchesUtils.fromSubMatches(matches);
   }
 
   static BulkScorer disableScoring(final BulkScorer scorer) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
index 3cdc0bc..43b42b5 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
@@ -127,7 +127,7 @@ public final class DisjunctionMaxQuery extends Query implements Iterable<Query>
           mis.add(mi);
         }
       }
-      return Matches.fromSubMatches(mis);
+      return MatchesUtils.fromSubMatches(mis);
     }
 
     /** Create the scorer used to score our associated DisjunctionMaxQuery */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java b/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
index fe6d551..6e2ec37 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
@@ -78,7 +78,7 @@ public final class DocValuesRewriteMethod extends MultiTermQuery.RewriteMethod {
         @Override
         public Matches matches(LeafReaderContext context, int doc) throws IOException {
           final SortedSetDocValues fcsi = DocValues.getSortedSet(context.reader(), query.field);
-          return Matches.forField(query.field, () -> DisjunctionMatchesIterator.fromTermsEnum(context, doc, query, query.field, getTermsEnum(fcsi)));
+          return MatchesUtils.forField(query.field, () -> DisjunctionMatchesIterator.fromTermsEnum(context, doc, query, query.field, getTermsEnum(fcsi)));
         }
 
         private TermsEnum getTermsEnum(SortedSetDocValues fcsi) throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/core/src/java/org/apache/lucene/search/FilterMatchesIterator.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/FilterMatchesIterator.java b/lucene/core/src/java/org/apache/lucene/search/FilterMatchesIterator.java
new file mode 100644
index 0000000..214cf61
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/FilterMatchesIterator.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;
+
+import java.io.IOException;
+
+/**
+ * A MatchesIterator that delegates all calls to another MatchesIterator
+ */
+public abstract class FilterMatchesIterator implements MatchesIterator {
+
+  /**
+   * The delegate
+   */
+  protected final MatchesIterator in;
+
+  /**
+   * Create a new FilterMatchesIterator
+   * @param in the delegate
+   */
+  protected FilterMatchesIterator(MatchesIterator in) {
+    this.in = in;
+  }
+
+  @Override
+  public boolean next() throws IOException {
+    return in.next();
+  }
+
+  @Override
+  public int startPosition() {
+    return in.startPosition();
+  }
+
+  @Override
+  public int endPosition() {
+    return in.endPosition();
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return in.startOffset();
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return in.endOffset();
+  }
+
+  @Override
+  public MatchesIterator getSubMatches() throws IOException {
+    return in.getSubMatches();
+  }
+
+  @Override
+  public Query getQuery() {
+    return in.getQuery();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/core/src/java/org/apache/lucene/search/Matches.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/Matches.java b/lucene/core/src/java/org/apache/lucene/search/Matches.java
index c832e74..43cffa6 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Matches.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Matches.java
@@ -18,15 +18,6 @@
 package org.apache.lucene.search;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Objects;
-import java.util.stream.Collectors;
-import java.util.stream.StreamSupport;
-
-import org.apache.lucene.util.IOSupplier;
 
 /**
  * Reports the positions and optionally offsets of all matching terms in a query
@@ -46,91 +37,4 @@ public interface Matches extends Iterable<String> {
    */
   MatchesIterator getMatches(String field) throws IOException;
 
-  /**
-   * Indicates a match with no term positions, for example on a Point or DocValues field,
-   * or a field indexed as docs and freqs only
-   */
-  Matches MATCH_WITH_NO_TERMS = new Matches() {
-    @Override
-    public Iterator<String> iterator() {
-      return Collections.emptyIterator();
-    }
-
-    @Override
-    public MatchesIterator getMatches(String field) {
-      return null;
-    }
-  };
-
-  /**
-   * Amalgamate a collection of {@link Matches} into a single object
-   */
-  static Matches fromSubMatches(List<Matches> subMatches) {
-    if (subMatches == null || subMatches.size() == 0) {
-      return null;
-    }
-    List<Matches> sm = subMatches.stream().filter(m -> m != MATCH_WITH_NO_TERMS).collect(Collectors.toList());
-    if (sm.size() == 0) {
-      return MATCH_WITH_NO_TERMS;
-    }
-    if (sm.size() == 1) {
-      return sm.get(0);
-    }
-
-    return new Matches() {
-      @Override
-      public MatchesIterator getMatches(String field) throws IOException {
-        List<MatchesIterator> subIterators = new ArrayList<>(sm.size());
-        for (Matches m : sm) {
-          MatchesIterator it = m.getMatches(field);
-          if (it != null) {
-            subIterators.add(it);
-          }
-        }
-        return DisjunctionMatchesIterator.fromSubIterators(subIterators);
-      }
-
-      @Override
-      public Iterator<String> iterator() {
-        // for each sub-match, iterate its fields (it's an Iterable of the fields), and return the distinct set
-        return sm.stream().flatMap(m -> StreamSupport.stream(m.spliterator(), false)).distinct().iterator();
-      }
-    };
-  }
-
-  /**
-   * Create a Matches for a single field
-   */
-  static Matches forField(String field, IOSupplier<MatchesIterator> mis) throws IOException {
-
-    // The indirection here, using a Supplier object rather than a MatchesIterator
-    // directly, is to allow for multiple calls to Matches.getMatches() to return
-    // new iterators.  We still need to call MatchesIteratorSupplier.get() eagerly
-    // to work out if we have a hit or not.
-
-    MatchesIterator mi = mis.get();
-    if (mi == null) {
-      return null;
-    }
-    return new Matches() {
-      boolean cached = true;
-      @Override
-      public MatchesIterator getMatches(String f) throws IOException {
-        if (Objects.equals(field, f) == false) {
-          return null;
-        }
-        if (cached == false) {
-          return mis.get();
-        }
-        cached = false;
-        return mi;
-      }
-
-      @Override
-      public Iterator<String> iterator() {
-        return Collections.singleton(field).iterator();
-      }
-    };
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/core/src/java/org/apache/lucene/search/MatchesUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MatchesUtils.java b/lucene/core/src/java/org/apache/lucene/search/MatchesUtils.java
new file mode 100644
index 0000000..a8438ae
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/search/MatchesUtils.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import org.apache.lucene.util.IOSupplier;
+
+/**
+ * Contains static functions that aid the implementation of {@link Matches} and
+ * {@link MatchesIterator} interfaces.
+ */
+public final class MatchesUtils {
+
+  private MatchesUtils() {}   // static functions only
+
+  /**
+   * Indicates a match with no term positions, for example on a Point or DocValues field,
+   * or a field indexed as docs and freqs only
+   */
+  public static final Matches MATCH_WITH_NO_TERMS = new Matches() {
+    @Override
+    public Iterator<String> iterator() {
+      return Collections.emptyIterator();
+    }
+
+    @Override
+    public MatchesIterator getMatches(String field) {
+      return null;
+    }
+  };
+
+  /**
+   * Amalgamate a collection of {@link Matches} into a single object
+   */
+  public static Matches fromSubMatches(List<Matches> subMatches) {
+    if (subMatches == null || subMatches.size() == 0) {
+      return null;
+    }
+    List<Matches> sm = subMatches.stream().filter(m -> m != MATCH_WITH_NO_TERMS).collect(Collectors.toList());
+    if (sm.size() == 0) {
+      return MATCH_WITH_NO_TERMS;
+    }
+    if (sm.size() == 1) {
+      return sm.get(0);
+    }
+
+    return new Matches() {
+      @Override
+      public MatchesIterator getMatches(String field) throws IOException {
+        List<MatchesIterator> subIterators = new ArrayList<>(sm.size());
+        for (Matches m : sm) {
+          MatchesIterator it = m.getMatches(field);
+          if (it != null) {
+            subIterators.add(it);
+          }
+        }
+        return DisjunctionMatchesIterator.fromSubIterators(subIterators);
+      }
+
+      @Override
+      public Iterator<String> iterator() {
+        // for each sub-match, iterate its fields (it's an Iterable of the fields), and return the distinct set
+        return sm.stream().flatMap(m -> StreamSupport.stream(m.spliterator(), false)).distinct().iterator();
+      }
+    };
+  }
+
+  /**
+   * Create a Matches for a single field
+   */
+  public static Matches forField(String field, IOSupplier<MatchesIterator> mis) throws IOException {
+
+    // The indirection here, using a Supplier object rather than a MatchesIterator
+    // directly, is to allow for multiple calls to Matches.getMatches() to return
+    // new iterators.  We still need to call MatchesIteratorSupplier.get() eagerly
+    // to work out if we have a hit or not.
+
+    MatchesIterator mi = mis.get();
+    if (mi == null) {
+      return null;
+    }
+    return new Matches() {
+      boolean cached = true;
+      @Override
+      public MatchesIterator getMatches(String f) throws IOException {
+        if (Objects.equals(field, f) == false) {
+          return null;
+        }
+        if (cached == false) {
+          return mis.get();
+        }
+        cached = false;
+        return mi;
+      }
+
+      @Override
+      public Iterator<String> iterator() {
+        return Collections.singleton(field).iterator();
+      }
+    };
+  }
+
+  /**
+   * Create a MatchesIterator that iterates in order over all matches in a set of subiterators
+   */
+  public static MatchesIterator disjunction(List<MatchesIterator> subMatches) throws IOException {
+    return DisjunctionMatchesIterator.fromSubIterators(subMatches);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java b/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
index 1e59ef9..17bdb56 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
@@ -211,7 +211,7 @@ final class MultiTermQueryConstantScoreWrapper<Q extends MultiTermQuery> extends
         if (terms.hasPositions() == false) {
           return super.matches(context, doc);
         }
-        return Matches.forField(query.field, () -> DisjunctionMatchesIterator.fromTermsEnum(context, doc, query, query.field, query.getTermsEnum(terms)));
+        return MatchesUtils.forField(query.field, () -> DisjunctionMatchesIterator.fromTermsEnum(context, doc, query, query.field, query.getTermsEnum(terms)));
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/core/src/java/org/apache/lucene/search/PhraseWeight.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PhraseWeight.java b/lucene/core/src/java/org/apache/lucene/search/PhraseWeight.java
index 94e57d6..30b01db 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PhraseWeight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PhraseWeight.java
@@ -85,7 +85,7 @@ abstract class PhraseWeight extends Weight {
 
   @Override
   public Matches matches(LeafReaderContext context, int doc) throws IOException {
-    return Matches.forField(field, () -> {
+    return MatchesUtils.forField(field, () -> {
       PhraseMatcher matcher = getPhraseMatcher(context, true);
       if (matcher == null || matcher.approximation.advance(doc) != doc) {
         return null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java b/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
index 97e127a..25205ad 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SynonymQuery.java
@@ -176,7 +176,7 @@ public final class SynonymQuery extends Query {
       if (terms == null || terms.hasPositions() == false) {
         return super.matches(context, doc);
       }
-      return Matches.forField(field, () -> DisjunctionMatchesIterator.fromTerms(context, doc, getQuery(), field, Arrays.asList(SynonymQuery.this.terms)));
+      return MatchesUtils.forField(field, () -> DisjunctionMatchesIterator.fromTerms(context, doc, getQuery(), field, Arrays.asList(SynonymQuery.this.terms)));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
index 3ec7380..896f20f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
@@ -226,7 +226,7 @@ public class TermInSetQuery extends Query implements Accountable {
         if (terms == null || terms.hasPositions() == false) {
           return super.matches(context, doc);
         }
-        return Matches.forField(field, () -> DisjunctionMatchesIterator.fromTermsEnum(context, doc, getQuery(), field, termData.iterator()));
+        return MatchesUtils.forField(field, () -> DisjunctionMatchesIterator.fromTermsEnum(context, doc, getQuery(), field, termData.iterator()));
       }
 
       /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
index 3fa4aa7..25c77c2 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
@@ -89,7 +89,7 @@ public class TermQuery extends Query {
       if (context.reader().terms(term.field()).hasPositions() == false) {
         return super.matches(context, doc);
       }
-      return Matches.forField(term.field(), () -> {
+      return MatchesUtils.forField(term.field(), () -> {
         PostingsEnum pe = te.postings(null, PostingsEnum.OFFSETS);
         if (pe.advance(doc) != doc) {
           return null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/core/src/java/org/apache/lucene/search/Weight.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/Weight.java b/lucene/core/src/java/org/apache/lucene/search/Weight.java
index 0e95aab..d2fac4c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Weight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Weight.java
@@ -74,7 +74,7 @@ public abstract class Weight implements SegmentCacheable {
    * does not match the parent query
    *
    * A query match that contains no position information (for example, a Point or
-   * DocValues query) will return {@link Matches#MATCH_WITH_NO_TERMS}
+   * DocValues query) will return {@link MatchesUtils#MATCH_WITH_NO_TERMS}
    *
    * @param context the reader's context to create the {@link Matches} for
    * @param doc     the document's id relative to the given context's reader
@@ -96,7 +96,7 @@ public abstract class Weight implements SegmentCacheable {
         return null;
       }
     }
-    return Matches.MATCH_WITH_NO_TERMS;
+    return MatchesUtils.MATCH_WITH_NO_TERMS;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java b/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
index edde8bb..c33235f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
@@ -32,6 +32,7 @@ import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.LeafSimScorer;
 import org.apache.lucene.search.Matches;
 import org.apache.lucene.search.MatchesIterator;
+import org.apache.lucene.search.MatchesUtils;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TermStatistics;
@@ -177,7 +178,7 @@ public abstract class SpanWeight extends Weight {
 
   @Override
   public Matches matches(LeafReaderContext context, int doc) throws IOException {
-    return Matches.forField(field, () -> {
+    return MatchesUtils.forField(field, () -> {
       Spans spans = getSpans(context, Postings.OFFSETS);
       if (spans == null || spans.advance(doc) != doc) {
         return null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
index 04e8959..8af14cb 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
@@ -29,6 +29,7 @@ import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.FilterWeight;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Matches;
+import org.apache.lucene.search.MatchesUtils;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.ScorerSupplier;
@@ -172,7 +173,7 @@ public class ToParentBlockJoinQuery extends Query {
           return null;
         }
       }
-      return Matches.MATCH_WITH_NO_TERMS;
+      return MatchesUtils.MATCH_WITH_NO_TERMS;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java
index 8d6836b..fd89888 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/CoveringQuery.java
@@ -155,7 +155,7 @@ public final class CoveringQuery extends Query {
       if (matchCount < minimumNumberMatch) {
         return null;
       }
-      return Matches.fromSubMatches(subMatches);
+      return MatchesUtils.fromSubMatches(subMatches);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/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
index d2805c9..6cbfada 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalsSource.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/ConjunctionIntervalsSource.java
@@ -26,11 +26,15 @@ import java.util.stream.Collectors;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.FilterMatchesIterator;
+import org.apache.lucene.search.MatchesIterator;
+import org.apache.lucene.search.MatchesUtils;
+import org.apache.lucene.search.Query;
 
 class ConjunctionIntervalsSource extends IntervalsSource {
 
-  final List<IntervalsSource> subSources;
-  final IntervalFunction function;
+  protected final List<IntervalsSource> subSources;
+  protected final IntervalFunction function;
 
   ConjunctionIntervalsSource(List<IntervalsSource> subSources, IntervalFunction function) {
     this.subSources = subSources;
@@ -71,7 +75,112 @@ class ConjunctionIntervalsSource extends IntervalsSource {
   }
 
   @Override
+  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    List<MatchesIterator> subs = new ArrayList<>();
+    for (IntervalsSource source : subSources) {
+      MatchesIterator mi = source.matches(field, ctx, doc);
+      if (mi == null) {
+        return null;
+      }
+      subs.add(mi);
+    }
+    IntervalIterator it = function.apply(subs.stream().map(m -> IntervalMatches.wrapMatches(m, doc)).collect(Collectors.toList()));
+    if (it.advance(doc) != doc) {
+      return null;
+    }
+    if (it.nextInterval() == IntervalIterator.NO_MORE_INTERVALS) {
+      return null;
+    }
+    return new ConjunctionMatchesIterator(it, subs);
+  }
+
+  @Override
   public int hashCode() {
     return Objects.hash(subSources, function);
   }
+
+  private static class ConjunctionMatchesIterator implements MatchesIterator {
+
+    final IntervalIterator iterator;
+    final List<MatchesIterator> subs;
+    boolean cached = true;
+
+    private ConjunctionMatchesIterator(IntervalIterator iterator, List<MatchesIterator> subs) {
+      this.iterator = iterator;
+      this.subs = subs;
+    }
+
+    @Override
+    public boolean next() throws IOException {
+      if (cached) {
+        cached = false;
+        return true;
+      }
+      return iterator.nextInterval() != IntervalIterator.NO_MORE_INTERVALS;
+    }
+
+    @Override
+    public int startPosition() {
+      return iterator.start();
+    }
+
+    @Override
+    public int endPosition() {
+      return iterator.end();
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      int start = Integer.MAX_VALUE;
+      for (MatchesIterator s : subs) {
+        start = Math.min(start, s.startOffset());
+      }
+      return start;
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      int end = -1;
+      for (MatchesIterator s : subs) {
+        end = Math.max(end, s.endOffset());
+      }
+      return end;
+    }
+
+    @Override
+    public MatchesIterator getSubMatches() throws IOException {
+      List<MatchesIterator> subMatches = new ArrayList<>();
+      for (MatchesIterator mi : subs) {
+        MatchesIterator sub = mi.getSubMatches();
+        if (sub == null) {
+          sub = new SingletonMatchesIterator(mi);
+        }
+        subMatches.add(sub);
+      }
+      return MatchesUtils.disjunction(subMatches);
+    }
+
+    @Override
+    public Query getQuery() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  private static class SingletonMatchesIterator extends FilterMatchesIterator {
+
+    boolean exhausted = false;
+
+    SingletonMatchesIterator(MatchesIterator in) {
+      super(in);
+    }
+
+    @Override
+    public boolean next() {
+      if (exhausted) {
+        return false;
+      }
+      return exhausted = true;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/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
index 316b6ff..e4b7fd9 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalsSource.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalsSource.java
@@ -23,14 +23,15 @@ import java.util.Set;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.MatchesIterator;
 
 class DifferenceIntervalsSource extends IntervalsSource {
 
-  final IntervalsSource minuend;
-  final IntervalsSource subtrahend;
-  final DifferenceIntervalFunction function;
+  private final IntervalsSource minuend;
+  private final IntervalsSource subtrahend;
+  private final DifferenceIntervalFunction function;
 
-  public DifferenceIntervalsSource(IntervalsSource minuend, IntervalsSource subtrahend, DifferenceIntervalFunction function) {
+  DifferenceIntervalsSource(IntervalsSource minuend, IntervalsSource subtrahend, DifferenceIntervalFunction function) {
     this.minuend = minuend;
     this.subtrahend = subtrahend;
     this.function = function;
@@ -48,6 +49,20 @@ class DifferenceIntervalsSource extends IntervalsSource {
   }
 
   @Override
+  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    MatchesIterator minIt = minuend.matches(field, ctx, doc);
+    if (minIt == null) {
+      return null;
+    }
+    MatchesIterator subIt = subtrahend.matches(field, ctx, doc);
+    if (subIt == null) {
+      return minIt;
+    }
+    IntervalIterator difference = function.apply(IntervalMatches.wrapMatches(minIt, doc), IntervalMatches.wrapMatches(subIt, doc));
+    return IntervalMatches.asMatches(difference, minIt, doc);
+  }
+
+  @Override
   public boolean equals(Object o) {
     if (this == o) return true;
     if (o == null || getClass() != o.getClass()) return false;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/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
index c7c7166..79c1bcf 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionIntervalsSource.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DisjunctionIntervalsSource.java
@@ -27,6 +27,8 @@ 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.search.MatchesIterator;
+import org.apache.lucene.search.MatchesUtils;
 import org.apache.lucene.util.PriorityQueue;
 
 class DisjunctionIntervalsSource extends IntervalsSource {
@@ -52,6 +54,18 @@ class DisjunctionIntervalsSource extends IntervalsSource {
   }
 
   @Override
+  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    List<MatchesIterator> subMatches = new ArrayList<>();
+    for (IntervalsSource subSource : subSources) {
+      MatchesIterator mi = subSource.matches(field, ctx, doc);
+      if (mi != null) {
+        subMatches.add(mi);
+      }
+    }
+    return MatchesUtils.disjunction(subMatches);
+  }
+
+  @Override
   public boolean equals(Object o) {
     if (this == o) return true;
     if (o == null || getClass() != o.getClass()) return false;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/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
index 7571fc2..a501b49 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFilter.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFilter.java
@@ -25,7 +25,7 @@ import java.util.Objects;
  */
 public abstract class IntervalFilter extends IntervalIterator {
 
-  private final IntervalIterator in;
+  protected final IntervalIterator in;
 
   /**
    * Create a new filter

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/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
index a6ce918..787574e 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFunction.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalFunction.java
@@ -341,7 +341,7 @@ abstract class IntervalFunction {
 
         @Override
         protected void reset() throws IOException {
-          bpos = true;
+          bpos = b.nextInterval() != NO_MORE_INTERVALS;
         }
       };
     }
@@ -388,7 +388,7 @@ abstract class IntervalFunction {
 
         @Override
         protected void reset() throws IOException {
-          bpos = true;
+          bpos = b.nextInterval() != NO_MORE_INTERVALS;
         }
       };
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/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
index 2428720..613cb78 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalIterator.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalIterator.java
@@ -74,4 +74,9 @@ public abstract class IntervalIterator extends DocIdSetIterator {
    */
   public abstract float matchCost();
 
+  @Override
+  public String toString() {
+    return docID() + ":[" + start() + "->" + end() + "]";
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalMatches.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalMatches.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalMatches.java
new file mode 100644
index 0000000..89f115d
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalMatches.java
@@ -0,0 +1,156 @@
+/*
+ * 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.MatchesIterator;
+import org.apache.lucene.search.Query;
+
+final class IntervalMatches {
+
+  static MatchesIterator asMatches(IntervalIterator iterator, MatchesIterator source, int doc) throws IOException {
+    if (source == null) {
+      return null;
+    }
+    if (iterator.advance(doc) != doc) {
+      return null;
+    }
+    if (iterator.nextInterval() == IntervalIterator.NO_MORE_INTERVALS) {
+      return null;
+    }
+    return new MatchesIterator() {
+
+      boolean cached = true;
+
+      @Override
+      public boolean next() throws IOException {
+        if (cached) {
+          cached = false;
+          return true;
+        }
+        return iterator.nextInterval() != IntervalIterator.NO_MORE_INTERVALS;
+      }
+
+      @Override
+      public int startPosition() {
+        return source.startPosition();
+      }
+
+      @Override
+      public int endPosition() {
+        return source.endPosition();
+      }
+
+      @Override
+      public int startOffset() throws IOException {
+        return source.startOffset();
+      }
+
+      @Override
+      public int endOffset() throws IOException {
+        return source.endOffset();
+      }
+
+      @Override
+      public MatchesIterator getSubMatches() throws IOException {
+        return source.getSubMatches();
+      }
+
+      @Override
+      public Query getQuery() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+  enum State { UNPOSITIONED, ITERATING, EXHAUSTED }
+
+  static IntervalIterator wrapMatches(MatchesIterator mi, int doc) {
+    return new IntervalIterator() {
+
+      State state = State.UNPOSITIONED;
+
+      @Override
+      public int start() {
+        assert state == State.ITERATING;
+        return mi.startPosition();
+      }
+
+      @Override
+      public int end() {
+        assert state == State.ITERATING;
+        return mi.endPosition();
+      }
+
+      @Override
+      public int nextInterval() throws IOException {
+        assert state == State.ITERATING;
+        if (mi.next()) {
+          return mi.startPosition();
+        }
+        return NO_MORE_INTERVALS;
+      }
+
+      @Override
+      public float matchCost() {
+        return 1;
+      }
+
+      @Override
+      public int docID() {
+        switch (state) {
+          case UNPOSITIONED:
+            return -1;
+          case ITERATING:
+            return doc;
+          case EXHAUSTED:
+        }
+        return NO_MORE_DOCS;
+      }
+
+      @Override
+      public int nextDoc() {
+        switch (state) {
+          case UNPOSITIONED:
+            state = State.ITERATING;
+            return doc;
+          case ITERATING:
+            state = State.EXHAUSTED;
+          case EXHAUSTED:
+        }
+        return NO_MORE_DOCS;
+      }
+
+      @Override
+      public int advance(int target) {
+        if (target == doc) {
+          state = State.ITERATING;
+          return doc;
+        }
+        state = State.EXHAUSTED;
+        return NO_MORE_DOCS;
+      }
+
+      @Override
+      public long cost() {
+        return 1;
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/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
index c1125c2..4e2569c 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalQuery.java
@@ -27,8 +27,12 @@ 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.FilterMatchesIterator;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.LeafSimScorer;
+import org.apache.lucene.search.Matches;
+import org.apache.lucene.search.MatchesIterator;
+import org.apache.lucene.search.MatchesUtils;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Scorer;
@@ -137,6 +141,22 @@ public final class IntervalQuery extends Query {
     }
 
     @Override
+    public Matches matches(LeafReaderContext context, int doc) throws IOException {
+      return MatchesUtils.forField(field, () -> {
+        MatchesIterator mi = intervalsSource.matches(field, context, doc);
+        if (mi == null) {
+          return null;
+        }
+        return new FilterMatchesIterator(mi) {
+          @Override
+          public Query getQuery() {
+            return new IntervalQuery(field, intervalsSource);
+          }
+        };
+      });
+    }
+
+    @Override
     public Scorer scorer(LeafReaderContext context) throws IOException {
       IntervalIterator intervals = intervalsSource.intervals(field, context);
       if (intervals == null)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/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
index 32ea6da..e9f1bd2 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/Intervals.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/Intervals.java
@@ -92,7 +92,7 @@ public final class Intervals {
    * @param subSources  an ordered set of {@link IntervalsSource} objects
    */
   public static IntervalsSource ordered(IntervalsSource... subSources) {
-    return new ConjunctionIntervalsSource(Arrays.asList(subSources), IntervalFunction.ORDERED);
+    return new MinimizingConjunctionIntervalsSource(Arrays.asList(subSources), IntervalFunction.ORDERED);
   }
 
   /**
@@ -115,7 +115,7 @@ public final class Intervals {
    * @param allowOverlaps whether or not the sources should be allowed to overlap in a hit
    */
   public static IntervalsSource unordered(boolean allowOverlaps, IntervalsSource... subSources) {
-    return new ConjunctionIntervalsSource(Arrays.asList(subSources),
+    return new MinimizingConjunctionIntervalsSource(Arrays.asList(subSources),
         allowOverlaps ? IntervalFunction.UNORDERED : IntervalFunction.UNORDERED_NO_OVERLAP);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/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
index 9791ff8..14d9471 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalsSource.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalsSource.java
@@ -22,6 +22,7 @@ import java.util.Set;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.MatchesIterator;
 
 /**
  * A helper class for {@link IntervalQuery} that provides an {@link IntervalIterator}
@@ -43,6 +44,18 @@ public abstract class IntervalsSource {
   public abstract IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException;
 
   /**
+   * Return a {@link MatchesIterator} over the intervals defined by this {@link IntervalsSource} for a
+   * given document and field
+   *
+   * Returns {@code null} if no intervals exist in the given document and field
+   *
+   * @param field the field to read positions from
+   * @param ctx   the document's context
+   * @param doc   the document to return matches for
+   */
+  public abstract MatchesIterator matches(String field, LeafReaderContext ctx, int doc) 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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/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
index 4d7846c..61cb1fc 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/LowpassIntervalsSource.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/LowpassIntervalsSource.java
@@ -23,6 +23,7 @@ import java.util.Set;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.MatchesIterator;
 
 class LowpassIntervalsSource extends IntervalsSource {
 
@@ -68,6 +69,21 @@ class LowpassIntervalsSource extends IntervalsSource {
   }
 
   @Override
+  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    MatchesIterator mi = in.matches(field, ctx, doc);
+    if (mi == null) {
+      return null;
+    }
+    IntervalIterator filtered = new IntervalFilter(IntervalMatches.wrapMatches(mi, doc)) {
+      @Override
+      protected boolean accept() {
+        return (this.in.end() - this.in.start()) + 1 <= maxWidth;
+      }
+    };
+    return IntervalMatches.asMatches(filtered, mi, doc);
+  }
+
+  @Override
   public int hashCode() {
     return Objects.hash(in, maxWidth);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/lucene/sandbox/src/java/org/apache/lucene/search/intervals/MinimizingConjunctionIntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/MinimizingConjunctionIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/MinimizingConjunctionIntervalsSource.java
new file mode 100644
index 0000000..669acae
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/MinimizingConjunctionIntervalsSource.java
@@ -0,0 +1,236 @@
+/*
+ * 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.stream.Collectors;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.FilterMatchesIterator;
+import org.apache.lucene.search.MatchesIterator;
+import org.apache.lucene.search.MatchesUtils;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.ArrayUtil;
+
+/**
+ * A ConjunctionIntervalsSource that attempts to minimize its internal intervals by
+ * eagerly advancing its first subinterval
+ *
+ * Uses caching to expose matches after its first subinterval has been moved on
+ */
+class MinimizingConjunctionIntervalsSource extends ConjunctionIntervalsSource {
+
+  MinimizingConjunctionIntervalsSource(List<IntervalsSource> subSources, IntervalFunction function) {
+    super(subSources, function);
+  }
+
+  @Override
+  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    List<CacheingMatchesIterator> subs = new ArrayList<>();
+    for (IntervalsSource source : subSources) {
+      MatchesIterator mi = source.matches(field, ctx, doc);
+      if (mi == null) {
+        return null;
+      }
+      subs.add(new CacheingMatchesIterator(mi));
+    }
+    IntervalIterator it = function.apply(subs.stream().map(m -> IntervalMatches.wrapMatches(m, doc)).collect(Collectors.toList()));
+    if (it.advance(doc) != doc) {
+      return null;
+    }
+    if (it.nextInterval() == IntervalIterator.NO_MORE_INTERVALS) {
+      return null;
+    }
+    return new ConjunctionMatchesIterator(it, subs);
+  }
+
+  private static class ConjunctionMatchesIterator implements MatchesIterator {
+
+    final IntervalIterator iterator;
+    final List<CacheingMatchesIterator> subs;
+    boolean cached = true;
+
+    private ConjunctionMatchesIterator(IntervalIterator iterator, List<CacheingMatchesIterator> subs) {
+      this.iterator = iterator;
+      this.subs = subs;
+    }
+
+    @Override
+    public boolean next() throws IOException {
+      if (cached) {
+        cached = false;
+        return true;
+      }
+      return iterator.nextInterval() != IntervalIterator.NO_MORE_INTERVALS;
+    }
+
+    @Override
+    public int startPosition() {
+      return iterator.start();
+    }
+
+    @Override
+    public int endPosition() {
+      return iterator.end();
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      int start = Integer.MAX_VALUE;
+      int endPos = endPosition();
+      for (CacheingMatchesIterator s : subs) {
+        start = Math.min(start, s.startOffset(endPos));
+      }
+      return start;
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      int end = 0;
+      int endPos = endPosition();
+      for (CacheingMatchesIterator s : subs) {
+        end = Math.max(end, s.endOffset(endPos));
+      }
+      return end;
+    }
+
+    @Override
+    public MatchesIterator getSubMatches() throws IOException {
+      List<MatchesIterator> mis = new ArrayList<>();
+      int endPos = endPosition();
+      for (CacheingMatchesIterator s : subs) {
+        mis.add(s.getSubMatches(endPos));
+      }
+      return MatchesUtils.disjunction(mis);
+    }
+
+    @Override
+    public Query getQuery() {
+      return null;
+    }
+  }
+
+  private static class CacheingMatchesIterator extends FilterMatchesIterator {
+
+    boolean positioned = false;
+    int posAndOffsets[] = new int[16];
+    int count = 0;
+
+    CacheingMatchesIterator(MatchesIterator in) {
+      super(in);
+    }
+
+    private void cache() throws IOException {
+      count = 0;
+      MatchesIterator mi = in.getSubMatches();
+      if (mi == null) {
+        count = 1;
+        posAndOffsets[0] = in.startPosition();
+        posAndOffsets[1] = in.endPosition();
+        posAndOffsets[2] = in.startOffset();
+        posAndOffsets[3] = in.endOffset();
+      }
+      else {
+        while (mi.next()) {
+          if (count * 4 >= posAndOffsets.length) {
+            posAndOffsets = ArrayUtil.grow(posAndOffsets, (count + 1) * 4);
+          }
+          posAndOffsets[count * 4] = mi.startPosition();
+          posAndOffsets[count * 4 + 1] = mi.endPosition();
+          posAndOffsets[count * 4 + 2] = mi.startOffset();
+          posAndOffsets[count * 4 + 3] = mi.endOffset();
+          count++;
+        }
+      }
+    }
+
+    @Override
+    public boolean next() throws IOException {
+      if (positioned == false) {
+        positioned = true;
+      }
+      else {
+        cache();
+      }
+      return in.next();
+    }
+
+    int startOffset(int endPos) throws IOException {
+      if (endPosition() <= endPos) {
+        return in.startOffset();
+      }
+      return posAndOffsets[2];
+    }
+
+    int endOffset(int endPos) throws IOException {
+      if (endPosition() <= endPos) {
+        return in.endOffset();
+      }
+      return posAndOffsets[count * 4 + 3];
+    }
+
+    MatchesIterator getSubMatches(int endPos) throws IOException {
+      if (endPosition() <= endPos) {
+        cache();
+      }
+      return new MatchesIterator() {
+
+        int upto = -1;
+
+        @Override
+        public boolean next() {
+          upto++;
+          return upto < count;
+        }
+
+        @Override
+        public int startPosition() {
+          return posAndOffsets[upto * 4];
+        }
+
+        @Override
+        public int endPosition() {
+          return posAndOffsets[upto * 4 + 1];
+        }
+
+        @Override
+        public int startOffset() {
+          return posAndOffsets[upto * 4 + 2];
+        }
+
+        @Override
+        public int endOffset() {
+          return posAndOffsets[upto * 4 + 3];
+        }
+
+        @Override
+        public MatchesIterator getSubMatches() {
+          return null;
+        }
+
+        @Override
+        public Query getQuery() {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/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
index 81578ef..27e5b79 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/TermIntervalsSource.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/TermIntervalsSource.java
@@ -30,6 +30,8 @@ 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.MatchesIterator;
+import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TwoPhaseIterator;
 import org.apache.lucene.util.BytesRef;
 
@@ -125,6 +127,70 @@ class TermIntervalsSource extends IntervalsSource {
   }
 
   @Override
+  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) 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();
+    if (te.seekExact(term) == false) {
+      return null;
+    }
+    PostingsEnum pe = te.postings(null, PostingsEnum.OFFSETS);
+    if (pe.advance(doc) != doc) {
+      return null;
+    }
+    return new MatchesIterator() {
+
+      int upto = pe.freq();
+      int pos = -1;
+
+      @Override
+      public boolean next() throws IOException {
+        if (upto <= 0) {
+          pos = IntervalIterator.NO_MORE_INTERVALS;
+          return false;
+        }
+        upto--;
+        pos = pe.nextPosition();
+        return true;
+      }
+
+      @Override
+      public int startPosition() {
+        return pos;
+      }
+
+      @Override
+      public int endPosition() {
+        return pos;
+      }
+
+      @Override
+      public int startOffset() throws IOException {
+        return pe.startOffset();
+      }
+
+      @Override
+      public int endOffset() throws IOException {
+        return pe.endOffset();
+      }
+
+      @Override
+      public MatchesIterator getSubMatches() {
+        return null;
+      }
+
+      @Override
+      public Query getQuery() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+  @Override
   public int hashCode() {
     return Objects.hash(term);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1acfca5e/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
index 33d3cd5..d7754be 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervals.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervals.java
@@ -24,15 +24,19 @@ import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
 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.IndexOptions;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchesIterator;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
@@ -41,6 +45,8 @@ import org.junit.BeforeClass;
 
 public class TestIntervals extends LuceneTestCase {
 
+  //   0         1         2         3         4         5         6         7         8         9
+  //   012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789
   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",
@@ -63,13 +69,19 @@ public class TestIntervals extends LuceneTestCase {
   private static IndexSearcher searcher;
   private static Analyzer analyzer = new StandardAnalyzer(CharArraySet.EMPTY_SET);
 
+  private static final FieldType FIELD_TYPE = new FieldType(TextField.TYPE_STORED);
+  static {
+    FIELD_TYPE.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+  }
+
   @BeforeClass
   public static void setupIndex() throws IOException {
     directory = newDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(random(), directory, newIndexWriterConfig(analyzer));
+    RandomIndexWriter writer = new RandomIndexWriter(random(), directory,
+        newIndexWriterConfig(analyzer).setMergePolicy(newLogMergePolicy()));
     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 Field("field1", field1_docs[i], FIELD_TYPE));
       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));
@@ -101,13 +113,15 @@ public class TestIntervals extends LuceneTestCase {
           assertEquals(-1, intervals.start());
           assertEquals(-1, intervals.end());
           while ((pos = intervals.nextInterval()) != IntervalIterator.NO_MORE_INTERVALS) {
-            //System.out.println(doc + ": " + intervals);
-            assertEquals("Wrong start value", expected[id][i], pos);
+            if (i >= expected[id].length) {
+              fail("Unexpected match in doc " + id + ": " + intervals);
+            }
+            assertEquals("Wrong start value in doc " + id, expected[id][i], pos);
             assertEquals("start() != pos returned from nextInterval()", expected[id][i], intervals.start());
-            assertEquals("Wrong end value", expected[id][i + 1], intervals.end());
+            assertEquals("Wrong end value in doc " + id, expected[id][i + 1], intervals.end());
             i += 2;
           }
-          assertEquals("Wrong number of endpoints", expected[id].length, i);
+          assertEquals("Wrong number of endpoints in doc " + id, expected[id].length, i);
           if (i > 0)
             matchedDocs++;
         }
@@ -119,6 +133,20 @@ public class TestIntervals extends LuceneTestCase {
     assertEquals(expectedMatchCount, matchedDocs);
   }
 
+  private MatchesIterator getMatches(IntervalsSource source, int doc, String field) throws IOException {
+    int ord = ReaderUtil.subIndex(doc, searcher.getIndexReader().leaves());
+    LeafReaderContext ctx = searcher.getIndexReader().leaves().get(ord);
+    return source.matches(field, ctx, doc - ctx.docBase);
+  }
+
+  private void assertMatch(MatchesIterator mi, int start, int end, int startOffset, int endOffset) throws IOException {
+    assertTrue(mi.next());
+    assertEquals(start, mi.startPosition());
+    assertEquals(end, mi.endPosition());
+    assertEquals(startOffset, mi.startOffset());
+    assertEquals(endOffset, mi.endOffset());
+  }
+
   public void testIntervalsOnFieldWithNoPositions() throws IOException {
     IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> {
       Intervals.term("wibble").intervals("id", searcher.getIndexReader().leaves().get(0));
@@ -127,7 +155,8 @@ public class TestIntervals extends LuceneTestCase {
   }
 
   public void testTermQueryIntervals() throws IOException {
-    checkIntervals(Intervals.term("porridge"), "field1", 4, new int[][]{
+    IntervalsSource source = Intervals.term("porridge");
+    checkIntervals(source, "field1", 4, new int[][]{
         {},
         { 1, 1, 4, 4, 7, 7 },
         { 1, 1, 4, 4, 7, 7 },
@@ -135,11 +164,18 @@ public class TestIntervals extends LuceneTestCase {
         { 1, 1, 4, 4, 7, 7 },
         { 0, 0 }
     });
+    assertNull(getMatches(source, 0, "field1"));
+    assertNull(getMatches(source, 2, "no_such_field"));
+    MatchesIterator mi = getMatches(source, 2, "field1");
+    assertMatch(mi, 1, 1, 6, 14);
+    assertMatch(mi, 4, 4, 27, 35);
+    assertMatch(mi, 7, 7, 47, 55);
+    assertFalse(mi.next());
   }
 
   public void testOrderedNearIntervals() throws IOException {
-    checkIntervals(Intervals.ordered(Intervals.term("pease"), Intervals.term("hot")),
-        "field1", 3, new int[][]{
+    IntervalsSource source = Intervals.ordered(Intervals.term("pease"), Intervals.term("hot"));
+    checkIntervals(source, "field1", 3, new int[][]{
         {},
         { 0, 2, 6, 17 },
         { 3, 5, 6, 21 },
@@ -147,10 +183,24 @@ public class TestIntervals extends LuceneTestCase {
         { 0, 2, 6, 17 },
         { }
     });
+    assertNull(getMatches(source, 3, "field1"));
+    MatchesIterator mi = getMatches(source, 4, "field1");
+    assertMatch(mi, 0, 2, 0, 18);
+    MatchesIterator sub = mi.getSubMatches();
+    assertMatch(sub, 0, 0, 0, 5);
+    assertMatch(sub, 2, 2, 15, 18);
+    assertFalse(sub.next());
+    assertMatch(mi, 6, 17, 41, 100);
+    sub = mi.getSubMatches();
+    assertMatch(sub, 6, 6, 41, 46);
+    assertMatch(sub, 17, 17, 97, 100);
+    assertFalse(sub.next());
+    assertFalse(mi.next());
   }
 
   public void testPhraseIntervals() throws IOException {
-    checkIntervals(Intervals.phrase("pease", "porridge"), "field1", 3, new int[][]{
+    IntervalsSource source = Intervals.phrase("pease", "porridge");
+    checkIntervals(source, "field1", 3, new int[][]{
         {},
         { 0, 1, 3, 4, 6, 7 },
         { 0, 1, 3, 4, 6, 7 },
@@ -158,11 +208,20 @@ public class TestIntervals extends LuceneTestCase {
         { 0, 1, 3, 4, 6, 7 },
         {}
     });
+    assertNull(getMatches(source, 0, "field1"));
+    MatchesIterator mi = getMatches(source, 1, "field1");
+    assertMatch(mi, 0, 1, 0, 14);
+    assertMatch(mi, 3, 4, 20, 34);
+    MatchesIterator sub = mi.getSubMatches();
+    assertMatch(sub, 3, 3, 20, 25);
+    assertMatch(sub, 4, 4, 26, 34);
+    assertFalse(sub.next());
+    assertMatch(mi, 6, 7, 41, 55);
   }
 
   public void testUnorderedNearIntervals() throws IOException {
-    checkIntervals(Intervals.unordered(Intervals.term("pease"), Intervals.term("hot")),
-        "field1", 4, new int[][]{
+    IntervalsSource source = Intervals.unordered(Intervals.term("pease"), Intervals.term("hot"));
+    checkIntervals(source, "field1", 4, new int[][]{
             {},
             { 0, 2, 2, 3, 6, 17 },
             { 3, 5, 5, 6, 6, 21 },
@@ -170,10 +229,21 @@ public class TestIntervals extends LuceneTestCase {
             { 0, 2, 2, 3, 6, 17 },
             {}
         });
+    assertNull(getMatches(source, 0, "field1"));
+    MatchesIterator mi = getMatches(source, 1, "field1");
+    assertMatch(mi, 0, 2, 0, 18);
+    assertMatch(mi, 2, 3, 15, 25);
+    assertMatch(mi, 6, 17, 41, 99);
+    MatchesIterator sub = mi.getSubMatches();
+    assertMatch(sub, 6, 6, 41, 46);
+    assertMatch(sub, 17, 17, 96, 99);
+    assertFalse(sub.next());
+    assertFalse(mi.next());
   }
 
   public void testIntervalDisjunction() throws IOException {
-    checkIntervals(Intervals.or(Intervals.term("pease"), Intervals.term("hot"), Intervals.term("notMatching")), "field1", 4, new int[][]{
+    IntervalsSource source = Intervals.or(Intervals.term("pease"), Intervals.term("hot"), Intervals.term("notMatching"));
+    checkIntervals(source, "field1", 4, new int[][]{
         {},
         { 0, 0, 2, 2, 3, 3, 6, 6, 17, 17},
         { 0, 0, 3, 3, 5, 5, 6, 6, 21, 21},
@@ -181,11 +251,21 @@ public class TestIntervals extends LuceneTestCase {
         { 0, 0, 2, 2, 3, 3, 6, 6, 17, 17},
         {}
     });
+    assertNull(getMatches(source, 0, "field1"));
+    MatchesIterator mi = getMatches(source, 3, "field1");
+    assertMatch(mi, 3, 3, 15, 18);
+    assertNull(mi.getSubMatches());
+    assertMatch(mi, 7, 7, 31, 36);
+    assertNull(mi.getSubMatches());
+    assertFalse(mi.next());
   }
 
   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[][]{
+    IntervalsSource source = Intervals.unordered(
+        Intervals.term("pease"),
+        Intervals.term("porridge"),
+        Intervals.or(Intervals.term("hot"), Intervals.term("cold")));
+    checkIntervals(source, "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 },
@@ -193,19 +273,33 @@ public class TestIntervals extends LuceneTestCase {
         { 0, 2, 1, 3, 2, 4, 3, 5, 4, 6, 5, 7, 6, 17 },
         {}
     });
+    assertNull(getMatches(source, 0, "field1"));
+    MatchesIterator mi = getMatches(source, 1, "field1");
+    assertMatch(mi, 0, 2, 0, 18);
+    assertMatch(mi, 1, 3, 6, 25);
+    assertMatch(mi, 2, 4, 15, 34);
+    assertMatch(mi, 3, 5, 20, 39);
+    MatchesIterator sub = mi.getSubMatches();
+    assertMatch(sub, 3, 3, 20, 25);
+    assertMatch(sub, 4, 4, 26, 34);
+    assertMatch(sub, 5, 5, 35, 39);
+    assertFalse(sub.next());
+    assertMatch(mi, 4, 6, 26, 46);
+    assertMatch(mi, 5, 7, 35, 55);
+    assertMatch(mi, 6, 17, 41, 99);
+    assertFalse(mi.next());
   }
 
   public void testNesting2() throws IOException {
-    checkIntervals(
-        Intervals.unordered(
-            Intervals.ordered(
-                Intervals.term("like"),
-                Intervals.term("it"),
-                Intervals.term("cold")
-            ),
-            Intervals.term("pease")
+    IntervalsSource source = Intervals.unordered(
+        Intervals.ordered(
+            Intervals.term("like"),
+            Intervals.term("it"),
+            Intervals.term("cold")
         ),
-        "field1", 2, new int[][]{
+        Intervals.term("pease")
+    );
+    checkIntervals(source, "field1", 2, new int[][]{
             {},
             {6, 21},
             {6, 17},
@@ -213,6 +307,16 @@ public class TestIntervals extends LuceneTestCase {
             {},
             {}
         });
+    assertNull(getMatches(source, 0, "field1"));
+    MatchesIterator it = getMatches(source, 1, "field1");
+    assertMatch(it, 6, 21, 41, 118);
+    MatchesIterator sub = it.getSubMatches();
+    assertMatch(sub, 6, 6, 41, 46);
+    assertMatch(sub, 19, 19, 106, 110);
+    assertMatch(sub, 20, 20, 111, 113);
+    assertMatch(sub, 21, 21, 114, 118);
+    assertFalse(sub.next());
+    assertFalse(it.next());
   }
 
   public void testUnorderedDistinct() throws IOException {
@@ -249,4 +353,84 @@ public class TestIntervals extends LuceneTestCase {
         });
   }
 
+  public void testContainedBy() throws IOException {
+    IntervalsSource source = Intervals.containedBy(
+        Intervals.term("porridge"),
+        Intervals.ordered(Intervals.term("pease"), Intervals.term("cold"))
+    );
+    checkIntervals(source, "field1", 3, new int[][]{
+        {},
+        { 4, 4, 7, 7 },
+        { 1, 1, 7, 7 },
+        {},
+        { 4, 4 },
+        {}
+    });
+    MatchesIterator mi = getMatches(source, 1, "field1");
+    assertMatch(mi, 4, 4, 20, 39);
+    MatchesIterator subs = mi.getSubMatches();
+    assertMatch(subs, 3, 3, 20, 25);
+    assertMatch(subs, 4, 4, 26, 34);
+    assertMatch(subs, 5, 5, 35, 39);
+    assertFalse(subs.next());
+    assertMatch(mi, 7, 7, 41, 118);
+    subs = mi.getSubMatches();
+    assertMatch(subs, 6, 6, 41, 46);
+    assertMatch(subs, 7, 7, 47, 55);
+    assertMatch(subs, 21, 21, 114, 118);
+    assertFalse(subs.next());
+    assertFalse(mi.next());
+  }
+
+  public void testContaining() throws IOException {
+    IntervalsSource source = Intervals.containing(
+        Intervals.ordered(Intervals.term("pease"), Intervals.term("cold")),
+        Intervals.term("porridge")
+    );
+    checkIntervals(source, "field1", 3, new int[][]{
+        {},
+        { 3, 5, 6, 21 },
+        { 0, 2, 6, 17 },
+        {},
+        { 3, 5 },
+        {}
+    });
+    MatchesIterator mi = getMatches(source, 1, "field1");
+    assertMatch(mi, 3, 5, 20, 39);
+    MatchesIterator subs = mi.getSubMatches();
+    assertMatch(subs, 3, 3, 20, 25);
+    assertMatch(subs, 4, 4, 26, 34);
+    assertMatch(subs, 5, 5, 35, 39);
+    assertFalse(subs.next());
+    assertMatch(mi, 6, 21, 41, 118);
+    subs = mi.getSubMatches();
+    assertMatch(subs, 6, 6, 41, 46);
+    assertMatch(subs, 7, 7, 47, 55);
+    assertMatch(subs, 21, 21, 114, 118);
+    assertFalse(subs.next());
+    assertFalse(mi.next());
+  }
+
+  public void testNotContaining() throws IOException {
+    IntervalsSource source = Intervals.notContaining(
+        Intervals.ordered(Intervals.term("porridge"), Intervals.term("pease")),
+        Intervals.term("hot")
+    );
+    checkIntervals(source, "field1", 3, new int[][]{
+        {},
+        { 4, 6 },
+        { 1, 3 },
+        {},
+        { 4, 6 },
+        {}
+    });
+    MatchesIterator mi = getMatches(source, 1, "field1");
+    assertMatch(mi, 4, 6, 26, 46);
+    MatchesIterator subs = mi.getSubMatches();
+    assertMatch(subs, 4, 4, 26, 34);
+    assertMatch(subs, 6, 6, 41, 46);
+    assertFalse(subs.next());
+    assertFalse(mi.next());
+  }
+
 }