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/12/11 09:28:02 UTC

[33/47] lucene-solr:jira/http2: LUCENE-8597: Add IntervalIterator.gaps() and Intervals.maxgaps()

LUCENE-8597: Add IntervalIterator.gaps() and Intervals.maxgaps()


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

Branch: refs/heads/jira/http2
Commit: 1305501dbd5bf79811eab9da873cf1567605e6a5
Parents: 18356de
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Dec 10 09:47:29 2018 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Dec 10 09:47:29 2018 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +
 .../intervals/DifferenceIntervalFunction.java   |  10 ++
 .../intervals/DisjunctionIntervalsSource.java   |  15 +++
 .../intervals/FilteredIntervalsSource.java      | 103 +++++++++++++++++++
 .../lucene/search/intervals/IntervalFilter.java |   5 +
 .../search/intervals/IntervalFunction.java      |  54 +++++++++-
 .../search/intervals/IntervalIterator.java      |  11 ++
 .../search/intervals/IntervalMatches.java       |   9 ++
 .../intervals/IntervalMatchesIterator.java      |  38 +++++++
 .../lucene/search/intervals/Intervals.java      |  23 ++++-
 .../intervals/LowpassIntervalsSource.java       |  90 ----------------
 .../MinimizingConjunctionIntervalsSource.java   |   7 +-
 .../search/intervals/TermIntervalsSource.java   |   5 +
 .../lucene/search/intervals/package-info.java   |   2 +
 .../search/intervals/TestIntervalQuery.java     |  10 ++
 .../lucene/search/intervals/TestIntervals.java  |  60 ++++++++++-
 16 files changed, 349 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 007d35b..41259bd 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -98,6 +98,10 @@ API Changes
   methods.  This decouples normalization from tokenization entirely.
   (Mayya Sharipova, Alan Woodward)
 
+* LUCENE-8597: IntervalIterator now exposes a gaps() method that reports the
+  number of gaps between its component sub-intervals.  This can be used in a 
+  new filter available via Intervals.maxgaps().  (Alan Woodward)
+
 Changes in Runtime Behavior
 
 * LUCENE-8333: Switch MoreLikeThis.setMaxDocFreqPct to use maxDoc instead of

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalFunction.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalFunction.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalFunction.java
index 18d4d67..def1d03 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalFunction.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/DifferenceIntervalFunction.java
@@ -126,6 +126,11 @@ abstract class DifferenceIntervalFunction {
     }
 
     @Override
+    public int gaps() {
+      return a.gaps();
+    }
+
+    @Override
     public float matchCost() {
       return a.matchCost() + b.matchCost();
     }
@@ -233,6 +238,11 @@ abstract class DifferenceIntervalFunction {
         }
 
         @Override
+        public int gaps() {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
         public int nextInterval() throws IOException {
           if (positioned == false) {
             positioned = true;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/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 c54f18b..68a9e5d 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
@@ -135,6 +135,11 @@ class DisjunctionIntervalsSource extends IntervalsSource {
       return current.end();
     }
 
+    @Override
+    public int gaps() {
+      return current.gaps();
+    }
+
     private void reset() throws IOException {
       intervalQueue.clear();
       for (DisiWrapper dw = disiQueue.topList(); dw != null; dw = dw.next) {
@@ -229,6 +234,11 @@ class DisjunctionIntervalsSource extends IntervalsSource {
     }
 
     @Override
+    public int gaps() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
     public int nextInterval() {
       return NO_MORE_INTERVALS;
     }
@@ -272,6 +282,11 @@ class DisjunctionIntervalsSource extends IntervalsSource {
     }
 
     @Override
+    public int gaps() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
     public int nextInterval() {
       return NO_MORE_INTERVALS;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/lucene/sandbox/src/java/org/apache/lucene/search/intervals/FilteredIntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/FilteredIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/FilteredIntervalsSource.java
new file mode 100644
index 0000000..8eac88d
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/FilteredIntervalsSource.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.intervals;
+
+import java.io.IOException;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.MatchesIterator;
+
+/**
+ * An IntervalsSource that filters the intervals from another IntervalsSource
+ */
+public abstract class FilteredIntervalsSource extends IntervalsSource {
+
+  private final String name;
+  private final IntervalsSource in;
+
+  /**
+   * Create a new FilteredIntervalsSource
+   * @param name  the name of the filter
+   * @param in    the source to filter
+   */
+  public FilteredIntervalsSource(String name, IntervalsSource in) {
+    this.name = name;
+    this.in = in;
+  }
+
+  /**
+   * @return {@code false} if the current interval should be filtered out
+   */
+  protected abstract boolean accept(IntervalIterator it);
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    IntervalIterator i = in.intervals(field, ctx);
+    if (i == null) {
+      return null;
+    }
+    return new IntervalFilter(i) {
+      @Override
+      protected boolean accept() {
+        return FilteredIntervalsSource.this.accept(in);
+      }
+    };
+  }
+
+  @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 FilteredIntervalsSource.this.accept(in);
+      }
+    };
+    return IntervalMatches.asMatches(filtered, mi, doc);
+  }
+
+  @Override
+  public void extractTerms(String field, Set<Term> terms) {
+    in.extractTerms(field, terms);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    FilteredIntervalsSource that = (FilteredIntervalsSource) o;
+    return Objects.equals(name, that.name) &&
+        Objects.equals(in, that.in);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(name, in);
+  }
+
+  @Override
+  public String toString() {
+    return name + "(" + in + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/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 a501b49..46c0302 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
@@ -65,6 +65,11 @@ public abstract class IntervalFilter extends IntervalIterator {
   }
 
   @Override
+  public int gaps() {
+    return in.gaps();
+  }
+
+  @Override
   public float matchCost() {
     return in.matchCost();
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/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 787574e..1a5eab6 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
@@ -18,6 +18,7 @@
 package org.apache.lucene.search.intervals;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.lucene.util.PriorityQueue;
@@ -67,6 +68,11 @@ abstract class IntervalFunction {
     }
 
     @Override
+    public int gaps() {
+      return 0;
+    }
+
+    @Override
     public int nextInterval() throws IOException {
       if (subIterators.get(0).nextInterval() == IntervalIterator.NO_MORE_INTERVALS)
         return IntervalIterator.NO_MORE_INTERVALS;
@@ -109,6 +115,7 @@ abstract class IntervalFunction {
   private static class OrderedIntervalIterator extends ConjunctionIntervalIterator {
 
     int start = -1, end = -1, i;
+    int firstEnd;
 
     private OrderedIntervalIterator(List<IntervalIterator> subIntervals) {
       super(subIntervals);
@@ -143,6 +150,7 @@ abstract class IntervalFunction {
           i++;
         }
         start = subIterators.get(0).start();
+        firstEnd = subIterators.get(0).end();
         end = subIterators.get(subIterators.size() - 1).end();
         b = subIterators.get(subIterators.size() - 1).start();
         i = 1;
@@ -152,10 +160,19 @@ abstract class IntervalFunction {
     }
 
     @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;
+    }
+
+    @Override
     protected void reset() throws IOException {
       subIterators.get(0).nextInterval();
       i = 1;
-      start = end = -1;
+      start = end = firstEnd = -1;
     }
   }
 
@@ -183,9 +200,10 @@ abstract class IntervalFunction {
 
     private final PriorityQueue<IntervalIterator> queue;
     private final IntervalIterator[] subIterators;
+    private final int[] innerPositions;
     private final boolean allowOverlaps;
 
-    int start = -1, end = -1, queueEnd;
+    int start = -1, end = -1, firstEnd, queueEnd;
 
     UnorderedIntervalIterator(List<IntervalIterator> subIterators, boolean allowOverlaps) {
       super(subIterators);
@@ -196,6 +214,7 @@ abstract class IntervalFunction {
         }
       };
       this.subIterators = new IntervalIterator[subIterators.size()];
+      this.innerPositions = new int[subIterators.size() * 2];
       this.allowOverlaps = allowOverlaps;
 
       for (int i = 0; i < subIterators.size(); i++) {
@@ -241,6 +260,7 @@ abstract class IntervalFunction {
       // then, minimize it
       do {
         start = queue.top().start();
+        firstEnd = queue.top().end();
         end = queueEnd;
         if (queue.top().end() == end)
           return start;
@@ -261,6 +281,26 @@ abstract class IntervalFunction {
     }
 
     @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;
+    }
+
+    @Override
     protected void reset() throws IOException {
       queueEnd = start = end = -1;
       this.queue.clear();
@@ -325,6 +365,11 @@ abstract class IntervalFunction {
         }
 
         @Override
+        public int gaps() {
+          return a.gaps();
+        }
+
+        @Override
         public int nextInterval() throws IOException {
           if (bpos == false)
             return IntervalIterator.NO_MORE_INTERVALS;
@@ -372,6 +417,11 @@ abstract class IntervalFunction {
         }
 
         @Override
+        public int gaps() {
+          return a.gaps();
+        }
+
+        @Override
         public int nextInterval() throws IOException {
           if (bpos == false)
             return IntervalIterator.NO_MORE_INTERVALS;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/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 613cb78..f819aab 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
@@ -60,6 +60,17 @@ public abstract class IntervalIterator extends DocIdSetIterator {
   public abstract int end();
 
   /**
+   * The number of gaps within the current interval
+   *
+   * Note that this returns the number of gaps between the immediate sub-intervals
+   * of this interval, and does not include the gaps inside those sub-intervals.
+   *
+   * Should not be called before {@link #nextInterval()}, or after it has returned
+   * {@link #NO_MORE_INTERVALS}
+   */
+  public abstract int gaps();
+
+  /**
    * Advance the iterator to the next interval
    *
    * @return the start of the next interval, or {@link IntervalIterator#NO_MORE_INTERVALS} if

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/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
index 89f115d..a28f6e4 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalMatches.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalMatches.java
@@ -99,6 +99,15 @@ final class IntervalMatches {
       }
 
       @Override
+      public int gaps() {
+        assert state == State.ITERATING;
+        if (mi instanceof IntervalMatchesIterator) {
+          return ((IntervalMatchesIterator)mi).gaps();
+        }
+        return 0;
+      }
+
+      @Override
       public int nextInterval() throws IOException {
         assert state == State.ITERATING;
         if (mi.next()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalMatchesIterator.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalMatchesIterator.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalMatchesIterator.java
new file mode 100644
index 0000000..55482e3
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/IntervalMatchesIterator.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.intervals;
+
+import org.apache.lucene.search.MatchesIterator;
+
+/**
+ * An extension of MatchesIterator that allows the gaps from a wrapped
+ * IntervalIterator to be reported.
+ *
+ * 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.
+ */
+interface IntervalMatchesIterator extends MatchesIterator {
+
+  /**
+   * The number of top-level gaps inside the current match
+   */
+  int gaps();
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/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 e9f1bd2..b0a4829 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
@@ -77,11 +77,30 @@ public final class Intervals {
 
   /**
    * Create an {@link IntervalsSource} that filters a sub-source by the width of its intervals
-   * @param width       the maximum width of intervals in the sub-source ot return
+   * @param width       the maximum width of intervals in the sub-source to filter
    * @param subSource   the sub-source to filter
    */
   public static IntervalsSource maxwidth(int width, IntervalsSource subSource) {
-    return new LowpassIntervalsSource(subSource, width);
+    return new FilteredIntervalsSource("MAXWIDTH/" + width, subSource) {
+      @Override
+      protected boolean accept(IntervalIterator it) {
+        return (it.end() - it.start()) + 1 <= width;
+      }
+    };
+  }
+
+  /**
+   * Create an {@link IntervalsSource} that filters a sub-source by its gaps
+   * @param gaps        the maximum number of gaps in the sub-source to filter
+   * @param subSource   the sub-source to filter
+   */
+  public static IntervalsSource maxgaps(int gaps, IntervalsSource subSource) {
+    return new FilteredIntervalsSource("MAXGAPS/" + gaps, subSource) {
+      @Override
+      protected boolean accept(IntervalIterator it) {
+        return it.gaps() <= gaps;
+      }
+    };
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/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
deleted file mode 100644
index 61cb1fc..0000000
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/LowpassIntervalsSource.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.lucene.search.intervals;
-
-import java.io.IOException;
-import java.util.Objects;
-import java.util.Set;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.MatchesIterator;
-
-class LowpassIntervalsSource extends IntervalsSource {
-
-  final IntervalsSource in;
-  private final int maxWidth;
-
-  LowpassIntervalsSource(IntervalsSource in, int maxWidth) {
-    this.in = in;
-    this.maxWidth = maxWidth;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-    LowpassIntervalsSource that = (LowpassIntervalsSource) o;
-    return maxWidth == that.maxWidth &&
-        Objects.equals(in, that.in);
-  }
-
-  @Override
-  public String toString() {
-    return "MAXWIDTH/" + maxWidth + "(" + in + ")";
-  }
-
-  @Override
-  public void extractTerms(String field, Set<Term> terms) {
-    in.extractTerms(field, terms);
-  }
-
-  @Override
-  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
-    IntervalIterator i = in.intervals(field, ctx);
-    if (i == null) {
-      return null;
-    }
-    return new IntervalFilter(i) {
-      @Override
-      protected boolean accept() {
-        return (i.end() - i.start()) + 1 <= maxWidth;
-      }
-    };
-  }
-
-  @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/1305501d/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
index 669acae..c509692 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/MinimizingConjunctionIntervalsSource.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/MinimizingConjunctionIntervalsSource.java
@@ -61,7 +61,7 @@ class MinimizingConjunctionIntervalsSource extends ConjunctionIntervalsSource {
     return new ConjunctionMatchesIterator(it, subs);
   }
 
-  private static class ConjunctionMatchesIterator implements MatchesIterator {
+  private static class ConjunctionMatchesIterator implements IntervalMatchesIterator {
 
     final IntervalIterator iterator;
     final List<CacheingMatchesIterator> subs;
@@ -112,6 +112,11 @@ class MinimizingConjunctionIntervalsSource extends ConjunctionIntervalsSource {
     }
 
     @Override
+    public int gaps() {
+      return iterator.gaps();
+    }
+
+    @Override
     public MatchesIterator getSubMatches() throws IOException {
       List<MatchesIterator> mis = new ArrayList<>();
       int endPos = endPosition();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/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 27e5b79..f58d867 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
@@ -96,6 +96,11 @@ class TermIntervalsSource extends IntervalsSource {
       }
 
       @Override
+      public int gaps() {
+        return 0;
+      }
+
+      @Override
       public int nextInterval() throws IOException {
         if (upto <= 0)
           return pos = NO_MORE_INTERVALS;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/lucene/sandbox/src/java/org/apache/lucene/search/intervals/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/package-info.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/package-info.java
index 88d93ea..cfcd9e7 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/package-info.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/package-info.java
@@ -44,6 +44,8 @@
  * <ul>
  *   <li>{@link org.apache.lucene.search.intervals.Intervals#maxwidth(int, org.apache.lucene.search.intervals.IntervalsSource)}
  *          &mdash; Filters out intervals that are larger than a set width</li>
+ *   <li>{@link org.apache.lucene.search.intervals.Intervals#maxgaps(int, org.apache.lucene.search.intervals.IntervalsSource)}
+ *          &mdash; Filters out intervals that have more than a set number of gaps between their constituent sub-intervals</li>
  *   <li>{@link org.apache.lucene.search.intervals.Intervals#containedBy(org.apache.lucene.search.intervals.IntervalsSource, org.apache.lucene.search.intervals.IntervalsSource)}
  *          &mdash; Returns intervals that are contained by another interval</li>
  *   <li>{@link org.apache.lucene.search.intervals.Intervals#notContainedBy(org.apache.lucene.search.intervals.IntervalsSource, org.apache.lucene.search.intervals.IntervalsSource)}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervalQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervalQuery.java b/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervalQuery.java
index 8f0623e..18c69a7 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervalQuery.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/intervals/TestIntervalQuery.java
@@ -91,6 +91,16 @@ public class TestIntervalQuery extends LuceneTestCase {
         new int[]{0, 1, 2, 3, 5});
   }
 
+  public void testOrderedNearQueryGaps1() throws IOException {
+    checkHits(new IntervalQuery(field, Intervals.maxgaps(1, Intervals.ordered(Intervals.term("w1"), Intervals.term("w2")))),
+        new int[]{0, 1, 2, 5});
+  }
+
+  public void testOrderedNearQueryGaps2() throws IOException {
+    checkHits(new IntervalQuery(field, Intervals.maxgaps(2, Intervals.ordered(Intervals.term("w1"), Intervals.term("w2")))),
+        new int[]{0, 1, 2, 3, 5});
+  }
+
   public void testNestedOrderedNearQuery() throws IOException {
     // onear/1(w1, onear/2(w2, w3))
     Query q = new IntervalQuery(field,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1305501d/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 6002b3a..0dee62f 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
@@ -64,7 +64,7 @@ public class TestIntervals extends LuceneTestCase {
       "Down to a sunless sea",
       "So thrice five miles of fertile ground",
       "Pease hot porridge porridge",
-      "Pease porridge porridge hot"
+      "w1 w2 w3 w4 w1 w6 w3 w8 w4 w7 w1 w6"
   };
 
   private static Directory directory;
@@ -84,7 +84,7 @@ public class TestIntervals extends LuceneTestCase {
     for (int i = 0; i < field1_docs.length; i++) {
       Document doc = new Document();
       doc.add(new Field("field1", field1_docs[i], FIELD_TYPE));
-      doc.add(new TextField("field2", field2_docs[i], Field.Store.NO));
+      doc.add(new Field("field2", field2_docs[i], FIELD_TYPE));
       doc.add(new StringField("id", Integer.toString(i), Field.Store.NO));
       doc.add(new NumericDocValuesField("id", i));
       writer.addDocument(doc);
@@ -149,6 +149,19 @@ public class TestIntervals extends LuceneTestCase {
     assertEquals(endOffset, mi.endOffset());
   }
 
+  private void assertGaps(IntervalsSource source, int doc, String field, int[] expectedGaps) throws IOException {
+    int ord = ReaderUtil.subIndex(doc, searcher.getIndexReader().leaves());
+    LeafReaderContext ctx = searcher.getIndexReader().leaves().get(ord);
+    IntervalIterator it = source.intervals(field, ctx);
+    assertEquals(doc, it.advance(doc));
+    for (int expectedGap : expectedGaps) {
+      if (it.nextInterval() == IntervalIterator.NO_MORE_INTERVALS) {
+        fail("Unexpected interval " + it);
+      }
+      assertEquals(expectedGap, it.gaps());
+    }
+  }
+
   public void testIntervalsOnFieldWithNoPositions() throws IOException {
     IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> {
       Intervals.term("wibble").intervals("id", searcher.getIndexReader().leaves().get(0));
@@ -241,6 +254,10 @@ public class TestIntervals extends LuceneTestCase {
     assertMatch(sub, 17, 17, 96, 99);
     assertFalse(sub.next());
     assertFalse(mi.next());
+
+    assertGaps(source, 1, "field1", new int[]{
+        1, 0, 10
+    });
   }
 
   public void testIntervalDisjunction() throws IOException {
@@ -287,6 +304,7 @@ 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);
@@ -302,6 +320,11 @@ public class TestIntervals extends LuceneTestCase {
     assertMatch(mi, 5, 7, 35, 55);
     assertMatch(mi, 6, 17, 41, 99);
     assertFalse(mi.next());
+
+    assertGaps(source, 1, "field1", new int[]{
+        0, 0, 0, 0, 0, 0, 9
+    });
+
   }
 
   public void testNesting2() throws IOException {
@@ -447,4 +470,37 @@ public class TestIntervals extends LuceneTestCase {
     assertFalse(mi.next());
   }
 
+  public void testMaxGaps() throws IOException {
+
+    IntervalsSource source = Intervals.maxgaps(1,
+        Intervals.unordered(Intervals.term("w1"), Intervals.term("w3"), Intervals.term("w4")));
+    checkIntervals(source, "field2", 1, new int[][]{
+        {}, {}, {}, {}, {},
+        { 0, 3, 2, 4, 3, 6 }
+    });
+
+    MatchesIterator mi = getMatches(source, 5, "field2");
+    assertMatch(mi, 0, 3, 0, 11);
+
+  }
+
+  public void testNestedMaxGaps() throws IOException {
+    IntervalsSource source = Intervals.maxgaps(1,
+        Intervals.unordered(
+            Intervals.ordered(Intervals.term("w1"), Intervals.term("w3")),
+            Intervals.term("w4")
+        ));
+    checkIntervals(source, "field2", 1, new int[][]{
+        {}, {}, {}, {}, {},
+        { 0, 3, 3, 6, 4, 8 }
+    });
+
+    assertGaps(source, 5, "field2", new int[]{ 0, 0, 1 });
+
+    MatchesIterator mi = getMatches(source, 5, "field2");
+    assertMatch(mi, 0, 3, 0, 11);
+    assertMatch(mi, 3, 6, 9, 20);
+    assertMatch(mi, 4, 8, 12, 26);
+  }
+
 }