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 2019/01/07 14:18:40 UTC

[1/2] lucene-solr:branch_8x: LUCENE-8622: Minimum-should-match interval function

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_8x 6db1f4eb7 -> bfbe80472


LUCENE-8622: Minimum-should-match interval function


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

Branch: refs/heads/branch_8x
Commit: bfbe80472ef330f3ad264eda7a9ef48eb91b58cc
Parents: 3593af2
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Jan 7 13:53:19 2019 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Jan 7 14:18:25 2019 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../intervals/CachingMatchesIterator.java       | 132 +++++++
 .../lucene/search/intervals/Intervals.java      |   7 +
 .../MinimizingConjunctionIntervalsSource.java   | 122 +-----
 .../MinimumShouldMatchIntervalsSource.java      | 387 +++++++++++++++++++
 .../lucene/search/intervals/TestIntervals.java  |  33 ++
 6 files changed, 569 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bfbe8047/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 0d10cfa..4f677e7 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -177,6 +177,9 @@ New Features
 * LUCENE-8629: New interval functions: Intervals.before(), Intervals.after(),
   Intervals.within() and Intervals.overlapping(). (Alan Woodward)
 
+* LUCENE-8622: Adds a minimum-should-match interval function that produces intervals
+  spanning a subset of a set of sources. (Alan Woodward)
+
 Improvements
 
 * LUCENE-7997: Add BaseSimilarityTestCase to sanity check similarities.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bfbe8047/lucene/sandbox/src/java/org/apache/lucene/search/intervals/CachingMatchesIterator.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/CachingMatchesIterator.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/CachingMatchesIterator.java
new file mode 100644
index 0000000..d522412
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/CachingMatchesIterator.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.intervals;
+
+import java.io.IOException;
+
+import org.apache.lucene.search.FilterMatchesIterator;
+import org.apache.lucene.search.MatchesIterator;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.ArrayUtil;
+
+class CachingMatchesIterator extends FilterMatchesIterator {
+
+  private boolean positioned = false;
+  private int[] posAndOffsets = new int[16];
+  private int count = 0;
+
+  CachingMatchesIterator(MatchesIterator in) {
+    super(in);
+  }
+
+  private void cache() throws IOException {
+    count = 0;
+    MatchesIterator mi = in.getSubMatches();
+    if (mi == null) {
+      count = 1;
+      posAndOffsets[0] = in.startPosition();
+      posAndOffsets[1] = in.endPosition();
+      posAndOffsets[2] = in.startOffset();
+      posAndOffsets[3] = in.endOffset();
+    }
+    else {
+      while (mi.next()) {
+        if (count * 4 >= posAndOffsets.length) {
+          posAndOffsets = ArrayUtil.grow(posAndOffsets, (count + 1) * 4);
+        }
+        posAndOffsets[count * 4] = mi.startPosition();
+        posAndOffsets[count * 4 + 1] = mi.endPosition();
+        posAndOffsets[count * 4 + 2] = mi.startOffset();
+        posAndOffsets[count * 4 + 3] = mi.endOffset();
+        count++;
+      }
+    }
+  }
+
+  @Override
+  public boolean next() throws IOException {
+    if (positioned == false) {
+      positioned = true;
+    }
+    else {
+      cache();
+    }
+    return in.next();
+  }
+
+  int startOffset(int endPos) throws IOException {
+    if (endPosition() <= endPos) {
+      return in.startOffset();
+    }
+    return posAndOffsets[2];
+  }
+
+  int endOffset(int endPos) throws IOException {
+    if (endPosition() <= endPos) {
+      return in.endOffset();
+    }
+    return posAndOffsets[count * 4 + 3];
+  }
+
+  MatchesIterator getSubMatches(int endPos) throws IOException {
+    if (endPosition() <= endPos) {
+      cache();
+    }
+    return new MatchesIterator() {
+
+      int upto = -1;
+
+      @Override
+      public boolean next() {
+        upto++;
+        return upto < count;
+      }
+
+      @Override
+      public int startPosition() {
+        return posAndOffsets[upto * 4];
+      }
+
+      @Override
+      public int endPosition() {
+        return posAndOffsets[upto * 4 + 1];
+      }
+
+      @Override
+      public int startOffset() {
+        return posAndOffsets[upto * 4 + 2];
+      }
+
+      @Override
+      public int endOffset() {
+        return posAndOffsets[upto * 4 + 3];
+      }
+
+      @Override
+      public MatchesIterator getSubMatches() {
+        return null;
+      }
+
+      @Override
+      public Query getQuery() {
+        throw new UnsupportedOperationException();
+      }
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bfbe8047/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 aa5b62c..1b6dbae 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
@@ -259,6 +259,13 @@ public final class Intervals {
   }
 
   /**
+   * Return intervals that span combinations of intervals from {@code minShouldMatch} of the sources
+   */
+  public static IntervalsSource atLeast(int minShouldMatch, IntervalsSource... sources) {
+    return new MinimumShouldMatchIntervalsSource(sources, minShouldMatch);
+  }
+
+  /**
    * Returns intervals from the source that appear before intervals from the reference
    */
   public static IntervalsSource before(IntervalsSource source, IntervalsSource reference) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bfbe8047/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 c509692..6e6f563 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
@@ -23,11 +23,9 @@ import java.util.List;
 import java.util.stream.Collectors;
 
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.FilterMatchesIterator;
 import org.apache.lucene.search.MatchesIterator;
 import org.apache.lucene.search.MatchesUtils;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.util.ArrayUtil;
 
 /**
  * A ConjunctionIntervalsSource that attempts to minimize its internal intervals by
@@ -43,13 +41,13 @@ class MinimizingConjunctionIntervalsSource extends ConjunctionIntervalsSource {
 
   @Override
   public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
-    List<CacheingMatchesIterator> subs = new ArrayList<>();
+    List<CachingMatchesIterator> subs = new ArrayList<>();
     for (IntervalsSource source : subSources) {
       MatchesIterator mi = source.matches(field, ctx, doc);
       if (mi == null) {
         return null;
       }
-      subs.add(new CacheingMatchesIterator(mi));
+      subs.add(new CachingMatchesIterator(mi));
     }
     IntervalIterator it = function.apply(subs.stream().map(m -> IntervalMatches.wrapMatches(m, doc)).collect(Collectors.toList()));
     if (it.advance(doc) != doc) {
@@ -64,10 +62,10 @@ class MinimizingConjunctionIntervalsSource extends ConjunctionIntervalsSource {
   private static class ConjunctionMatchesIterator implements IntervalMatchesIterator {
 
     final IntervalIterator iterator;
-    final List<CacheingMatchesIterator> subs;
+    final List<CachingMatchesIterator> subs;
     boolean cached = true;
 
-    private ConjunctionMatchesIterator(IntervalIterator iterator, List<CacheingMatchesIterator> subs) {
+    private ConjunctionMatchesIterator(IntervalIterator iterator, List<CachingMatchesIterator> subs) {
       this.iterator = iterator;
       this.subs = subs;
     }
@@ -95,7 +93,7 @@ class MinimizingConjunctionIntervalsSource extends ConjunctionIntervalsSource {
     public int startOffset() throws IOException {
       int start = Integer.MAX_VALUE;
       int endPos = endPosition();
-      for (CacheingMatchesIterator s : subs) {
+      for (CachingMatchesIterator s : subs) {
         start = Math.min(start, s.startOffset(endPos));
       }
       return start;
@@ -105,7 +103,7 @@ class MinimizingConjunctionIntervalsSource extends ConjunctionIntervalsSource {
     public int endOffset() throws IOException {
       int end = 0;
       int endPos = endPosition();
-      for (CacheingMatchesIterator s : subs) {
+      for (CachingMatchesIterator s : subs) {
         end = Math.max(end, s.endOffset(endPos));
       }
       return end;
@@ -120,7 +118,7 @@ class MinimizingConjunctionIntervalsSource extends ConjunctionIntervalsSource {
     public MatchesIterator getSubMatches() throws IOException {
       List<MatchesIterator> mis = new ArrayList<>();
       int endPos = endPosition();
-      for (CacheingMatchesIterator s : subs) {
+      for (CachingMatchesIterator s : subs) {
         mis.add(s.getSubMatches(endPos));
       }
       return MatchesUtils.disjunction(mis);
@@ -132,110 +130,4 @@ class MinimizingConjunctionIntervalsSource extends ConjunctionIntervalsSource {
     }
   }
 
-  private static class CacheingMatchesIterator extends FilterMatchesIterator {
-
-    boolean positioned = false;
-    int posAndOffsets[] = new int[16];
-    int count = 0;
-
-    CacheingMatchesIterator(MatchesIterator in) {
-      super(in);
-    }
-
-    private void cache() throws IOException {
-      count = 0;
-      MatchesIterator mi = in.getSubMatches();
-      if (mi == null) {
-        count = 1;
-        posAndOffsets[0] = in.startPosition();
-        posAndOffsets[1] = in.endPosition();
-        posAndOffsets[2] = in.startOffset();
-        posAndOffsets[3] = in.endOffset();
-      }
-      else {
-        while (mi.next()) {
-          if (count * 4 >= posAndOffsets.length) {
-            posAndOffsets = ArrayUtil.grow(posAndOffsets, (count + 1) * 4);
-          }
-          posAndOffsets[count * 4] = mi.startPosition();
-          posAndOffsets[count * 4 + 1] = mi.endPosition();
-          posAndOffsets[count * 4 + 2] = mi.startOffset();
-          posAndOffsets[count * 4 + 3] = mi.endOffset();
-          count++;
-        }
-      }
-    }
-
-    @Override
-    public boolean next() throws IOException {
-      if (positioned == false) {
-        positioned = true;
-      }
-      else {
-        cache();
-      }
-      return in.next();
-    }
-
-    int startOffset(int endPos) throws IOException {
-      if (endPosition() <= endPos) {
-        return in.startOffset();
-      }
-      return posAndOffsets[2];
-    }
-
-    int endOffset(int endPos) throws IOException {
-      if (endPosition() <= endPos) {
-        return in.endOffset();
-      }
-      return posAndOffsets[count * 4 + 3];
-    }
-
-    MatchesIterator getSubMatches(int endPos) throws IOException {
-      if (endPosition() <= endPos) {
-        cache();
-      }
-      return new MatchesIterator() {
-
-        int upto = -1;
-
-        @Override
-        public boolean next() {
-          upto++;
-          return upto < count;
-        }
-
-        @Override
-        public int startPosition() {
-          return posAndOffsets[upto * 4];
-        }
-
-        @Override
-        public int endPosition() {
-          return posAndOffsets[upto * 4 + 1];
-        }
-
-        @Override
-        public int startOffset() {
-          return posAndOffsets[upto * 4 + 2];
-        }
-
-        @Override
-        public int endOffset() {
-          return posAndOffsets[upto * 4 + 3];
-        }
-
-        @Override
-        public MatchesIterator getSubMatches() {
-          return null;
-        }
-
-        @Override
-        public Query getQuery() {
-          throw new UnsupportedOperationException();
-        }
-      };
-    }
-
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bfbe8047/lucene/sandbox/src/java/org/apache/lucene/search/intervals/MinimumShouldMatchIntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/MinimumShouldMatchIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/MinimumShouldMatchIntervalsSource.java
new file mode 100644
index 0000000..3c53963
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/MinimumShouldMatchIntervalsSource.java
@@ -0,0 +1,387 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.intervals;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.MatchesIterator;
+import org.apache.lucene.search.MatchesUtils;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.PriorityQueue;
+
+class MinimumShouldMatchIntervalsSource extends IntervalsSource {
+
+  private final IntervalsSource[] sources;
+  private final int minShouldMatch;
+
+  MinimumShouldMatchIntervalsSource(IntervalsSource[] sources, int minShouldMatch) {
+    this.sources = sources;
+    this.minShouldMatch = minShouldMatch;
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    List<IntervalIterator> iterators = new ArrayList<>();
+    for (IntervalsSource source : sources) {
+      IntervalIterator it = source.intervals(field, ctx);
+      if (it != null) {
+        iterators.add(it);
+      }
+    }
+    if (iterators.size() < minShouldMatch) {
+      return null;
+    }
+    return new MinimumShouldMatchIntervalIterator(iterators, minShouldMatch);
+  }
+
+  @Override
+  public MatchesIterator 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);
+      if (mi != null) {
+        CachingMatchesIterator cmi = new CachingMatchesIterator(mi);
+        lookup.put(IntervalMatches.wrapMatches(cmi, doc), cmi);
+      }
+    }
+    if (lookup.size() < minShouldMatch) {
+      return null;
+    }
+    MinimumShouldMatchIntervalIterator it = new MinimumShouldMatchIntervalIterator(lookup.keySet(), minShouldMatch);
+    if (it.advance(doc) != doc) {
+      return null;
+    }
+    if (it.nextInterval() == IntervalIterator.NO_MORE_INTERVALS) {
+      return null;
+    }
+    return new MinimumMatchesIterator(it, lookup);
+  }
+
+  @Override
+  public void extractTerms(String field, Set<Term> terms) {
+    for (IntervalsSource source : sources) {
+      source.extractTerms(field, terms);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "ProxBoost("
+        + Arrays.stream(sources).map(IntervalsSource::toString).collect(Collectors.joining(","))
+        + "~" + minShouldMatch + ")";
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    MinimumShouldMatchIntervalsSource that = (MinimumShouldMatchIntervalsSource) o;
+    return minShouldMatch == that.minShouldMatch &&
+        Arrays.equals(sources, that.sources);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = Objects.hash(minShouldMatch);
+    result = 31 * result + Arrays.hashCode(sources);
+    return result;
+  }
+
+  // This works as a combination of unordered-AND and OR
+  // First of all, iterators are advanced using a DisjunctionDISIApproximation
+  // Once positioned on a document, nextInterval() is called on each interval, and
+  // those that have intervals are added to an OR-based priority queue (the background queue)
+  // The top-n iterators (where n = minimumShouldMatch) are popped from this queue
+  // and added to an AND-based priority queue (the proximity queue)
+  // Iteration over intervals then proceeds according to the algorithm used by
+  // UnorderedIntervalIterator based on intervals in the proximity queue, with
+  // the one change that when an iterator is popped from the proximity queue, it
+  // is inserted back into the background queue, and replaced by the top iterator
+  // from the background queue.
+  static class MinimumShouldMatchIntervalIterator extends IntervalIterator {
+
+    private final DocIdSetIterator approximation;
+    private final DisiPriorityQueue disiQueue;
+    private final PriorityQueue<IntervalIterator> proximityQueue;
+    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 IntervalIterator lead;
+
+    MinimumShouldMatchIntervalIterator(Collection<IntervalIterator> subs, int minShouldMatch) {
+      this.disiQueue = new DisiPriorityQueue(subs.size());
+      float mc = 0;
+      for (IntervalIterator it : subs) {
+        this.disiQueue.add(new DisiWrapper(it));
+        mc += it.matchCost();
+      }
+      this.approximation = new DisjunctionDISIApproximation(disiQueue);
+      this.matchCost = mc;
+      this.minShouldMatch = minShouldMatch;
+      this.innerPositions = new int[minShouldMatch * 2];
+
+      this.proximityQueue = new PriorityQueue<IntervalIterator>(minShouldMatch) {
+        @Override
+        protected boolean lessThan(IntervalIterator a, IntervalIterator b) {
+          return a.start() < b.start() || (a.start() == b.start() && a.end() >= b.end());
+        }
+      };
+      this.backgroundQueue = new PriorityQueue<IntervalIterator>(subs.size()) {
+        @Override
+        protected boolean lessThan(IntervalIterator a, IntervalIterator b) {
+          return a.end() < b.end() || (a.end() == b.end() && a.start() >= b.start());
+        }
+      };
+    }
+
+    @Override
+    public int start() {
+      return start;
+    }
+
+    @Override
+    public int end() {
+      return end;
+    }
+
+    @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;
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      // first, find a matching interval beyond the current start
+      while (this.proximityQueue.size() == minShouldMatch && proximityQueue.top().start() == start) {
+        IntervalIterator it = proximityQueue.pop();
+        if (it != null && it.nextInterval() != IntervalIterator.NO_MORE_INTERVALS) {
+          backgroundQueue.add(it);
+          IntervalIterator next = backgroundQueue.pop();
+          assert next != null;  // it's just been added!
+          proximityQueue.add(next);
+          updateRightExtreme(next);
+        }
+      }
+      if (this.proximityQueue.size() < minShouldMatch)
+        return start = end = IntervalIterator.NO_MORE_INTERVALS;
+      // then, minimize it
+      do {
+        start = proximityQueue.top().start();
+        firstEnd = proximityQueue.top().end();
+        end = queueEnd;
+        if (proximityQueue.top().end() == end)
+          return start;
+        lead = proximityQueue.pop();
+        if (lead != null) {
+          if (lead.nextInterval() != NO_MORE_INTERVALS) {
+            backgroundQueue.add(lead);
+          }
+          IntervalIterator next = backgroundQueue.pop();
+          if (next != null) {
+            proximityQueue.add(next);
+            updateRightExtreme(next);
+          }
+        }
+      } while (this.proximityQueue.size() == minShouldMatch && end == queueEnd);
+      return start;
+    }
+
+    Collection<IntervalIterator> getCurrentIterators() {
+      currentIterators.clear();
+      currentIterators.add(lead);
+      for (IntervalIterator it : this.proximityQueue) {
+        if (it.end() <= end) {
+          currentIterators.add(it);
+        }
+      }
+      return currentIterators;
+    }
+
+    private void reset() throws IOException {
+      this.proximityQueue.clear();
+      this.backgroundQueue.clear();
+      // First we populate the background queue
+      for (DisiWrapper dw = disiQueue.topList(); dw != null; dw = dw.next) {
+        if (dw.intervals.nextInterval() != NO_MORE_INTERVALS) {
+          this.backgroundQueue.add(dw.intervals);
+        }
+      }
+      // Then we pop the first minShouldMatch entries and add them to the proximity queue
+      this.queueEnd = -1;
+      for (int i = 0; i < minShouldMatch; i++) {
+        IntervalIterator it = this.backgroundQueue.pop();
+        if (it == null) {
+          break;
+        }
+        this.proximityQueue.add(it);
+        updateRightExtreme(it);
+      }
+      start = end = -1;
+    }
+
+    private void updateRightExtreme(IntervalIterator it) {
+      int itEnd = it.end();
+      if (itEnd > queueEnd) {
+        queueEnd = itEnd;
+      }
+    }
+
+    @Override
+    public float matchCost() {
+      return matchCost;
+    }
+
+    @Override
+    public int docID() {
+      return approximation.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      int doc = approximation.nextDoc();
+      reset();
+      return doc;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      int doc = approximation.advance(target);
+      reset();
+      return doc;
+    }
+
+    @Override
+    public long cost() {
+      return approximation.cost();
+    }
+  }
+
+  static class MinimumMatchesIterator implements IntervalMatchesIterator {
+
+    boolean cached = true;
+    final MinimumShouldMatchIntervalIterator iterator;
+    final Map<IntervalIterator, CachingMatchesIterator> lookup;
+
+    MinimumMatchesIterator(MinimumShouldMatchIntervalIterator iterator,
+                           Map<IntervalIterator, CachingMatchesIterator> lookup) {
+      this.iterator = iterator;
+      this.lookup = lookup;
+    }
+
+    @Override
+    public boolean next() throws IOException {
+      if (cached) {
+        cached = false;
+        return true;
+      }
+      return iterator.nextInterval() != IntervalIterator.NO_MORE_INTERVALS;
+    }
+
+    @Override
+    public int startPosition() {
+      return iterator.start();
+    }
+
+    @Override
+    public int endPosition() {
+      return iterator.end();
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      int start = Integer.MAX_VALUE;
+      int endPos = endPosition();
+      for (IntervalIterator it : iterator.getCurrentIterators()) {
+        CachingMatchesIterator cms = lookup.get(it);
+        start = Math.min(start, cms.startOffset(endPos));
+      }
+      return start;
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      int end = 0;
+      int endPos = endPosition();
+      for (IntervalIterator it : iterator.getCurrentIterators()) {
+        CachingMatchesIterator cms = lookup.get(it);
+        end = Math.max(end, cms.endOffset(endPos));
+      }
+      return end;
+    }
+
+    @Override
+    public int gaps() {
+      return iterator.gaps();
+    }
+
+    @Override
+    public MatchesIterator getSubMatches() throws IOException {
+      List<MatchesIterator> mis = new ArrayList<>();
+      int endPos = endPosition();
+      for (IntervalIterator it : iterator.getCurrentIterators()) {
+        CachingMatchesIterator cms = lookup.get(it);
+        mis.add(cms.getSubMatches(endPos));
+      }
+      return MatchesUtils.disjunction(mis);
+    }
+
+    @Override
+    public Query getQuery() {
+      return null;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bfbe8047/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 3276caa..d1c2479 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
@@ -533,6 +533,39 @@ public class TestIntervals extends LuceneTestCase {
     assertMatch(mi, 4, 8, 12, 26);
   }
 
+  public void testMinimumShouldMatch() throws IOException {
+    IntervalsSource source = Intervals.atLeast(3,
+        Intervals.term("porridge"), Intervals.term("hot"), Intervals.term("twelve"),
+        Intervals.term("nine"), Intervals.term("pease"));
+    checkIntervals(source, "field1", 3, new int[][]{
+        {},
+        {0, 2, 1, 3, 2, 4, 6, 11, 7, 17},
+        {3, 5, 4, 6, 5, 7, 6, 11, 7, 21},
+        {},
+        {0, 2, 1, 3, 2, 4, 6, 11, 7, 17, 11, 21},
+        {}
+    });
+
+    assertGaps(source, 1, "field1", new int[]{0, 0, 0, 3, 8});
+
+    MatchesIterator mi = getMatches(source, 1, "field1");
+    assertMatch(mi, 0, 2, 0, 18);
+    MatchesIterator subs = mi.getSubMatches();
+    assertNotNull(subs);
+    assertMatch(subs, 0, 0, 0, 5);
+    assertMatch(subs, 1, 1, 6, 14);
+    assertMatch(subs, 2, 2, 15, 18);
+    assertFalse(subs.next());
+    assertTrue(mi.next());
+    assertTrue(mi.next());
+    assertMatch(mi, 6, 11, 41, 71);
+    subs = mi.getSubMatches();
+    assertMatch(subs, 6, 6, 41, 46);
+    assertMatch(subs, 7, 7, 47, 55);
+    assertMatch(subs, 11, 11, 67, 71);
+
+  }
+
   public void testDefinedGaps() throws IOException {
     IntervalsSource source = Intervals.phrase(
         Intervals.term("pease"),


[2/2] lucene-solr:branch_8x: LUCENE-8629: New interval functions: before, after, within, overlapping

Posted by ro...@apache.org.
LUCENE-8629: New interval functions: before, after, within, overlapping


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

Branch: refs/heads/branch_8x
Commit: 3593af2ac55f12a2ce2cfa76fd58407aa3fafa67
Parents: 6db1f4e
Author: Alan Woodward <ro...@apache.org>
Authored: Mon Jan 7 13:33:59 2019 +0000
Committer: Alan Woodward <ro...@apache.org>
Committed: Mon Jan 7 14:18:25 2019 +0000

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  11 +-
 .../search/intervals/IntervalFunction.java      | 119 +++++++------
 .../lucene/search/intervals/Intervals.java      |  41 ++++-
 .../search/intervals/OffsetIntervalsSource.java | 172 +++++++++++++++++++
 .../lucene/search/intervals/TestIntervals.java  | 103 +++++++++++
 5 files changed, 389 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3593af2a/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index c68af3e..0d10cfa 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -105,10 +105,6 @@ API Changes
 * LUCENE-8609: Remove IndexWriter#numDocs() and IndexWriter#maxDoc() in favor
   of IndexWriter#getDocStats(). (Simon Willnauer)
 
-* LUCENE-8612: Intervals.extend() treats an interval as if it covered a wider
-  span than it actually does, allowing users to force minimum gaps between
-  intervals in a phrase. (Alan Woodward)
-
 Changes in Runtime Behavior
 
 * LUCENE-8333: Switch MoreLikeThis.setMaxDocFreqPct to use maxDoc instead of
@@ -174,6 +170,13 @@ New Features
   to read-ahead in the token stream and take into account graph structures.  This
   also changes FixedShingleFilter to extend GraphTokenFilter (Alan Woodward)
 
+* LUCENE-8612: Intervals.extend() treats an interval as if it covered a wider
+  span than it actually does, allowing users to force minimum gaps between
+  intervals in a phrase. (Alan Woodward)
+
+* LUCENE-8629: New interval functions: Intervals.before(), Intervals.after(),
+  Intervals.within() and Intervals.overlapping(). (Alan Woodward)
+
 Improvements
 
 * LUCENE-7997: Add BaseSimilarityTestCase to sanity check similarities.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3593af2a/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 9460d8d..862e9a8 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
@@ -353,25 +353,7 @@ abstract class IntervalFunction {
         throw new IllegalStateException("CONTAINING function requires two iterators");
       IntervalIterator a = iterators.get(0);
       IntervalIterator b = iterators.get(1);
-      return new ConjunctionIntervalIterator(iterators) {
-
-        boolean bpos;
-
-        @Override
-        public int start() {
-          return a.start();
-        }
-
-        @Override
-        public int end() {
-          return a.end();
-        }
-
-        @Override
-        public int gaps() {
-          return a.gaps();
-        }
-
+      return new FilteringIntervalIterator(a, b) {
         @Override
         public int nextInterval() throws IOException {
           if (bpos == false)
@@ -386,11 +368,6 @@ abstract class IntervalFunction {
           }
           return IntervalIterator.NO_MORE_INTERVALS;
         }
-
-        @Override
-        protected void reset() throws IOException {
-          bpos = b.nextInterval() != NO_MORE_INTERVALS;
-        }
       };
     }
   };
@@ -405,31 +382,7 @@ abstract class IntervalFunction {
         throw new IllegalStateException("CONTAINED_BY function requires two iterators");
       IntervalIterator a = iterators.get(0);
       IntervalIterator b = iterators.get(1);
-      return new ConjunctionIntervalIterator(iterators) {
-
-        boolean bpos;
-
-        @Override
-        public int start() {
-          if (bpos == false) {
-            return NO_MORE_INTERVALS;
-          }
-          return a.start();
-        }
-
-        @Override
-        public int end() {
-          if (bpos == false) {
-            return NO_MORE_INTERVALS;
-          }
-          return a.end();
-        }
-
-        @Override
-        public int gaps() {
-          return a.gaps();
-        }
-
+      return new FilteringIntervalIterator(a, b) {
         @Override
         public int nextInterval() throws IOException {
           if (bpos == false)
@@ -447,15 +400,79 @@ abstract class IntervalFunction {
           bpos = false;
           return IntervalIterator.NO_MORE_INTERVALS;
         }
+      };
+    }
+  };
 
+  static final IntervalFunction OVERLAPPING = new SingletonFunction("OVERLAPPING") {
+    @Override
+    public IntervalIterator apply(List<IntervalIterator> iterators) {
+      if (iterators.size() != 2)
+        throw new IllegalStateException("OVERLAPPING function requires two iterators");
+      IntervalIterator a = iterators.get(0);
+      IntervalIterator b = iterators.get(1);
+      return new FilteringIntervalIterator(a, b) {
         @Override
-        protected void reset() throws IOException {
-          bpos = b.nextInterval() != NO_MORE_INTERVALS;
+        public int nextInterval() throws IOException {
+          if (bpos == false)
+            return IntervalIterator.NO_MORE_INTERVALS;
+          while (a.nextInterval() != IntervalIterator.NO_MORE_INTERVALS) {
+            while (b.end() < a.start()) {
+              if (b.nextInterval() == IntervalIterator.NO_MORE_INTERVALS) {
+                bpos = false;
+                return IntervalIterator.NO_MORE_INTERVALS;
+              }
+            }
+            if (b.start() <= a.end())
+              return a.start();
+          }
+          bpos = false;
+          return IntervalIterator.NO_MORE_INTERVALS;
         }
       };
     }
   };
 
+  private static abstract class FilteringIntervalIterator extends ConjunctionIntervalIterator {
+
+    final IntervalIterator a;
+    final IntervalIterator b;
+
+    boolean bpos;
+
+    protected FilteringIntervalIterator(IntervalIterator a, IntervalIterator b) {
+      super(Arrays.asList(a, b));
+      this.a = a;
+      this.b = b;
+    }
+
+    @Override
+    public int start() {
+      if (bpos == false) {
+        return NO_MORE_INTERVALS;
+      }
+      return a.start();
+    }
+
+    @Override
+    public int end() {
+      if (bpos == false) {
+        return NO_MORE_INTERVALS;
+      }
+      return a.end();
+    }
+
+    @Override
+    public int gaps() {
+      return a.gaps();
+    }
+
+    @Override
+    protected void reset() throws IOException {
+      bpos = b.nextInterval() != NO_MORE_INTERVALS;
+    }
+  }
+
   private static abstract class SingletonFunction extends IntervalFunction {
 
     private final String name;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3593af2a/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 a98adbd..aa5b62c 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
@@ -172,13 +172,22 @@ public final class Intervals {
   }
 
   /**
+   * Returns intervals from a source that overlap with intervals from another source
+   * @param source      the source to filter
+   * @param reference   the source to filter by
+   */
+  public static IntervalsSource overlapping(IntervalsSource source, IntervalsSource reference) {
+    return new ConjunctionIntervalsSource(Arrays.asList(source, reference), IntervalFunction.OVERLAPPING);
+  }
+
+  /**
    * Create a not-within {@link IntervalsSource}
    *
    * Returns intervals of the minuend that do not appear within a set number of positions of
    * intervals from the subtrahend query
    *
    * @param minuend     the {@link IntervalsSource} to filter
-   * @param positions   the maximum distance that intervals from the minuend may occur from intervals
+   * @param positions   the minimum distance that intervals from the minuend may occur from intervals
    *                    of the subtrahend
    * @param subtrahend  the {@link IntervalsSource} to filter by
    */
@@ -188,6 +197,18 @@ public final class Intervals {
   }
 
   /**
+   * Returns intervals of the source that appear within a set number of positions of intervals from
+   * the reference
+   *
+   * @param source    the {@link IntervalsSource} to filter
+   * @param positions the maximum distance that intervals of the source may occur from intervals of the reference
+   * @param reference the {@link IntervalsSource} to filter by
+   */
+  public static IntervalsSource within(IntervalsSource source, int positions, IntervalsSource reference) {
+    return containedBy(source, Intervals.extend(reference, positions, positions));
+  }
+
+  /**
    * Create a not-containing {@link IntervalsSource}
    *
    * Returns intervals from the minuend that do not contain intervals of the subtrahend
@@ -237,6 +258,22 @@ public final class Intervals {
     return new ConjunctionIntervalsSource(Arrays.asList(small, big), IntervalFunction.CONTAINED_BY);
   }
 
-  // TODO: beforeQuery, afterQuery, arbitrary IntervalFunctions
+  /**
+   * Returns intervals from the source that appear before intervals from the reference
+   */
+  public static IntervalsSource before(IntervalsSource source, IntervalsSource reference) {
+    return new ConjunctionIntervalsSource(Arrays.asList(source,
+        Intervals.extend(new OffsetIntervalsSource(reference, true), Integer.MAX_VALUE, 0)),
+        IntervalFunction.CONTAINED_BY);
+  }
+
+  /**
+   * Returns intervals from the source that appear after intervals from the reference
+   */
+  public static IntervalsSource after(IntervalsSource source, IntervalsSource reference) {
+    return new ConjunctionIntervalsSource(Arrays.asList(source,
+        Intervals.extend(new OffsetIntervalsSource(reference, false), 0, Integer.MAX_VALUE)),
+        IntervalFunction.CONTAINED_BY);
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3593af2a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/OffsetIntervalsSource.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/OffsetIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/OffsetIntervalsSource.java
new file mode 100644
index 0000000..470e2b5
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/OffsetIntervalsSource.java
@@ -0,0 +1,172 @@
+/*
+ * 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;
+
+/**
+ * Tracks a reference intervals source, and produces a pseudo-interval that appears
+ * either one position before or one position after each interval from the reference
+ */
+class OffsetIntervalsSource extends IntervalsSource {
+
+  private final IntervalsSource in;
+  private final boolean before;
+
+  OffsetIntervalsSource(IntervalsSource in, boolean before) {
+    this.in = in;
+    this.before = before;
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    IntervalIterator it = in.intervals(field, ctx);
+    if (it == null) {
+      return null;
+    }
+    return offset(it);
+  }
+
+  private IntervalIterator offset(IntervalIterator it) {
+    if (before) {
+      return new OffsetIntervalIterator(it) {
+        @Override
+        public int start() {
+          int pos = in.start();
+          if (pos == -1) {
+            return -1;
+          }
+          if (pos == NO_MORE_INTERVALS) {
+            return NO_MORE_INTERVALS;
+          }
+          return Math.max(0, pos - 1);
+        }
+      };
+    }
+    else {
+      return new OffsetIntervalIterator(it) {
+        @Override
+        public int start() {
+          int pos = in.end() + 1;
+          if (pos == 0) {
+            return -1;
+          }
+          if (pos < 0) { // overflow
+            return Integer.MAX_VALUE;
+          }
+          if (pos == Integer.MAX_VALUE) {
+            return Integer.MAX_VALUE - 1;
+          }
+          return pos;
+        }
+      };
+    }
+  }
+
+  private static abstract class OffsetIntervalIterator extends IntervalIterator {
+
+    final IntervalIterator in;
+
+    OffsetIntervalIterator(IntervalIterator in) {
+      this.in = in;
+    }
+
+    @Override
+    public int end() {
+      return start();
+    }
+
+    @Override
+    public int gaps() {
+      return 0;
+    }
+
+    @Override
+    public int nextInterval() throws IOException {
+      in.nextInterval();
+      return start();
+    }
+
+    @Override
+    public float matchCost() {
+      return in.matchCost();
+    }
+
+    @Override
+    public int docID() {
+      return in.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return in.nextDoc();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      return in.advance(target);
+    }
+
+    @Override
+    public long cost() {
+      return in.cost();
+    }
+  }
+
+  @Override
+  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    MatchesIterator mi = in.matches(field, ctx, doc);
+    if (mi == null) {
+      return null;
+    }
+    return IntervalMatches.asMatches(offset(IntervalMatches.wrapMatches(mi, doc)), 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;
+    OffsetIntervalsSource that = (OffsetIntervalsSource) o;
+    return before == that.before &&
+        Objects.equals(in, that.in);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(in, before);
+  }
+
+  @Override
+  public String toString() {
+    if (before) {
+      return ("PRECEDING(" + in + ")");
+    }
+    return ("FOLLOWING(" + in + ")");
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3593af2a/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 139cea9..3276caa 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
@@ -330,6 +330,33 @@ public class TestIntervals extends LuceneTestCase {
 
   }
 
+  public void testOffsetIntervals() throws IOException {
+    IntervalsSource source = Intervals.unordered(
+        Intervals.term("pease"),
+        Intervals.term("porridge"),
+        Intervals.or(Intervals.term("hot"), Intervals.term("cold")));
+
+    IntervalsSource before = new OffsetIntervalsSource(source, true);
+    checkIntervals(before, "field1", 3, new int[][]{
+        {},
+        { 0, 0, 0, 0, 1, 1, 2, 2, 3, 3, 4, 4, 5, 5 },
+        { 0, 0, 0, 0, 1, 1, 2, 2, 3, 3, 4, 4, 5, 5 },
+        {},
+        { 0, 0, 0, 0, 1, 1, 2, 2, 3, 3, 4, 4, 5, 5 },
+        {}
+    });
+
+    IntervalsSource after = new OffsetIntervalsSource(source, false);
+    checkIntervals(after, "field1", 3, new int[][]{
+        {},
+        { 3, 3, 4, 4, 5, 5, 6, 6, 7, 7, 8, 8, 18, 18 },
+        { 3, 3, 4, 4, 5, 5, 6, 6, 7, 7, 8, 8, 18, 18 },
+        {},
+        { 3, 3, 4, 4, 5, 5, 6, 6, 7, 7, 8, 8, 18, 18 },
+        {}
+    });
+  }
+
   public void testNesting2() throws IOException {
     IntervalsSource source = Intervals.unordered(
         Intervals.ordered(
@@ -536,4 +563,80 @@ public class TestIntervals extends LuceneTestCase {
     });
   }
 
+  public void testAfter() throws IOException {
+    IntervalsSource source = Intervals.after(Intervals.term("porridge"),
+        Intervals.ordered(Intervals.term("pease"), Intervals.term("cold")));
+    checkIntervals(source, "field1", 3, new int[][]{
+        {},
+        { 7, 7 },
+        { 4, 4, 7, 7 },
+        {},
+        { 7, 7 },
+        {}
+    });
+
+    MatchesIterator mi = getMatches(source, 1, "field1");
+    assertMatch(mi, 7, 7, 20, 55);
+    MatchesIterator sub = mi.getSubMatches();
+    assertNotNull(sub);
+    assertMatch(sub, 3, 3, 20, 25);
+    assertMatch(sub, 5, 5, 35, 39);
+    assertMatch(sub, 7, 7, 47, 55);
+    assertFalse(sub.next());
+  }
+
+  public void testBefore() throws IOException {
+    IntervalsSource source = Intervals.before(Intervals.term("cold"), Intervals.term("twelve"));
+    checkIntervals(source, "field1", 2, new int[][]{
+        {},
+        {},
+        { 2, 2 },
+        {},
+        { 5, 5 },
+        {}
+    });
+  }
+
+  public void testWithin() throws IOException {
+    IntervalsSource source = Intervals.within(Intervals.term("hot"), 6,
+        Intervals.or(Intervals.term("porridge"), Intervals.term("fraggle")));
+    checkIntervals(source, "field1", 3, new int[][]{
+        {},
+        { 2, 2 },
+        { 5, 5, 21, 21 },
+        {},
+        { 2, 2 },
+        {}
+    });
+  }
+
+  public void testOverlapping() throws IOException {
+    IntervalsSource source = Intervals.overlapping(
+        Intervals.unordered(Intervals.term("hot"), Intervals.term("porridge")),
+        Intervals.unordered(Intervals.term("cold"), Intervals.term("pease"))
+    );
+    checkIntervals(source, "field1", 3, new int[][]{
+        {},
+        { 2, 4, 7, 17 },
+        { 5, 7, 7, 21 },
+        {},
+        { 2, 4 },
+        {}
+    });
+
+    assertGaps(source, 2, "field1", new int[]{ 1, 13 });
+
+    MatchesIterator mi = getMatches(source, 1, "field1");
+    assertNotNull(mi);
+    assertMatch(mi, 2, 4, 15, 39);
+    MatchesIterator sub = mi.getSubMatches();
+    assertNotNull(sub);
+    assertMatch(sub, 2, 2, 15, 18);
+    assertMatch(sub, 3, 3, 20, 25);
+    assertMatch(sub, 4, 4, 26, 34);
+    assertMatch(sub, 5, 5, 35, 39);
+    assertFalse(sub.next());
+    assertMatch(mi, 7, 17, 41, 118);
+  }
+
 }