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

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

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);
+  }
+
 }