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/22 13:57:00 UTC

[lucene-solr] 02/02: LUCENE-8646: Multi-term intervals

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

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

commit ceadb5f1339fe5bf09f9818470872d4957b63ba5
Author: Alan Woodward <ro...@apache.org>
AuthorDate: Thu Jan 17 13:41:19 2019 +0000

    LUCENE-8646: Multi-term intervals
---
 .../intervals/DisjunctionIntervalsSource.java      |   2 +-
 .../apache/lucene/search/intervals/Intervals.java  |  26 +++++
 .../search/intervals/MultiTermIntervalsSource.java | 120 +++++++++++++++++++++
 .../search/intervals/TermIntervalsSource.java      |   8 ++
 .../lucene/search/intervals/TestIntervals.java     |  36 +++++++
 5 files changed, 191 insertions(+), 1 deletion(-)

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 79089c7..b280885 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
@@ -99,7 +99,7 @@ class DisjunctionIntervalsSource extends IntervalsSource {
     return minExtent;
   }
 
-  private static class DisjunctionIntervalIterator extends IntervalIterator {
+  static class DisjunctionIntervalIterator extends IntervalIterator {
 
     final DocIdSetIterator approximation;
     final PriorityQueue<IntervalIterator> intervalQueue;
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 ae2029a..e8c4cb0 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
@@ -19,7 +19,11 @@ package org.apache.lucene.search.intervals;
 
 import java.util.Arrays;
 
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.PrefixQuery;
+import org.apache.lucene.search.WildcardQuery;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 /**
  * Constructor functions for {@link IntervalsSource} types
@@ -76,6 +80,28 @@ public final class Intervals {
   }
 
   /**
+   * Return an {@link IntervalsSource} over the disjunction of all terms that begin with a prefix
+   *
+   * @throws IllegalStateException if the prefix expands to more than 128 terms
+   */
+  public static IntervalsSource prefix(String prefix) {
+    CompiledAutomaton ca = new CompiledAutomaton(PrefixQuery.toAutomaton(new BytesRef(prefix)));
+    return new MultiTermIntervalsSource(ca, 128, prefix);
+  }
+
+  /**
+   * Return an {@link IntervalsSource} over the disjunction of all terms that match a wildcard glob
+   *
+   * @throws IllegalStateException if the wildcard glob expands to more than 128 terms
+   *
+   * @see WildcardQuery for glob format
+   */
+  public static IntervalsSource wildcard(String wildcard) {
+    CompiledAutomaton ca = new CompiledAutomaton(WildcardQuery.toAutomaton(new Term("", wildcard)));
+    return new MultiTermIntervalsSource(ca, 128, wildcard);
+  }
+
+  /**
    * 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 to filter
    * @param subSource   the sub-source to filter
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/MultiTermIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/MultiTermIntervalsSource.java
new file mode 100644
index 0000000..7689d1d
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/MultiTermIntervalsSource.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.search.intervals;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.MatchesIterator;
+import org.apache.lucene.search.MatchesUtils;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+
+class MultiTermIntervalsSource extends IntervalsSource {
+
+  private final CompiledAutomaton automaton;
+  private final int maxExpansions;
+  private final String pattern;
+
+  MultiTermIntervalsSource(CompiledAutomaton automaton, int maxExpansions, String pattern) {
+    this.automaton = automaton;
+    this.maxExpansions = maxExpansions;
+    this.pattern = pattern;
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    Terms terms = ctx.reader().terms(field);
+    if (terms == null) {
+      return null;
+    }
+    List<IntervalIterator> subSources = new ArrayList<>();
+    TermsEnum te = automaton.getTermsEnum(terms);
+    BytesRef term;
+    int count = 0;
+    while ((term = te.next()) != null) {
+      subSources.add(TermIntervalsSource.intervals(term, te));
+      if (count++ > maxExpansions) {
+        throw new IllegalStateException("Automaton " + this.pattern + " expanded to too many terms (limit " + maxExpansions + ")");
+      }
+    }
+    if (subSources.size() == 0) {
+      return null;
+    }
+    return new DisjunctionIntervalsSource.DisjunctionIntervalIterator(subSources);
+  }
+
+  @Override
+  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    Terms terms = ctx.reader().terms(field);
+    if (terms == null) {
+      return null;
+    }
+    List<MatchesIterator> subMatches = new ArrayList<>();
+    TermsEnum te = automaton.getTermsEnum(terms);
+    BytesRef term;
+    int count = 0;
+    while ((term = te.next()) != null) {
+      MatchesIterator mi = TermIntervalsSource.matches(te, doc);
+      if (mi != null) {
+        subMatches.add(mi);
+        if (count++ > maxExpansions) {
+          throw new IllegalStateException("Automaton " + term + " expanded to too many terms (limit " + maxExpansions + ")");
+        }
+      }
+    }
+    return MatchesUtils.disjunction(subMatches);
+  }
+
+  @Override
+  public void extractTerms(String field, Set<Term> terms) {
+
+  }
+
+  @Override
+  public int minExtent() {
+    return 1;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    MultiTermIntervalsSource that = (MultiTermIntervalsSource) o;
+    return maxExpansions == that.maxExpansions &&
+        Objects.equals(automaton, that.automaton) &&
+        Objects.equals(pattern, that.pattern);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(automaton, maxExpansions, pattern);
+  }
+
+  @Override
+  public String toString() {
+    return "MultiTerm(" + pattern + ")";
+  }
+}
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 1b5444a..4539d2f 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
@@ -55,6 +55,10 @@ class TermIntervalsSource extends IntervalsSource {
     if (te.seekExact(term) == false) {
       return null;
     }
+    return intervals(term, te);
+  }
+
+  static IntervalIterator intervals(BytesRef term, TermsEnum te) throws IOException {
     PostingsEnum pe = te.postings(null, PostingsEnum.POSITIONS);
     float cost = termPositionsCost(te);
     return new IntervalIterator() {
@@ -143,6 +147,10 @@ class TermIntervalsSource extends IntervalsSource {
     if (te.seekExact(term) == false) {
       return null;
     }
+    return matches(te, doc);
+  }
+
+  static MatchesIterator matches(TermsEnum te, int doc) throws IOException {
     PostingsEnum pe = te.postings(null, PostingsEnum.OFFSETS);
     if (pe.advance(doc) != doc) {
       return null;
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 61b06bb..c8452a8 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
@@ -726,4 +726,40 @@ public class TestIntervals extends LuceneTestCase {
 
   }
 
+  public void testPrefix() throws IOException {
+    IntervalsSource source = Intervals.prefix("p");
+    checkIntervals(source, "field1", 5, new int[][]{
+        {},
+        { 0, 0, 1, 1, 3, 3, 4, 4, 6, 6, 7, 7, 10, 10, 27, 27 },
+        { 0, 0, 1, 1, 3, 3, 4, 4, 6, 6, 7, 7, 10, 10 },
+        { 7, 7 },
+        { 0, 0, 1, 1, 3, 3, 4, 4, 6, 6, 7, 7, 10, 10 },
+        { 0, 0 }
+    });
+    MatchesIterator mi = getMatches(source, 1, "field1");
+    assertNotNull(mi);
+    assertMatch(mi, 0, 0, 0, 5);
+    assertMatch(mi, 1, 1, 6, 14);
+
+    IntervalsSource noSuch = Intervals.prefix("qqq");
+    checkIntervals(noSuch, "field1", 0, new int[][]{});
+  }
+
+  public void testWildcard() throws IOException {
+    IntervalsSource source = Intervals.wildcard("?ot");
+    checkIntervals(source, "field1", 4, new int[][]{
+        {},
+        { 2, 2, 10, 10, 17, 17, 27, 27 },
+        { 5, 5, 10, 10, 21, 21 },
+        { 3, 3 },
+        { 2, 2, 10, 10, 17, 17 },
+        {}
+    });
+    MatchesIterator mi = getMatches(source, 4, "field1");
+    assertNotNull(mi);
+    assertMatch(mi, 2, 2, 15, 18);
+    assertMatch(mi, 10, 10, 63, 66);
+    assertMatch(mi, 17, 17, 97, 100);
+  }
+
 }