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 2020/02/06 14:58:52 UTC

[lucene-solr] branch branch_8x updated: LUCENE-9099: Correctly handle repeats in ORDERED and UNORDERED intervals (#1097)

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new aa916ba  LUCENE-9099: Correctly handle repeats in ORDERED and UNORDERED intervals (#1097)
aa916ba is described below

commit aa916bac3c3369a461afa06e384e070657c32973
Author: Alan Woodward <ro...@apache.org>
AuthorDate: Thu Feb 6 14:44:47 2020 +0000

    LUCENE-9099: Correctly handle repeats in ORDERED and UNORDERED intervals (#1097)
    
    If you have repeating intervals in an ordered or unordered interval source, you currently
    get somewhat confusing behaviour:
    
    * `ORDERED(a, a, b)` will return an extra interval over just a b if it first matches a a b, meaning
    that you can get incorrect results if used in a `CONTAINING` filter -
    `CONTAINING(ORDERED(x, y), ORDERED(a, a, b))` will match on the document `a x a b y`
    * `UNORDERED(a, a)` will match on documents that just containg a single a.
    
    This commit adds a RepeatingIntervalsSource that correctly handles repeats within
    ordered and unordered sources. It also changes the way that gaps are calculated within
    ordered and unordered sources, by using a new width() method on IntervalIterator. The
    default implementation just returns end() - start() + 1, but RepeatingIntervalsSource
    instead returns the sum of the widths of its child iterators. This preserves maxgaps filtering
    on ordered and unordered sources that contain repeats.
    
    In order to correctly handle matches in this scenario, IntervalsSource#matches now always
    returns an explicit IntervalsMatchesIterator rather than a plain MatchesIterator, which adds
    gaps() and width() methods so that submatches can be combined in the same way that
    subiterators are. Extra checks have been added to checkIntervals() to ensure that the same
    intervals are returned by both iterator and matches, and a fix to
    DisjunctionIntervalIterator#matches() is also included - DisjunctionIntervalIterator minimizes
    its intervals, while MatchesUtils.disjunction does not, so there was a discrepancy between
    the two methods.
---
 lucene/CHANGES.txt                                 |   7 +
 .../queries/intervals/CachingMatchesIterator.java  |  13 +-
 .../intervals/ConjunctionIntervalsSource.java      |  24 +-
 .../intervals/DifferenceIntervalsSource.java       |   7 +-
 .../intervals/DisjunctionIntervalsSource.java      |  88 +++++-
 .../queries/intervals/ExtendedIntervalsSource.java |   5 +-
 .../queries/intervals/FilteredIntervalsSource.java |   5 +-
 .../intervals/FixedFieldIntervalsSource.java       |   3 +-
 .../lucene/queries/intervals/IntervalIterator.java |   7 +
 .../lucene/queries/intervals/IntervalMatches.java  |  38 ++-
 .../queries/intervals/IntervalMatchesIterator.java |  19 +-
 .../lucene/queries/intervals/IntervalsSource.java  |   2 +-
 .../MinimizingConjunctionMatchesIterator.java      |   7 +-
 .../MinimumShouldMatchIntervalsSource.java         |  42 +--
 .../intervals/MultiTermIntervalsSource.java        |  54 +++-
 .../queries/intervals/OffsetIntervalsSource.java   |   5 +-
 .../queries/intervals/OrderedIntervalsSource.java  |  52 +++-
 .../PayloadFilteredTermIntervalsSource.java        |  16 +-
 .../intervals/RepeatingIntervalsSource.java        | 323 +++++++++++++++++++++
 .../queries/intervals/TermIntervalsSource.java     |  16 +-
 .../intervals/UnorderedIntervalsSource.java        |  50 ++--
 .../lucene/queries/intervals/TestIntervals.java    |  70 ++++-
 .../queries/intervals/TestSimplifications.java     |  14 +
 23 files changed, 747 insertions(+), 120 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index a3a59fd..e8c614e 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -87,6 +87,13 @@ Bug Fixes
 * LUCENE-9200: Fix TieredMergePolicy to use double (not float) math to make its merging decisions, fixing
   a corner-case bug uncovered by fun randomized tests (Robert Muir, Mike McCandless)
 
+* LUCENE-9099: Unordered and Ordered interval queries now correctly handle
+  repeated subterms - ordered intervals could supply an 'extra' minimized
+  interval, resulting in odd matches when combined with eg CONTAINS queries;
+  and unordered intervals would match duplicate subterms on the same position,
+  so an query for UNORDERED(foo, foo) would match a document containing 'foo'
+  only once.  (Alan Woodward)
+
 Other
 ---------------------
 
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/CachingMatchesIterator.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/CachingMatchesIterator.java
index b272244..cedf955 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/CachingMatchesIterator.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/CachingMatchesIterator.java
@@ -24,14 +24,14 @@ import org.apache.lucene.search.MatchesIterator;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.ArrayUtil;
 
-class CachingMatchesIterator extends FilterMatchesIterator {
+class CachingMatchesIterator extends FilterMatchesIterator implements IntervalMatchesIterator {
 
   private boolean positioned = false;
   private int[] posAndOffsets = new int[4*4];
   private Query[] matchingQueries = new Query[4];
   private int count = 0; 
 
-  CachingMatchesIterator(MatchesIterator in) {
+  CachingMatchesIterator(IntervalMatchesIterator in) {
     super(in);
   }
 
@@ -133,4 +133,13 @@ class CachingMatchesIterator extends FilterMatchesIterator {
     };
   }
 
+  @Override
+  public int gaps() {
+    return ((IntervalMatchesIterator)in).gaps();
+  }
+
+  @Override
+  public int width() {
+    return ((IntervalMatchesIterator)in).width();
+  }
 }
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/ConjunctionIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/ConjunctionIntervalsSource.java
index bdfb55c..1cd2c95 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/ConjunctionIntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/ConjunctionIntervalsSource.java
@@ -65,10 +65,10 @@ abstract class ConjunctionIntervalsSource extends IntervalsSource {
   protected abstract IntervalIterator combine(List<IntervalIterator> iterators);
 
   @Override
-  public final MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
-    List<MatchesIterator> subs = new ArrayList<>();
+  public final IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    List<IntervalMatchesIterator> subs = new ArrayList<>();
     for (IntervalsSource source : subSources) {
-      MatchesIterator mi = source.matches(field, ctx, doc);
+      IntervalMatchesIterator mi = source.matches(field, ctx, doc);
       if (mi == null) {
         return null;
       }
@@ -87,13 +87,13 @@ abstract class ConjunctionIntervalsSource extends IntervalsSource {
     return isMinimizing ? new MinimizingConjunctionMatchesIterator(it, subs) : new ConjunctionMatchesIterator(it, subs);
   }
 
-  private static class ConjunctionMatchesIterator implements MatchesIterator {
+  private static class ConjunctionMatchesIterator implements IntervalMatchesIterator {
 
     final IntervalIterator iterator;
-    final List<MatchesIterator> subs;
+    final List<IntervalMatchesIterator> subs;
     boolean cached = true;
 
-    private ConjunctionMatchesIterator(IntervalIterator iterator, List<MatchesIterator> subs) {
+    private ConjunctionMatchesIterator(IntervalIterator iterator, List<IntervalMatchesIterator> subs) {
       this.iterator = iterator;
       this.subs = subs;
     }
@@ -152,9 +152,19 @@ abstract class ConjunctionIntervalsSource extends IntervalsSource {
     public Query getQuery() {
       throw new UnsupportedOperationException();
     }
+
+    @Override
+    public int gaps() {
+      return iterator.gaps();
+    }
+
+    @Override
+    public int width() {
+      return iterator.width();
+    }
   }
 
-  private static class SingletonMatchesIterator extends FilterMatchesIterator {
+  static class SingletonMatchesIterator extends FilterMatchesIterator {
 
     boolean exhausted = false;
 
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/DifferenceIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/DifferenceIntervalsSource.java
index 7ca3cb7..41149f9 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/DifferenceIntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/DifferenceIntervalsSource.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.MatchesIterator;
 import org.apache.lucene.search.QueryVisitor;
 
 abstract class DifferenceIntervalsSource extends IntervalsSource {
@@ -48,12 +47,12 @@ abstract class DifferenceIntervalsSource extends IntervalsSource {
   }
 
   @Override
-  public final MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
-    MatchesIterator minIt = minuend.matches(field, ctx, doc);
+  public final IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    IntervalMatchesIterator minIt = minuend.matches(field, ctx, doc);
     if (minIt == null) {
       return null;
     }
-    MatchesIterator subIt = subtrahend.matches(field, ctx, doc);
+    IntervalMatchesIterator subIt = subtrahend.matches(field, ctx, doc);
     if (subIt == null) {
       return minIt;
     }
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/DisjunctionIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/DisjunctionIntervalsSource.java
index 67d6f6f..f4ee197 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/DisjunctionIntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/DisjunctionIntervalsSource.java
@@ -31,7 +31,6 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.MatchesIterator;
-import org.apache.lucene.search.MatchesUtils;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.util.PriorityQueue;
@@ -82,15 +81,24 @@ class DisjunctionIntervalsSource extends IntervalsSource {
   }
 
   @Override
-  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
-    List<MatchesIterator> subMatches = new ArrayList<>();
+  public IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    List<IntervalMatchesIterator> subMatches = new ArrayList<>();
     for (IntervalsSource subSource : subSources) {
-      MatchesIterator mi = subSource.matches(field, ctx, doc);
+      IntervalMatchesIterator mi = subSource.matches(field, ctx, doc);
       if (mi != null) {
         subMatches.add(mi);
       }
     }
-    return MatchesUtils.disjunction(subMatches);
+    if (subMatches.size() == 0) {
+      return null;
+    }
+    DisjunctionIntervalIterator it = new DisjunctionIntervalIterator(
+        subMatches.stream().map(m -> IntervalMatches.wrapMatches(m, doc)).collect(Collectors.toList())
+    );
+    if (it.advance(doc) != doc) {
+      return null;
+    }
+    return new DisjunctionMatchesIterator(it, subMatches);
   }
 
   @Override
@@ -196,6 +204,16 @@ class DisjunctionIntervalsSource extends IntervalsSource {
       current = EMPTY;
     }
 
+    int currentOrd() {
+      assert current != EMPTY && current != EXHAUSTED;
+      for (int i = 0; i < iterators.size(); i++) {
+        if (iterators.get(i) == current) {
+          return i;
+        }
+      }
+      throw new IllegalStateException();
+    }
+
     @Override
     public int nextInterval() throws IOException {
       if (current == EMPTY || current == EXHAUSTED) {
@@ -344,4 +362,64 @@ class DisjunctionIntervalsSource extends IntervalsSource {
     }
   };
 
+  private static class DisjunctionMatchesIterator implements IntervalMatchesIterator {
+
+    final DisjunctionIntervalIterator it;
+    final List<IntervalMatchesIterator> subs;
+
+    private DisjunctionMatchesIterator(DisjunctionIntervalIterator it, List<IntervalMatchesIterator> subs) {
+      this.it = it;
+      this.subs = subs;
+    }
+
+    @Override
+    public boolean next() throws IOException {
+      return it.nextInterval() != IntervalIterator.NO_MORE_INTERVALS;
+    }
+
+    @Override
+    public int startPosition() {
+      return it.start();
+    }
+
+    @Override
+    public int endPosition() {
+      return it.end();
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      int ord = it.currentOrd();
+      return subs.get(ord).startOffset();
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      int ord = it.currentOrd();
+      return subs.get(ord).endOffset();
+    }
+
+    @Override
+    public MatchesIterator getSubMatches() throws IOException {
+      int ord = it.currentOrd();
+      return subs.get(ord).getSubMatches();
+    }
+
+    @Override
+    public Query getQuery() {
+      int ord = it.currentOrd();
+      return subs.get(ord).getQuery();
+    }
+
+    @Override
+    public int gaps() {
+      return it.gaps();
+    }
+
+    @Override
+    public int width() {
+      return it.width();
+    }
+  }
+
 }
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/ExtendedIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/ExtendedIntervalsSource.java
index 00eb866..27f5773 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/ExtendedIntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/ExtendedIntervalsSource.java
@@ -24,7 +24,6 @@ import java.util.Objects;
 import java.util.stream.Collectors;
 
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.MatchesIterator;
 import org.apache.lucene.search.QueryVisitor;
 
 class ExtendedIntervalsSource extends IntervalsSource {
@@ -49,8 +48,8 @@ class ExtendedIntervalsSource extends IntervalsSource {
   }
 
   @Override
-  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
-    MatchesIterator in = source.matches(field, ctx, doc);
+  public IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    IntervalMatchesIterator in = source.matches(field, ctx, doc);
     if (in == null) {
       return null;
     }
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/FilteredIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/FilteredIntervalsSource.java
index 05ecc4c..adf147e 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/FilteredIntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/FilteredIntervalsSource.java
@@ -24,7 +24,6 @@ import java.util.Objects;
 import java.util.stream.Collectors;
 
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.MatchesIterator;
 import org.apache.lucene.search.QueryVisitor;
 
 /**
@@ -108,8 +107,8 @@ public abstract class FilteredIntervalsSource extends IntervalsSource {
   }
 
   @Override
-  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
-    MatchesIterator mi = in.matches(field, ctx, doc);
+  public IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    IntervalMatchesIterator mi = in.matches(field, ctx, doc);
     if (mi == null) {
       return null;
     }
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/FixedFieldIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/FixedFieldIntervalsSource.java
index 94cba9a..95dea3a 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/FixedFieldIntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/FixedFieldIntervalsSource.java
@@ -24,7 +24,6 @@ import java.util.Objects;
 import java.util.stream.Collectors;
 
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.MatchesIterator;
 import org.apache.lucene.search.QueryVisitor;
 
 class FixedFieldIntervalsSource extends IntervalsSource {
@@ -43,7 +42,7 @@ class FixedFieldIntervalsSource extends IntervalsSource {
   }
 
   @Override
-  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+  public IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
     return source.matches(this.field, ctx, doc);
   }
 
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalIterator.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalIterator.java
index b97ce05..c5fbf2a 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalIterator.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalIterator.java
@@ -73,6 +73,13 @@ public abstract class IntervalIterator extends DocIdSetIterator {
   public abstract int gaps();
 
   /**
+   * The width of the current interval
+   */
+  public int width() {
+    return end() - start() + 1;
+  }
+
+  /**
    * Advance the iterator to the next interval
    *
    * @return the start of the next interval, or {@link IntervalIterator#NO_MORE_INTERVALS} if
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalMatches.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalMatches.java
index 2bc10f6..82d9d09 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalMatches.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalMatches.java
@@ -24,7 +24,7 @@ import org.apache.lucene.search.Query;
 
 final class IntervalMatches {
 
-  static MatchesIterator asMatches(IntervalIterator iterator, MatchesIterator source, int doc) throws IOException {
+  static IntervalMatchesIterator asMatches(IntervalIterator iterator, IntervalMatchesIterator source, int doc) throws IOException {
     if (source == null) {
       return null;
     }
@@ -34,7 +34,7 @@ final class IntervalMatches {
     if (iterator.nextInterval() == IntervalIterator.NO_MORE_INTERVALS) {
       return null;
     }
-    return new MatchesIterator() {
+    return new IntervalMatchesIterator() {
 
       boolean cached = true;
 
@@ -68,6 +68,16 @@ final class IntervalMatches {
       }
 
       @Override
+      public int gaps() {
+        return iterator.gaps();
+      }
+
+      @Override
+      public int width() {
+        return iterator.width();
+      }
+
+      @Override
       public MatchesIterator getSubMatches() throws IOException {
         return source.getSubMatches();
       }
@@ -79,21 +89,27 @@ final class IntervalMatches {
     };
   }
 
-  enum State { UNPOSITIONED, ITERATING, EXHAUSTED }
+  enum State { UNPOSITIONED, ITERATING, NO_MORE_INTERVALS, EXHAUSTED }
 
-  static IntervalIterator wrapMatches(MatchesIterator mi, int doc) {
+  static IntervalIterator wrapMatches(IntervalMatchesIterator mi, int doc) {
     return new IntervalIterator() {
 
       State state = State.UNPOSITIONED;
 
       @Override
       public int start() {
+        if (state == State.NO_MORE_INTERVALS) {
+          return NO_MORE_INTERVALS;
+        }
         assert state == State.ITERATING;
         return mi.startPosition();
       }
 
       @Override
       public int end() {
+        if (state == State.NO_MORE_INTERVALS) {
+          return NO_MORE_INTERVALS;
+        }
         assert state == State.ITERATING;
         return mi.endPosition();
       }
@@ -101,10 +117,13 @@ final class IntervalMatches {
       @Override
       public int gaps() {
         assert state == State.ITERATING;
-        if (mi instanceof IntervalMatchesIterator) {
-          return ((IntervalMatchesIterator)mi).gaps();
-        }
-        return 0;
+        return mi.gaps();
+      }
+
+      @Override
+      public int width() {
+        assert state == State.ITERATING;
+        return mi.width();
       }
 
       @Override
@@ -113,6 +132,7 @@ final class IntervalMatches {
         if (mi.next()) {
           return mi.startPosition();
         }
+        state = State.NO_MORE_INTERVALS;
         return NO_MORE_INTERVALS;
       }
 
@@ -127,6 +147,7 @@ final class IntervalMatches {
           case UNPOSITIONED:
             return -1;
           case ITERATING:
+          case NO_MORE_INTERVALS:
             return doc;
           case EXHAUSTED:
         }
@@ -140,6 +161,7 @@ final class IntervalMatches {
             state = State.ITERATING;
             return doc;
           case ITERATING:
+          case NO_MORE_INTERVALS:
             state = State.EXHAUSTED;
           case EXHAUSTED:
         }
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalMatchesIterator.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalMatchesIterator.java
index 2c3d31d..63e5b5e 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalMatchesIterator.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalMatchesIterator.java
@@ -20,19 +20,24 @@ package org.apache.lucene.queries.intervals;
 import org.apache.lucene.search.MatchesIterator;
 
 /**
- * An extension of MatchesIterator that allows the gaps from a wrapped
- * IntervalIterator to be reported.
+ * An extension of MatchesIterator that allows it to be treated as
+ * an IntervalIterator
  *
- * This is necessary because {@link MatchesIterator#getSubMatches()} returns
- * the submatches of all nested matches as a flat iterator, but
- * {@link IntervalIterator#gaps()} only returns the gaps between its immediate
- * sub-matches, so we can't calculate the latter using the former.
+ * This is necessary to get access to {@link IntervalIterator#gaps()}
+ * and {@link IntervalIterator#width()} when constructing matches
  */
-interface IntervalMatchesIterator extends MatchesIterator {
+public interface IntervalMatchesIterator extends MatchesIterator {
 
   /**
    * The number of top-level gaps inside the current match
+   * @see IntervalIterator#gaps()
    */
   int gaps();
 
+  /**
+   * The width of the current match
+   * @see IntervalIterator#width()
+   */
+  int width();
+
 }
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalsSource.java
index ae23978..f1c89d3 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/IntervalsSource.java
@@ -53,7 +53,7 @@ public abstract class IntervalsSource {
    * @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;
+  public abstract IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException;
 
   /**
    * Expert: visit the tree of sources
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/MinimizingConjunctionMatchesIterator.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/MinimizingConjunctionMatchesIterator.java
index 2a7490c..4477991 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/MinimizingConjunctionMatchesIterator.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/MinimizingConjunctionMatchesIterator.java
@@ -31,7 +31,7 @@ class MinimizingConjunctionMatchesIterator implements IntervalMatchesIterator {
   private final List<CachingMatchesIterator> subs = new ArrayList<>();
   private boolean cached = true;
 
-  MinimizingConjunctionMatchesIterator(IntervalIterator iterator, List<MatchesIterator> subs) {
+  MinimizingConjunctionMatchesIterator(IntervalIterator iterator, List<IntervalMatchesIterator> subs) {
     this.iterator = iterator;
     for (MatchesIterator mi : subs) {
       assert mi instanceof CachingMatchesIterator;
@@ -84,6 +84,11 @@ class MinimizingConjunctionMatchesIterator implements IntervalMatchesIterator {
   }
 
   @Override
+  public int width() {
+    return iterator.width();
+  }
+
+  @Override
   public MatchesIterator getSubMatches() throws IOException {
     List<MatchesIterator> mis = new ArrayList<>();
     int endPos = endPosition();
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/MinimumShouldMatchIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/MinimumShouldMatchIntervalsSource.java
index bfdee2e..ce3a6de 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/MinimumShouldMatchIntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/MinimumShouldMatchIntervalsSource.java
@@ -63,10 +63,10 @@ class MinimumShouldMatchIntervalsSource extends IntervalsSource {
   }
 
   @Override
-  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+  public IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
     Map<IntervalIterator, CachingMatchesIterator> lookup = new IdentityHashMap<>();
     for (IntervalsSource source : sources) {
-      MatchesIterator mi = source.matches(field, ctx, doc);
+      IntervalMatchesIterator mi = source.matches(field, ctx, doc);
       if (mi != null) {
         CachingMatchesIterator cmi = new CachingMatchesIterator(mi);
         lookup.put(IntervalMatches.wrapMatches(cmi, doc), cmi);
@@ -155,10 +155,9 @@ class MinimumShouldMatchIntervalsSource extends IntervalsSource {
     private final PriorityQueue<IntervalIterator> backgroundQueue;
     private final float matchCost;
     private final int minShouldMatch;
-    private final int[] innerPositions;
     private final Collection<IntervalIterator> currentIterators = new ArrayList<>();
 
-    private int start, end, queueEnd, firstEnd;
+    private int start, end, queueEnd, slop;
     private IntervalIterator lead;
 
     MinimumShouldMatchIntervalIterator(Collection<IntervalIterator> subs, int minShouldMatch) {
@@ -171,7 +170,6 @@ class MinimumShouldMatchIntervalsSource extends IntervalsSource {
       this.approximation = new DisjunctionDISIApproximation(disiQueue);
       this.matchCost = mc;
       this.minShouldMatch = minShouldMatch;
-      this.innerPositions = new int[minShouldMatch * 2];
 
       this.proximityQueue = new PriorityQueue<IntervalIterator>(minShouldMatch) {
         @Override
@@ -199,29 +197,7 @@ class MinimumShouldMatchIntervalsSource extends IntervalsSource {
 
     @Override
     public int gaps() {
-      int i = 0;
-      for (IntervalIterator it : proximityQueue) {
-        if (it.end() > end) {
-          innerPositions[i * 2] = start;
-          innerPositions[i * 2 + 1] = firstEnd;
-        }
-        else {
-          innerPositions[i * 2] = it.start();
-          innerPositions[i * 2 + 1] = it.end();
-        }
-        i++;
-      }
-      if (proximityQueue.size() < minShouldMatch) {
-        // the leading iterator has been exhausted and removed from the queue
-        innerPositions[i * 2] = start;
-        innerPositions[i * 2 + 1] = firstEnd;
-      }
-      Arrays.sort(innerPositions);
-      int gaps = 0;
-      for (int j = 1; j < minShouldMatch; j++) {
-        gaps += (innerPositions[j * 2] - innerPositions[j * 2 - 1] - 1);
-      }
-      return gaps;
+      return slop;
     }
 
     @Override
@@ -242,8 +218,11 @@ class MinimumShouldMatchIntervalsSource extends IntervalsSource {
       // then, minimize it
       do {
         start = proximityQueue.top().start();
-        firstEnd = proximityQueue.top().end();
         end = queueEnd;
+        slop = width();
+        for (IntervalIterator it : proximityQueue) {
+          slop -= it.width();
+        }
         if (proximityQueue.top().end() == end)
           return start;
         lead = proximityQueue.pop();
@@ -390,6 +369,11 @@ class MinimumShouldMatchIntervalsSource extends IntervalsSource {
     }
 
     @Override
+    public int width() {
+      return iterator.width();
+    }
+
+    @Override
     public MatchesIterator getSubMatches() throws IOException {
       List<MatchesIterator> mis = new ArrayList<>();
       int endPos = endPosition();
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/MultiTermIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/MultiTermIntervalsSource.java
index 8fa60fc..f32a204 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/MultiTermIntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/MultiTermIntervalsSource.java
@@ -30,6 +30,7 @@ import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.MatchesIterator;
 import org.apache.lucene.search.MatchesUtils;
+import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
@@ -73,7 +74,7 @@ class MultiTermIntervalsSource extends IntervalsSource {
   }
 
   @Override
-  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+  public IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
     Terms terms = ctx.reader().terms(field);
     if (terms == null) {
       return null;
@@ -91,7 +92,56 @@ class MultiTermIntervalsSource extends IntervalsSource {
         }
       }
     }
-    return MatchesUtils.disjunction(subMatches);
+    MatchesIterator mi = MatchesUtils.disjunction(subMatches);
+    if (mi == null) {
+      return null;
+    }
+    return new IntervalMatchesIterator() {
+      @Override
+      public int gaps() {
+        return 0;
+      }
+
+      @Override
+      public int width() {
+        return 1;
+      }
+
+      @Override
+      public boolean next() throws IOException {
+        return mi.next();
+      }
+
+      @Override
+      public int startPosition() {
+        return mi.startPosition();
+      }
+
+      @Override
+      public int endPosition() {
+        return mi.endPosition();
+      }
+
+      @Override
+      public int startOffset() throws IOException {
+        return mi.startOffset();
+      }
+
+      @Override
+      public int endOffset() throws IOException {
+        return mi.endOffset();
+      }
+
+      @Override
+      public MatchesIterator getSubMatches() throws IOException {
+        return mi.getSubMatches();
+      }
+
+      @Override
+      public Query getQuery() {
+        return mi.getQuery();
+      }
+    };
   }
 
   @Override
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/OffsetIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/OffsetIntervalsSource.java
index 62e5667..1f96b40 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/OffsetIntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/OffsetIntervalsSource.java
@@ -24,7 +24,6 @@ import java.util.Objects;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.MatchesIterator;
 import org.apache.lucene.search.QueryVisitor;
 
 /**
@@ -137,8 +136,8 @@ class OffsetIntervalsSource extends IntervalsSource {
   }
 
   @Override
-  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
-    MatchesIterator mi = in.matches(field, ctx, doc);
+  public IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    IntervalMatchesIterator mi = in.matches(field, ctx, doc);
     if (mi == null) {
       return null;
     }
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/OrderedIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/OrderedIntervalsSource.java
index e736a59..6331688 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/OrderedIntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/OrderedIntervalsSource.java
@@ -30,7 +30,11 @@ class OrderedIntervalsSource extends ConjunctionIntervalsSource {
     if (sources.size() == 1) {
       return sources.get(0);
     }
-    return new OrderedIntervalsSource(flatten(sources));
+    List<IntervalsSource> rewritten = deduplicate(flatten(sources));
+    if (rewritten.size() == 1) {
+      return rewritten.get(0);
+    }
+    return new OrderedIntervalsSource(rewritten);
   }
 
   private static List<IntervalsSource> flatten(List<IntervalsSource> sources) {
@@ -46,6 +50,26 @@ class OrderedIntervalsSource extends ConjunctionIntervalsSource {
     return flattened;
   }
 
+  private static List<IntervalsSource> deduplicate(List<IntervalsSource> sources) {
+    List<IntervalsSource> deduplicated = new ArrayList<>();
+    List<IntervalsSource> current = new ArrayList<>();
+    for (IntervalsSource source : sources) {
+      if (current.size() == 0 || current.get(0).equals(source)) {
+        current.add(source);
+      }
+      else {
+        deduplicated.add(RepeatingIntervalsSource.build(current.get(0), current.size()));
+        current.clear();
+        current.add(source);
+      }
+    }
+    deduplicated.add(RepeatingIntervalsSource.build(current.get(0), current.size()));
+    if (deduplicated.size() == 1 && deduplicated.get(0) instanceof RepeatingIntervalsSource) {
+      ((RepeatingIntervalsSource)deduplicated.get(0)).setName("ORDERED");
+    }
+    return deduplicated;
+  }
+
   private OrderedIntervalsSource(List<IntervalsSource> sources) {
     super(sources, true);
   }
@@ -89,7 +113,7 @@ class OrderedIntervalsSource extends ConjunctionIntervalsSource {
   private static class OrderedIntervalIterator extends ConjunctionIntervalIterator {
 
     int start = -1, end = -1, i;
-    int firstEnd;
+    int slop;
 
     private OrderedIntervalIterator(List<IntervalIterator> subIntervals) {
       super(subIntervals);
@@ -107,17 +131,17 @@ class OrderedIntervalsSource extends ConjunctionIntervalsSource {
 
     @Override
     public int nextInterval() throws IOException {
-      start = end = IntervalIterator.NO_MORE_INTERVALS;
-      int b = Integer.MAX_VALUE;
+      start = end = slop = IntervalIterator.NO_MORE_INTERVALS;
+      int lastStart = Integer.MAX_VALUE;
       i = 1;
       while (true) {
         while (true) {
-          if (subIterators.get(i - 1).end() >= b)
+          if (subIterators.get(i - 1).end() >= lastStart)
             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)
+            if (subIterators.get(i).end() >= lastStart || subIterators.get(i).nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
               return start;
           }
           while (subIterators.get(i).start() <= subIterators.get(i - 1).end());
@@ -127,9 +151,12 @@ class OrderedIntervalsSource extends ConjunctionIntervalsSource {
         if (start == NO_MORE_INTERVALS) {
           return end = NO_MORE_INTERVALS;
         }
-        firstEnd = subIterators.get(0).end();
         end = subIterators.get(subIterators.size() - 1).end();
-        b = subIterators.get(subIterators.size() - 1).start();
+        slop = end - start + 1;
+        for (IntervalIterator subIterator : subIterators) {
+          slop -= subIterator.width();
+        }
+        lastStart = subIterators.get(subIterators.size() - 1).start();
         i = 1;
         if (subIterators.get(0).nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
           return start;
@@ -138,18 +165,15 @@ class OrderedIntervalsSource extends ConjunctionIntervalsSource {
 
     @Override
     public int gaps() {
-      int gaps = subIterators.get(1).start() - firstEnd - 1;
-      for (int i = 2; i < subIterators.size(); i++) {
-        gaps += (subIterators.get(i).start() - subIterators.get(i - 1).end() - 1);
-      }
-      return gaps;
+      return slop;
     }
 
     @Override
     protected void reset() throws IOException {
       subIterators.get(0).nextInterval();
       i = 1;
-      start = end = firstEnd = -1;
+      start = end = slop = -1;
     }
   }
+
 }
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/PayloadFilteredTermIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/PayloadFilteredTermIntervalsSource.java
index 25e4da5..1ec20e0 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/PayloadFilteredTermIntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/PayloadFilteredTermIntervalsSource.java
@@ -143,7 +143,7 @@ class PayloadFilteredTermIntervalsSource extends IntervalsSource {
   }
 
   @Override
-  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+  public IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
     Terms terms = ctx.reader().terms(field);
     if (terms == null)
       return null;
@@ -165,12 +165,22 @@ class PayloadFilteredTermIntervalsSource extends IntervalsSource {
     visitor.consumeTerms(new IntervalQuery(field, this), new Term(field, term));
   }
 
-  private MatchesIterator matches(TermsEnum te, int doc) throws IOException {
+  private IntervalMatchesIterator matches(TermsEnum te, int doc) throws IOException {
     PostingsEnum pe = te.postings(null, PostingsEnum.ALL);
     if (pe.advance(doc) != doc) {
       return null;
     }
-    return new MatchesIterator() {
+    return new IntervalMatchesIterator() {
+
+      @Override
+      public int gaps() {
+        return 0;
+      }
+
+      @Override
+      public int width() {
+        return 1;
+      }
 
       int upto = pe.freq();
       int pos = -1;
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/RepeatingIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/RepeatingIntervalsSource.java
new file mode 100644
index 0000000..1a5c891
--- /dev/null
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/RepeatingIntervalsSource.java
@@ -0,0 +1,323 @@
+/*
+ * 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.queries.intervals;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.MatchesIterator;
+import org.apache.lucene.search.MatchesUtils;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+
+/**
+ * Generates an iterator that spans repeating instances of a sub-iterator,
+ * avoiding minimization.  This is useful for repeated terms within an
+ * unordered interval, for example, ensuring that multiple iterators do
+ * not match on a single term.
+ *
+ * The generated iterators have a specialized {@link IntervalIterator#width()}
+ * implementation that sums up the widths of the individual sub-iterators,
+ * rather than just returning the full span of the iterator.
+ */
+class RepeatingIntervalsSource extends IntervalsSource {
+
+  static IntervalsSource build(IntervalsSource in, int childCount) {
+    if (childCount == 1) {
+      return in;
+    }
+    assert childCount > 0;
+    return new RepeatingIntervalsSource(in, childCount);
+  }
+
+  final IntervalsSource in;
+  final int childCount;
+  String name;
+
+  private RepeatingIntervalsSource(IntervalsSource in, int childCount) {
+    this.in = in;
+    this.childCount = childCount;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    IntervalIterator it = in.intervals(field, ctx);
+    if (it == null) {
+      return null;
+    }
+    return new DuplicateIntervalIterator(it, childCount);
+  }
+
+  @Override
+  public IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    List<IntervalMatchesIterator> subs = new ArrayList<>();
+    for (int i = 0; i < childCount; i++) {
+      IntervalMatchesIterator mi = in.matches(field, ctx, doc);
+      if (mi == null) {
+        return null;
+      }
+      subs.add(mi);
+    }
+    return DuplicateMatchesIterator.build(subs);
+  }
+
+  @Override
+  public void visit(String field, QueryVisitor visitor) {
+    in.visit(field, visitor);
+  }
+
+  @Override
+  public int minExtent() {
+    return in.minExtent();
+  }
+
+  @Override
+  public Collection<IntervalsSource> pullUpDisjunctions() {
+    return Collections.singleton(this);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(in, childCount);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other instanceof RepeatingIntervalsSource == false) return false;
+    RepeatingIntervalsSource o = (RepeatingIntervalsSource) other;
+    return Objects.equals(this.in, o.in) && Objects.equals(this.childCount, o.childCount);
+  }
+
+  @Override
+  public String toString() {
+    String s = in.toString();
+    StringBuilder out = new StringBuilder(s);
+    for (int i = 1; i < childCount; i++) {
+      out.append(",").append(s);
+    }
+    if (name != null) {
+      return name + "(" + out.toString() + ")";
+    }
+    return out.toString();
+  }
+
+  private static class DuplicateIntervalIterator extends IntervalIterator {
+
+    private final IntervalIterator in;
+    final int[] cache;
+    final int cacheLength;
+    int cacheBase;
+    boolean started = false;
+    boolean exhausted = false;
+
+    private DuplicateIntervalIterator(IntervalIterator primary, int copies) {
+      this.in = primary;
+      this.cacheLength = copies;
+      this.cache = new int[this.cacheLength * 2];
+    }
+
+    @Override
+    public int start() {
+      return exhausted ? NO_MORE_INTERVALS : cache[(cacheBase % cacheLength) * 2];
+    }
+
+    @Override
+    public int end() {
+      return exhausted ? NO_MORE_INTERVALS : cache[(((cacheBase + cacheLength - 1) % cacheLength) * 2) + 1];
+    }
+
+    @Override
+    public int width() {
+      int width = 0;
+      for (int i = 0; i < cacheLength; i++) {
+        int pos = (cacheBase + i) % cacheLength;
+        width += cache[pos * 2] - cache[pos * 2 + 1] + 1;
+      }
+      return width;
+    }
+
+    @Override
+    public int gaps() {
+      return super.width() - width();
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      if (exhausted) {
+        return NO_MORE_INTERVALS;
+      }
+      if (started == false) {
+        for (int i = 0; i < cacheLength; i++) {
+          if (cacheNextInterval(i) == NO_MORE_INTERVALS) {
+            return NO_MORE_INTERVALS;
+          }
+        }
+        cacheBase = 0;
+        started = true;
+        return start();
+      }
+      else {
+        int insert = (cacheBase + cacheLength) % cacheLength;
+        cacheBase = (cacheBase + 1) % cacheLength;
+        return cacheNextInterval(insert);
+      }
+    }
+
+    private int cacheNextInterval(int linePos) throws IOException {
+      if (in.nextInterval() == NO_MORE_INTERVALS) {
+        exhausted = true;
+        return NO_MORE_INTERVALS;
+      }
+      cache[linePos * 2] = in.start();
+      cache[linePos * 2 + 1] = in.end();
+      return start();
+    }
+
+    @Override
+    public float matchCost() {
+      return in.matchCost();
+    }
+
+    @Override
+    public int docID() {
+      return in.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      started = exhausted = false;
+      Arrays.fill(cache, -1);
+      return in.nextDoc();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      started = exhausted = false;
+      Arrays.fill(cache, -1);
+      return in.advance(target);
+    }
+
+    @Override
+    public long cost() {
+      return in.cost();
+    }
+  }
+
+  private static class DuplicateMatchesIterator implements IntervalMatchesIterator {
+
+    List<IntervalMatchesIterator> subs;
+    boolean cached = false;
+
+    static IntervalMatchesIterator build(List<IntervalMatchesIterator> subs) throws IOException {
+      int count = subs.size();
+      while (count > 0) {
+        for (int i = 0; i < count; i++) {
+          if (subs.get(count - 1).next() == false) {
+            return null;
+          }
+        }
+        count--;
+      }
+      return new DuplicateMatchesIterator(subs);
+    }
+
+    private DuplicateMatchesIterator(List<IntervalMatchesIterator> subs) throws IOException {
+      this.subs = subs;
+    }
+
+    @Override
+    public boolean next() throws IOException {
+      if (cached == false) {
+        return cached = true;
+      }
+      if (subs.get(subs.size() - 1).next() == false) {
+        return false;
+      }
+      for (int i = 0; i < subs.size() - 1; i++) {
+        subs.get(i).next();
+      }
+      return true;
+    }
+
+    @Override
+    public int startPosition() {
+      return subs.get(0).startPosition();
+    }
+
+    @Override
+    public int endPosition() {
+      return subs.get(subs.size() - 1).endPosition();
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      return subs.get(0).startOffset();
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      return subs.get(subs.size() - 1).endOffset();
+    }
+
+    @Override
+    public MatchesIterator getSubMatches() throws IOException {
+      List<MatchesIterator> subMatches = new ArrayList<>();
+      for (MatchesIterator mi : subs) {
+        MatchesIterator sub = mi.getSubMatches();
+        if (sub == null) {
+          sub = new ConjunctionIntervalsSource.SingletonMatchesIterator(mi);
+        }
+        subMatches.add(sub);
+      }
+      return MatchesUtils.disjunction(subMatches);
+    }
+
+    @Override
+    public Query getQuery() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int gaps() {
+      int width = endPosition() - startPosition() + 1;
+      for (MatchesIterator mi : subs) {
+        width = width - (mi.endPosition() - mi.startPosition() + 1);
+      }
+      return width;
+    }
+
+    @Override
+    public int width() {
+      int width = 0;
+      for (MatchesIterator mi : subs) {
+        width += (mi.endPosition() - mi.startPosition() + 1);
+      }
+      return width;
+    }
+  }
+}
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java
index ac98b25..16c886b 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/TermIntervalsSource.java
@@ -139,7 +139,7 @@ class TermIntervalsSource extends IntervalsSource {
   }
 
   @Override
-  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+  public IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
     Terms terms = ctx.reader().terms(field);
     if (terms == null)
       return null;
@@ -153,13 +153,23 @@ class TermIntervalsSource extends IntervalsSource {
     return matches(te, doc, field);
   }
 
-  static MatchesIterator matches(TermsEnum te, int doc, String field) throws IOException {
+  static IntervalMatchesIterator matches(TermsEnum te, int doc, String field) throws IOException {
     TermQuery query = new TermQuery(new Term(field, te.term()));
     PostingsEnum pe = te.postings(null, PostingsEnum.OFFSETS);
     if (pe.advance(doc) != doc) {
       return null;
     }
-    return new MatchesIterator() {
+    return new IntervalMatchesIterator() {
+
+      @Override
+      public int gaps() {
+        return 0;
+      }
+
+      @Override
+      public int width() {
+        return 1;
+      }
 
       int upto = pe.freq();
       int pos = -1;
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/UnorderedIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/UnorderedIntervalsSource.java
index 2af850e..e5dfeac 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/UnorderedIntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/UnorderedIntervalsSource.java
@@ -19,9 +19,10 @@ package org.apache.lucene.queries.intervals;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
+import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
 import java.util.stream.Collectors;
 
@@ -33,7 +34,26 @@ class UnorderedIntervalsSource extends ConjunctionIntervalsSource {
     if (sources.size() == 1) {
       return sources.get(0);
     }
-    return new UnorderedIntervalsSource(flatten(sources));
+    List<IntervalsSource> rewritten = deduplicate(flatten(sources));
+    if (rewritten.size() == 1) {
+      return rewritten.get(0);
+    }
+    return new UnorderedIntervalsSource(rewritten);
+  }
+
+  private static List<IntervalsSource> deduplicate(List<IntervalsSource> sources) {
+    Map<IntervalsSource, Integer> counts = new LinkedHashMap<>(); // preserve order for testing
+    for (IntervalsSource source : sources) {
+      counts.compute(source, (k, v) -> v == null ? 1 : v + 1);
+    }
+    List<IntervalsSource> deduplicated = new ArrayList<>();
+    for (IntervalsSource source : counts.keySet()) {
+      deduplicated.add(RepeatingIntervalsSource.build(source, counts.get(source)));
+    }
+    if (deduplicated.size() == 1 && deduplicated.get(0) instanceof RepeatingIntervalsSource) {
+      ((RepeatingIntervalsSource)deduplicated.get(0)).setName("UNORDERED");
+    }
+    return deduplicated;
   }
 
   private static List<IntervalsSource> flatten(List<IntervalsSource> sources) {
@@ -94,9 +114,8 @@ class UnorderedIntervalsSource extends ConjunctionIntervalsSource {
 
     private final PriorityQueue<IntervalIterator> queue;
     private final IntervalIterator[] subIterators;
-    private final int[] innerPositions;
 
-    int start = -1, end = -1, firstEnd, queueEnd;
+    int start = -1, end = -1, slop, queueEnd;
 
     UnorderedIntervalIterator(List<IntervalIterator> subIterators) {
       super(subIterators);
@@ -107,7 +126,6 @@ class UnorderedIntervalsSource extends ConjunctionIntervalsSource {
         }
       };
       this.subIterators = new IntervalIterator[subIterators.size()];
-      this.innerPositions = new int[subIterators.size() * 2];
 
       for (int i = 0; i < subIterators.size(); i++) {
         this.subIterators[i] = subIterators.get(i);
@@ -146,8 +164,11 @@ class UnorderedIntervalsSource extends ConjunctionIntervalsSource {
       // then, minimize it
       do {
         start = queue.top().start();
-        firstEnd = queue.top().end();
         end = queueEnd;
+        slop = width();
+        for (IntervalIterator it : subIterators) {
+          slop -= it.width();
+        }
         if (queue.top().end() == end)
           return start;
         IntervalIterator it = queue.pop();
@@ -161,22 +182,7 @@ class UnorderedIntervalsSource extends ConjunctionIntervalsSource {
 
     @Override
     public int gaps() {
-      for (int i = 0; i < subIterators.length; i++) {
-        if (subIterators[i].end() > end) {
-          innerPositions[i * 2] = start;
-          innerPositions[i * 2 + 1] = firstEnd;
-        }
-        else {
-          innerPositions[i * 2] = subIterators[i].start();
-          innerPositions[i * 2 + 1] = subIterators[i].end();
-        }
-      }
-      Arrays.sort(innerPositions);
-      int gaps = 0;
-      for (int i = 1; i < subIterators.length; i++) {
-        gaps += (innerPositions[i * 2] - innerPositions[i * 2 - 1] - 1);
-      }
-      return gaps;
+      return slop;
     }
 
     @Override
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestIntervals.java b/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestIntervals.java
index 336292e..e98f115 100644
--- a/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestIntervals.java
+++ b/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestIntervals.java
@@ -119,6 +119,7 @@ public class TestIntervals extends LuceneTestCase {
         continue;
       for (int doc = 0; doc < ctx.reader().maxDoc(); doc++) {
         ids.advance(doc);
+        MatchesIterator mi = source.matches(field, ctx, doc);
         int id = (int) ids.longValue();
         if (intervals.docID() == doc ||
             (intervals.docID() < doc && intervals.advance(doc) == doc)) {
@@ -133,15 +134,23 @@ public class TestIntervals extends LuceneTestCase {
             assertEquals("start() != pos returned from nextInterval()", expected[id][i], intervals.start());
             assertEquals("Wrong end value in doc " + id, expected[id][i + 1], intervals.end());
             i += 2;
+            assertTrue(mi.next());
+            assertEquals(source + ": wrong start value in match in doc " + id, intervals.start(), mi.startPosition());
+            assertEquals(source + ": wrong end value in match in doc " + id, intervals.end(), mi.endPosition());
           }
           assertEquals(source + ": wrong number of endpoints in doc " + id, expected[id].length, i);
           assertEquals(IntervalIterator.NO_MORE_INTERVALS, intervals.start());
           assertEquals(IntervalIterator.NO_MORE_INTERVALS, intervals.end());
-          if (i > 0)
+          if (i > 0) {
             matchedDocs++;
+            assertFalse(mi.next());
+          } else {
+            assertNull("Expected null matches iterator on doc " + id, mi);
+          }
         }
         else {
           assertEquals(0, expected[id].length);
+          assertNull(mi);
         }
       }
     }
@@ -271,6 +280,24 @@ public class TestIntervals extends LuceneTestCase {
     checkVisits(source, 3, "pease", "hot");
   }
 
+  public void testOrderedNearWithDuplicates() throws IOException {
+    IntervalsSource source = Intervals.ordered(Intervals.term("pease"), Intervals.term("pease"), Intervals.term("porridge"));
+    checkIntervals(source, "field1", 3, new int[][]{
+        {}, { 0, 4, 3, 7 }, { 0, 4, 3, 7 }, {}, { 0, 4, 3, 7 }, {}
+    });
+    assertGaps(source, 1, "field1", new int[]{ 2, 2 });
+
+    MatchesIterator mi = getMatches(source, 1, "field1");
+    assertMatch(mi, 0, 4, 0, 34);
+    MatchesIterator sub = mi.getSubMatches();
+    assertNotNull(sub);
+    assertMatch(sub, 0, 0, 0, 5);
+    assertMatch(sub, 3, 3, 20, 25);
+    assertMatch(sub, 4, 4, 26, 34);
+    assertMatch(mi, 3, 7, 20, 55);
+    assertFalse(mi.next());
+  }
+
   public void testPhraseIntervals() throws IOException {
     IntervalsSource source = Intervals.phrase("pease", "porridge");
     checkIntervals(source, "field1", 3, new int[][]{
@@ -335,6 +362,28 @@ public class TestIntervals extends LuceneTestCase {
     checkVisits(source, 3, "pease", "hot");
   }
 
+  public void testUnorderedWithRepeats() throws IOException {
+    IntervalsSource source = Intervals.unordered(Intervals.term("pease"), Intervals.term("pease"), Intervals.term("hot"));
+    checkIntervals(source, "field1", 3, new int[][]{
+        {}, { 0, 3, 2, 6, 3, 17 }, { 0, 5, 3, 6 }, {}, { 0, 3, 2, 6, 3, 17 }, {}
+    });
+    MatchesIterator mi = getMatches(source, 1, "field1");
+    assertMatch(mi, 0, 3, 0, 25);
+    MatchesIterator sub = mi.getSubMatches();
+    assertNotNull(sub);
+    assertMatch(sub, 0, 0, 0, 5);
+    assertMatch(sub, 2, 2, 15, 18);
+    assertMatch(sub, 3, 3, 20, 25);
+  }
+
+  public void testUnorderedWithRepeatsAndMaxGaps() throws IOException {
+    IntervalsSource source = Intervals.maxgaps(2,
+        Intervals.unordered(Intervals.term("pease"), Intervals.term("pease"), Intervals.term("hot")));
+    checkIntervals(source, "field1", 3, new int[][]{
+        {}, { 0, 3, 2, 6 }, { 3, 6 }, {}, { 0, 3, 2, 6 }, {}
+    });
+  }
+
   public void testIntervalDisjunction() throws IOException {
     IntervalsSource source = Intervals.or(Intervals.term("pease"), Intervals.term("hot"), Intervals.term("notMatching"));
     checkIntervals(source, "field1", 4, new int[][]{
@@ -623,6 +672,25 @@ public class TestIntervals extends LuceneTestCase {
 
   }
 
+  public void testMaxGapsWithRepeats() throws IOException {
+    IntervalsSource source = Intervals.maxgaps(11,
+        Intervals.ordered(Intervals.term("pease"), Intervals.term("pease"), Intervals.term("hot")));
+    checkIntervals(source, "field1", 1, new int[][]{
+        {}, {}, { 0, 5 }, {}, {}, {}
+    });
+    assertGaps(source, 2, "field1", new int[]{ 3 });
+  }
+
+  public void testMaxGapsWithOnlyRepeats() throws IOException {
+    IntervalsSource source = Intervals.maxgaps(1, Intervals.ordered(
+        Intervals.or(Intervals.term("pease"), Intervals.term("hot")), Intervals.or(Intervals.term("pease"), Intervals.term("hot"))
+    ));
+    checkIntervals(source, "field1", 3, new int[][]{
+        {}, { 0, 2, 2, 3 }, { 3, 5, 5, 6 }, {}, { 0, 2, 2, 3 }, {}
+    });
+    assertGaps(source, 1, "field1", new int[]{ 1, 0 });
+  }
+
   public void testNestedMaxGaps() throws IOException {
     IntervalsSource source = Intervals.maxgaps(1,
         Intervals.unordered(
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestSimplifications.java b/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestSimplifications.java
index b54c2a1..76a4857 100644
--- a/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestSimplifications.java
+++ b/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestSimplifications.java
@@ -38,12 +38,26 @@ public class TestSimplifications extends LuceneTestCase {
     assertEquals(Intervals.term("term"), actual);
   }
 
+  public void testOrderedWithDuplicates() {
+    IntervalsSource actual = Intervals.ordered(Intervals.term("term"), Intervals.term("term"));
+    assertEquals("ORDERED(term,term)", actual.toString());
+    actual = Intervals.ordered(Intervals.term("term"), Intervals.term("term"), Intervals.term("bar"));
+    assertEquals("ORDERED(term,term,bar)", actual.toString());
+  }
+
   public void testUnordered() {
     // UNORDERED(term) => term
     IntervalsSource actual = Intervals.unordered(Intervals.term("term"));
     assertEquals(Intervals.term("term"), actual);
   }
 
+  public void testUnorderedWithDuplicates() {
+    IntervalsSource actual = Intervals.unordered(Intervals.term("term"), Intervals.term("term"));
+    assertEquals("UNORDERED(term,term)", actual.toString());
+    actual = Intervals.unordered(Intervals.term("term"), Intervals.term("term"), Intervals.term("bar"));
+    assertEquals("UNORDERED(term,term,bar)", actual.toString());
+  }
+
   public void testUnorderedOverlaps() {
     // UNORDERED_NO_OVERLAPS(term, term) => ORDERED(term, term)
     IntervalsSource actual = Intervals.unorderedNoOverlaps(Intervals.term("term"), Intervals.term("term"));