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

[lucene-solr] branch master updated (e68697a -> 7d7ab14)

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

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


    from e68697a  SOLR-13156: documenting functionality gap.
     new 87d68c8  LUCENE-8645: Intervals.fixField()
     new 7d7ab14  LUCENE-8646: Multi-term intervals

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 lucene/CHANGES.txt                                 |   6 ++
 .../intervals/DisjunctionIntervalsSource.java      |   2 +-
 ...sSource.java => FixedFieldIntervalsSource.java} |  42 +++-----
 .../apache/lucene/search/intervals/Intervals.java  |  37 +++++++
 .../search/intervals/MultiTermIntervalsSource.java | 120 +++++++++++++++++++++
 .../search/intervals/TermIntervalsSource.java      |   8 ++
 .../lucene/search/intervals/TestIntervals.java     |  58 ++++++++++
 7 files changed, 243 insertions(+), 30 deletions(-)
 copy lucene/sandbox/src/java/org/apache/lucene/search/intervals/{ExtendedIntervalsSource.java => FixedFieldIntervalsSource.java} (60%)
 create mode 100644 lucene/sandbox/src/java/org/apache/lucene/search/intervals/MultiTermIntervalsSource.java


[lucene-solr] 01/02: LUCENE-8645: Intervals.fixField()

Posted by ro...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 87d68c8253fcb928be4eb2b2d908393252a50ec5
Author: Alan Woodward <ro...@apache.org>
AuthorDate: Wed Jan 16 20:35:39 2019 +0000

    LUCENE-8645: Intervals.fixField()
---
 lucene/CHANGES.txt                                 |  6 ++
 .../intervals/FixedFieldIntervalsSource.java       | 76 ++++++++++++++++++++++
 .../apache/lucene/search/intervals/Intervals.java  | 11 ++++
 .../lucene/search/intervals/TestIntervals.java     | 22 +++++++
 4 files changed, 115 insertions(+)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 7e38150..4e3e18f 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -187,6 +187,12 @@ New Features
 * LUCENE-8622: Adds a minimum-should-match interval function that produces intervals
   spanning a subset of a set of sources. (Alan Woodward)
 
+* LUCENE-8645: Intervals.fixField() allows you to report intervals from one field
+  as if they came from another. (Alan Woodward)
+
+* LUCENE-8646: New interval functions: Intervals.prefix() and Intervals.wildcard()
+  (Alan Woodward)
+
 Improvements
 
 * LUCENE-7997: Add BaseSimilarityTestCase to sanity check similarities.
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/intervals/FixedFieldIntervalsSource.java b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/FixedFieldIntervalsSource.java
new file mode 100644
index 0000000..7776a2b
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/intervals/FixedFieldIntervalsSource.java
@@ -0,0 +1,76 @@
+/*
+ * 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 FixedFieldIntervalsSource extends IntervalsSource {
+
+  private final String field;
+  private final IntervalsSource source;
+
+  FixedFieldIntervalsSource(String field, IntervalsSource source) {
+    this.field = field;
+    this.source = source;
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    return source.intervals(this.field, ctx);
+  }
+
+  @Override
+  public MatchesIterator matches(String field, LeafReaderContext ctx, int doc) throws IOException {
+    return source.matches(this.field, ctx, doc);
+  }
+
+  @Override
+  public void extractTerms(String field, Set<Term> terms) {
+    source.extractTerms(this.field, terms);
+  }
+
+  @Override
+  public int minExtent() {
+    return source.minExtent();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    FixedFieldIntervalsSource that = (FixedFieldIntervalsSource) o;
+    return Objects.equals(field, that.field) &&
+        Objects.equals(source, that.source);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(field, source);
+  }
+
+  @Override
+  public String toString() {
+    return "FIELD(" + field + "," + source + ")";
+  }
+}
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 1c8d71a..ae2029a 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
@@ -160,6 +160,17 @@ public final class Intervals {
   }
 
   /**
+   * Create an {@link IntervalsSource} that always returns intervals from a specific field
+   *
+   * This is useful for comparing intervals across multiple fields, for example fields that
+   * have been analyzed differently, allowing you to search for stemmed terms near unstemmed
+   * terms, etc.
+   */
+  public static IntervalsSource fixField(String field, IntervalsSource source) {
+    return new FixedFieldIntervalsSource(field, source);
+  }
+
+  /**
    * Create a non-overlapping IntervalsSource
    *
    * Returns intervals of the minuend that do not overlap with intervals from the subtrahend
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 5f58ebf..61b06bb 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
@@ -704,4 +704,26 @@ public class TestIntervals extends LuceneTestCase {
     assertEquals(2, source.minExtent());
   }
 
+  public void testFixedField() throws IOException {
+
+    IntervalsSource source = Intervals.phrase(
+        Intervals.term("alph"),
+        Intervals.fixField("field1", Intervals.term("hot")));
+
+    // We search in field2, but 'hot' will report intervals from field1
+    checkIntervals(source, "field2", 1, new int[][]{
+        {},
+        { 1, 2 },
+        {},
+        {},
+        {},
+        {}
+    });
+
+    MatchesIterator mi = getMatches(source, 1, "field2");
+    assertNotNull(mi);
+    assertMatch(mi, 1, 2, 6, 18);
+
+  }
+
 }


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

Posted by ro...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

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