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 2021/03/30 11:25:11 UTC

[lucene-solr] branch branch_8_8 updated (be8fb34 -> bcce437)

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

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


    from be8fb34  LUCENE-9870: Fix Circle2D intersectsLine t-value (distance) range clamp (#41)
     new 2e1bc25  LUCENE-9744: NPE on a degenerate query in MinimumShouldMatchIntervalsSource$MinimumMatchesIterator.getSubMatches() (#2323) (#2324)
     new bcce437  LUCENE-9762: DoubleValuesSource.fromQuery bug (#2365)

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                                 |  9 ++++-
 .../apache/lucene/search/DoubleValuesSource.java   | 14 ++++++-
 .../apache/lucene/queries/intervals/Intervals.java | 10 +++++
 .../MinimumShouldMatchIntervalsSource.java         |  1 +
 ...valsSource.java => NoMatchIntervalsSource.java} | 43 ++++++++--------------
 .../queries/function/TestFunctionScoreQuery.java   | 38 ++++++++++++++++---
 .../lucene/queries/intervals/TestIntervals.java    | 21 +++++++++++
 .../queries/intervals/TestSimplifications.java     |  8 ++++
 8 files changed, 108 insertions(+), 36 deletions(-)
 copy lucene/queries/src/java/org/apache/lucene/queries/intervals/{FixedFieldIntervalsSource.java => NoMatchIntervalsSource.java} (60%)

[lucene-solr] 02/02: LUCENE-9762: DoubleValuesSource.fromQuery bug (#2365)

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

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

commit bcce43770618bc8ba7dbf33c81f56f80bc073c90
Author: David Smiley <ds...@apache.org>
AuthorDate: Tue Feb 16 22:51:17 2021 -0500

    LUCENE-9762: DoubleValuesSource.fromQuery bug (#2365)
    
    Also used by FunctionScoreQuery.boostByQuery.
    Could throw an exception when the query implements TwoPhaseIterator
    and when the score is requested repeatedly.
    
    Co-authored-by: Chris Hostetter <ho...@apache.org>
    
    (cherry picked from commit 25554180480fdd5722e43ab9356be0f8bab936d0)
---
 lucene/CHANGES.txt                                 |  4 +++
 .../apache/lucene/search/DoubleValuesSource.java   | 14 ++++++--
 .../queries/function/TestFunctionScoreQuery.java   | 38 +++++++++++++++++++---
 3 files changed, 49 insertions(+), 7 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 19853e3..2055eb5 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -13,6 +13,10 @@ Bug Fixes
 * LUCENE-9744: NPE on a degenerate query in MinimumShouldMatchIntervalsSource
   $MinimumMatchesIterator.getSubMatches(). (Alan Woodward)
 
+* LUCENE-9762: DoubleValuesSource.fromQuery (also used by FunctionScoreQuery.boostByQuery) could
+  throw an exception when the query implements TwoPhaseIterator and when the score is requested
+  repeatedly. (David Smiley, hossman)
+
 ======================= Lucene 8.8.1 =======================
 
 (No changes)
diff --git a/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
index 87e2ecfd..ca82a2c 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java
@@ -493,7 +493,7 @@ public abstract class DoubleValuesSource implements SegmentCacheable {
 
           return new DoubleComparator.DoubleLeafComparator(context) {
             LeafReaderContext ctx;
-            
+
             @Override
             protected NumericDocValues getNumericDocValues(LeafReaderContext context, String field) {
               ctx = context;
@@ -617,6 +617,7 @@ public abstract class DoubleValuesSource implements SegmentCacheable {
       return new DoubleValues() {
         private final TwoPhaseIterator tpi = scorer.twoPhaseIterator();
         private final DocIdSetIterator disi = (tpi == null) ? scorer.iterator() : tpi.approximation();
+        private Boolean tpiMatch = null; // cache tpi.matches()
 
         @Override
         public double doubleValue() throws IOException {
@@ -627,8 +628,17 @@ public abstract class DoubleValuesSource implements SegmentCacheable {
         public boolean advanceExact(int doc) throws IOException {
           if (disi.docID() < doc) {
             disi.advance(doc);
+            tpiMatch = null;
+          }
+          if (disi.docID() == doc) {
+            if (tpi == null) {
+              return true;
+            } else if (tpiMatch == null) {
+              tpiMatch = tpi.matches();
+            }
+            return tpiMatch;
           }
-          return disi.docID() == doc && (tpi == null || tpi.matches());
+          return false;
         }
       };
     }
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/function/TestFunctionScoreQuery.java b/lucene/queries/src/test/org/apache/lucene/queries/function/TestFunctionScoreQuery.java
index 6c8ea52..e7a3dda 100644
--- a/lucene/queries/src/test/org/apache/lucene/queries/function/TestFunctionScoreQuery.java
+++ b/lucene/queries/src/test/org/apache/lucene/queries/function/TestFunctionScoreQuery.java
@@ -21,13 +21,16 @@ import java.io.IOException;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.TextField;
 import org.apache.lucene.expressions.Expression;
 import org.apache.lucene.expressions.SimpleBindings;
 import org.apache.lucene.expressions.js.JavascriptCompiler;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -36,6 +39,7 @@ import org.apache.lucene.search.DoubleValuesSource;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.QueryUtils;
 import org.apache.lucene.search.ScoreMode;
@@ -252,28 +256,52 @@ public class TestFunctionScoreQuery extends FunctionTestSetup {
     assertInnerScoreMode(ScoreMode.COMPLETE_NO_SCORES, ScoreMode.COMPLETE, DoubleValuesSource.fromDoubleField("foo"));
     assertInnerScoreMode(ScoreMode.COMPLETE_NO_SCORES, ScoreMode.COMPLETE_NO_SCORES, DoubleValuesSource.fromDoubleField("foo"));
     assertInnerScoreMode(ScoreMode.COMPLETE_NO_SCORES, ScoreMode.TOP_SCORES, DoubleValuesSource.fromDoubleField("foo"));
-    
+
     // Value Source needs scores
     assertInnerScoreMode(ScoreMode.COMPLETE, ScoreMode.COMPLETE, DoubleValuesSource.SCORES);
     assertInnerScoreMode(ScoreMode.COMPLETE_NO_SCORES, ScoreMode.COMPLETE_NO_SCORES, DoubleValuesSource.SCORES);
     assertInnerScoreMode(ScoreMode.COMPLETE, ScoreMode.TOP_SCORES, DoubleValuesSource.SCORES);
-    
+
   }
-  
+
   private void assertInnerScoreMode(ScoreMode expectedScoreMode, ScoreMode inputScoreMode, DoubleValuesSource valueSource) throws IOException {
     final AtomicReference<ScoreMode> scoreModeInWeight = new AtomicReference<ScoreMode>();
     Query innerQ = new TermQuery(new Term(TEXT_FIELD, "a")) {
-      
+
       @Override
       public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
         scoreModeInWeight.set(scoreMode);
         return super.createWeight(searcher, scoreMode, boost);
       }
     };
-    
+
     FunctionScoreQuery fq = new FunctionScoreQuery(innerQ, valueSource);
     fq.createWeight(searcher, inputScoreMode, 1f);
     assertEquals(expectedScoreMode, scoreModeInWeight.get());
   }
 
+  /** The FunctionScoreQuery's Scorer score() is going to be called twice for the same doc. */
+  public void testScoreCalledTwice() throws Exception {
+    try (Directory dir = newDirectory()) {
+      IndexWriterConfig conf = newIndexWriterConfig();
+      IndexWriter indexWriter = new IndexWriter(dir, conf);
+      Document doc = new Document();
+      doc.add(new TextField("ExampleText", "periodic function", Field.Store.NO));
+      doc.add(new TextField("ExampleText", "plot of the original function", Field.Store.NO));
+      indexWriter.addDocument(doc);
+      indexWriter.commit();
+      indexWriter.close();
+
+      try (DirectoryReader reader = DirectoryReader.open(dir)) {
+        Query q = new TermQuery(new Term("ExampleText", "function"));
+
+        q =
+            FunctionScoreQuery.boostByQuery(
+                q, new PhraseQuery(1, "ExampleText", "function", "plot"), 2);
+        q = FunctionScoreQuery.boostByValue(q, DoubleValuesSource.SCORES);
+
+        assertEquals(1, new IndexSearcher(reader).search(q, 10).totalHits.value);
+      }
+    }
+  }
 }

[lucene-solr] 01/02: LUCENE-9744: NPE on a degenerate query in MinimumShouldMatchIntervalsSource$MinimumMatchesIterator.getSubMatches() (#2323) (#2324)

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

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

commit 2e1bc256c8e11690414fa3856fca93eaecb39662
Author: Dawid Weiss <da...@carrotsearch.com>
AuthorDate: Mon Feb 8 22:43:08 2021 +0100

    LUCENE-9744: NPE on a degenerate query in MinimumShouldMatchIntervalsSource$MinimumMatchesIterator.getSubMatches() (#2323) (#2324)
---
 lucene/CHANGES.txt                                 |  5 +-
 .../apache/lucene/queries/intervals/Intervals.java | 10 +++
 .../MinimumShouldMatchIntervalsSource.java         |  1 +
 .../queries/intervals/NoMatchIntervalsSource.java  | 75 ++++++++++++++++++++++
 .../lucene/queries/intervals/TestIntervals.java    | 21 ++++++
 .../queries/intervals/TestSimplifications.java     |  8 +++
 6 files changed, 118 insertions(+), 2 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index c85e58d..19853e3 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -10,10 +10,11 @@ Bug Fixes
 
 * LUCENE-9870: Fix Circle2D intersectsLine t-value (distance) range clamp (Jørgen Nystad)
 
+* LUCENE-9744: NPE on a degenerate query in MinimumShouldMatchIntervalsSource
+  $MinimumMatchesIterator.getSubMatches(). (Alan Woodward)
+
 ======================= Lucene 8.8.1 =======================
 
-Bug Fixes
----------------------
 (No changes)
 
 ======================= Lucene 8.8.0 =======================
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/Intervals.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/Intervals.java
index 5d835bd..fcdcf4f 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/Intervals.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/Intervals.java
@@ -442,6 +442,16 @@ 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) {
+    if (minShouldMatch == sources.length) {
+      return unordered(sources);
+    }
+    if (minShouldMatch > sources.length) {
+      return new NoMatchIntervalsSource(
+          "Too few sources to match minimum of ["
+              + minShouldMatch
+              + "]: "
+              + Arrays.toString(sources));
+    }
     return new MinimumShouldMatchIntervalsSource(sources, minShouldMatch);
   }
 
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/MinimumShouldMatchIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/MinimumShouldMatchIntervalsSource.java
index ce3a6de..4cc2cfb 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/intervals/MinimumShouldMatchIntervalsSource.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/MinimumShouldMatchIntervalsSource.java
@@ -43,6 +43,7 @@ class MinimumShouldMatchIntervalsSource extends IntervalsSource {
   private final int minShouldMatch;
 
   MinimumShouldMatchIntervalsSource(IntervalsSource[] sources, int minShouldMatch) {
+    assert minShouldMatch < sources.length;
     this.sources = sources;
     this.minShouldMatch = minShouldMatch;
   }
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/intervals/NoMatchIntervalsSource.java b/lucene/queries/src/java/org/apache/lucene/queries/intervals/NoMatchIntervalsSource.java
new file mode 100644
index 0000000..cfa7364
--- /dev/null
+++ b/lucene/queries/src/java/org/apache/lucene/queries/intervals/NoMatchIntervalsSource.java
@@ -0,0 +1,75 @@
+/*
+ * 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.queries.intervals;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Objects;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.QueryVisitor;
+
+/** A source returning no matches */
+class NoMatchIntervalsSource extends IntervalsSource {
+  final String reason;
+
+  NoMatchIntervalsSource(String reason) {
+    this.reason = reason;
+  }
+
+  @Override
+  public IntervalIterator intervals(String field, LeafReaderContext ctx) throws IOException {
+    return null;
+  }
+
+  @Override
+  public IntervalMatchesIterator matches(String field, LeafReaderContext ctx, int doc)
+      throws IOException {
+    return null;
+  }
+
+  @Override
+  public void visit(String field, QueryVisitor visitor) {}
+
+  @Override
+  public int minExtent() {
+    return 0;
+  }
+
+  @Override
+  public Collection<IntervalsSource> pullUpDisjunctions() {
+    return Collections.singleton(this);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    NoMatchIntervalsSource that = (NoMatchIntervalsSource) o;
+    return Objects.equals(reason, that.reason);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(reason);
+  }
+
+  @Override
+  public String toString() {
+    return "NOMATCH(" + reason + ")";
+  }
+}
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestIntervals.java b/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestIntervals.java
index 5444b66..2329ea8 100644
--- a/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestIntervals.java
+++ b/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestIntervals.java
@@ -761,6 +761,27 @@ public class TestIntervals extends LuceneTestCase {
 
   }
 
+  public void testDegenerateMinShouldMatch() throws IOException {
+    IntervalsSource source =
+        Intervals.ordered(
+            Intervals.atLeast(1, Intervals.term("interest")),
+            Intervals.atLeast(1, Intervals.term("anyone")));
+
+    MatchesIterator mi = getMatches(source, 0, "field1");
+    assertMatch(mi, 2, 4, 11, 29);
+    MatchesIterator subs = mi.getSubMatches();
+    assertNotNull(subs);
+    assertMatch(subs, 2, 2, 11, 19);
+    assertMatch(subs, 4, 4, 23, 29);
+    assertFalse(subs.next());
+    assertFalse(mi.next());
+  }
+
+  public void testNoMatchMinShouldMatch() throws IOException {
+    IntervalsSource source = Intervals.atLeast(4, Intervals.term("a"), Intervals.term("b"));
+    checkIntervals(source, "field", 0, new int[][] {});
+  }
+
   public void testDefinedGaps() throws IOException {
     IntervalsSource source = Intervals.phrase(
         Intervals.term("pease"),
diff --git a/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestSimplifications.java b/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestSimplifications.java
index 76a4857..8706c12 100644
--- a/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestSimplifications.java
+++ b/lucene/queries/src/test/org/apache/lucene/queries/intervals/TestSimplifications.java
@@ -93,4 +93,12 @@ public class TestSimplifications extends LuceneTestCase {
     assertEquals(Intervals.or(Intervals.term("a"), Intervals.term("b"), Intervals.term("c"), Intervals.term("d")), actual);
   }
 
+  public void testMinShouldMatchSimplifications() {
+    IntervalsSource expected = Intervals.unordered(Intervals.term("a"), Intervals.term("b"));
+    assertEquals(expected, Intervals.atLeast(2, Intervals.term("a"), Intervals.term("b")));
+
+    assertEquals(
+        "NOMATCH(Too few sources to match minimum of [3]: [a, b])",
+        Intervals.atLeast(3, Intervals.term("a"), Intervals.term("b")).toString());
+  }
 }