You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by tf...@apache.org on 2020/05/21 22:48:50 UTC

[lucene-solr] branch master updated: SOLR-13289: Use the final collector's scoreMode (#1517)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 5e9483e  SOLR-13289: Use the final collector's scoreMode (#1517)
5e9483e is described below

commit 5e9483e7885cab47b7d0e6249cfeb1fc02ffc257
Author: Tomas Fernandez Lobbe <tf...@apache.org>
AuthorDate: Thu May 21 15:48:37 2020 -0700

    SOLR-13289: Use the final collector's scoreMode (#1517)
    
    This is needed in case a PostFilter changes the scoreMode
---
 solr/CHANGES.txt                                   |   2 +-
 .../org/apache/solr/search/SolrIndexSearcher.java  |  14 +-
 .../apache/solr/search/SolrIndexSearcherTest.java  | 268 +++++++++++++++------
 .../solr/search/TestCollapseQParserPlugin.java     |  29 +++
 4 files changed, 239 insertions(+), 74 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 4185e9b..1e0fa23 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -136,7 +136,7 @@ Optimizations
 * SOLR-13289: When the "minExactHits" parameters is provided in queries and it's value is lower than the number of hits,
   Solr can speedup the query resolution by using the Block-Max WAND algorithm (see LUCENE-8135). When doing this, the
   value of matching documents in the response (numFound) will be an approximation.
-  (Ishan Chattopadhyaya, Munendra S N, Tomás Fernández Löbbe)
+  (Ishan Chattopadhyaya, Munendra S N, Tomás Fernández Löbbe, David Smiley)
 
 * SOLR-14472: Autoscaling "cores" preference now retrieves the core count more efficiently, and counts all cores.
   (David Smiley)
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index bf85d6c..af968d6 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -160,13 +160,14 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
         UninvertingReader.wrap(reader, core.getLatestSchema().getUninversionMapper()),
         SolrQueryTimeoutImpl.getInstance());
   }
-
+  
   /**
    * Builds the necessary collector chain (via delegate wrapping) and executes the query against it. This method takes
    * into consideration both the explicitly provided collector and postFilter as well as any needed collector wrappers
    * for dealing with options specified in the QueryCommand.
+   * @return The collector used for search
    */
-  private void buildAndRunCollectorChain(QueryResult qr, Query query, Collector collector, QueryCommand cmd,
+  private Collector buildAndRunCollectorChain(QueryResult qr, Query query, Collector collector, QueryCommand cmd,
       DelegatingCollector postFilter) throws IOException {
 
     EarlyTerminatingSortingCollector earlyTerminatingSortingCollector = null;
@@ -216,6 +217,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     if (collector instanceof DelegatingCollector) {
       ((DelegatingCollector) collector).finish();
     }
+    return collector;
   }
 
   public SolrIndexSearcher(SolrCore core, String path, IndexSchema schema, SolrIndexConfig config, String name,
@@ -1580,11 +1582,15 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
         maxScoreCollector = new MaxScoreCollector();
         collector = MultiCollector.wrap(topCollector, maxScoreCollector);
       }
-      buildAndRunCollectorChain(qr, query, collector, cmd, pf.postFilter);
+      ScoreMode scoreModeUsed = buildAndRunCollectorChain(qr, query, collector, cmd, pf.postFilter).scoreMode();
 
       totalHits = topCollector.getTotalHits();
       TopDocs topDocs = topCollector.topDocs(0, len);
-      hitsRelation = topDocs.totalHits.relation;
+      if (scoreModeUsed == ScoreMode.COMPLETE || scoreModeUsed == ScoreMode.COMPLETE_NO_SCORES) {
+        hitsRelation = TotalHits.Relation.EQUAL_TO;
+      } else {
+        hitsRelation = topDocs.totalHits.relation;
+      }
       if (cmd.getSort() != null && query instanceof RankQuery == false && (cmd.getFlags() & GET_SCORES) != 0) {
         TopFieldCollector.populateScores(topDocs.scoreDocs, this, query);
       }
diff --git a/solr/core/src/test/org/apache/solr/search/SolrIndexSearcherTest.java b/solr/core/src/test/org/apache/solr/search/SolrIndexSearcherTest.java
index 309af8a..e670133 100644
--- a/solr/core/src/test/org/apache/solr/search/SolrIndexSearcherTest.java
+++ b/solr/core/src/test/org/apache/solr/search/SolrIndexSearcherTest.java
@@ -16,31 +16,51 @@
  */
 package org.apache.solr.search;
 
+import java.io.IOException;
+
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.search.Weight;
 import org.apache.solr.SolrTestCaseJ4;
 import org.junit.Before;
 import org.junit.BeforeClass;
 
-import java.io.IOException;
-
 public class SolrIndexSearcherTest extends SolrTestCaseJ4 {
-  
+
   private final static int NUM_DOCS = 20;
 
   @BeforeClass
   public static void setUpClass() throws Exception {
     initCore("solrconfig.xml", "schema.xml");
     for (int i = 0 ; i < NUM_DOCS ; i ++) {
-      assertU(adoc("id", String.valueOf(i), "field1_s", "foo", "field2_s", String.valueOf(i % 2), "field3_s", String.valueOf(i)));
-      assertU(commit());
+      assertU(adoc("id", String.valueOf(i),
+          "field1_s", "foo",
+          "field2_s", String.valueOf(i % 2),
+          "field3_i_dvo", String.valueOf(i),
+          "field4_t", numbersTo(i)));
+      assertU(commit()); //commit inside the loop to get multiple segments
     }
   }
   
+  private static String numbersTo(int i) {
+    StringBuilder numbers = new StringBuilder();
+    for (int j = 0; j <= i ; j++) {
+      numbers.append(String.valueOf(j) + " ");
+    }
+    return numbers.toString();
+  }
+
   @Before
   public void setUp() throws Exception {
-    assertU(adoc("id", "1", "field1_s", "foo", "field2_s", "1", "field3_s", "1"));
+    assertU(adoc("id", "1",
+        "field1_s", "foo",
+        "field2_s", "1",
+        "field3_i_dvo", "1",
+        "field4_t", numbersTo(1)));
     assertU(commit());
     super.setUp();
   }
@@ -72,129 +92,239 @@ public class SolrIndexSearcherTest extends SolrTestCaseJ4 {
         );
   }
   
-  private void assertMatchesEqual(int expectedCount, QueryResult qr) {
+  private void assertMatchesEqual(int expectedCount, SolrIndexSearcher searcher, QueryCommand cmd) throws IOException {
+    QueryResult qr = new QueryResult();
+    searcher.search(qr, cmd);
     assertEquals(expectedCount, qr.getDocList().matches());
     assertEquals(TotalHits.Relation.EQUAL_TO, qr.getDocList().hitCountRelation());
   }
   
-  private void assertMatchesGraterThan(int expectedCount, QueryResult qr) {
+  private QueryResult assertMatchesGreaterThan(int expectedCount, SolrIndexSearcher searcher, QueryCommand cmd) throws IOException {
+    QueryResult qr = new QueryResult();
+    searcher.search(qr, cmd);
     assertTrue("Expecting returned matches to be greater than " + expectedCount + " but got " + qr.getDocList().matches(),
         expectedCount >= qr.getDocList().matches());
     assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, qr.getDocList().hitCountRelation());
+    return qr;
   }
   
   public void testLowMinExactHitsGeneratesApproximation() throws IOException {
     h.getCore().withSearcher(searcher -> {
-      QueryCommand cmd = new QueryCommand();
-      cmd.setMinExactHits(NUM_DOCS / 2);
-      cmd.setQuery(new TermQuery(new Term("field1_s", "foo")));
-      QueryResult qr = new QueryResult();
-      searcher.search(qr, cmd);
-      assertMatchesGraterThan(NUM_DOCS, qr);
+      QueryCommand cmd = createBasicQueryCommand(NUM_DOCS / 2, 10, "field1_s", "foo");
+      assertMatchesGreaterThan(NUM_DOCS, searcher, cmd);
       return null;
     });
     
     h.getCore().withSearcher(searcher -> {
-      QueryCommand cmd = new QueryCommand();
-      cmd.setMinExactHits(1);
-      cmd.setLen(1);
-      // We need to disable cache, otherwise the search will be done for 20 docs (cache window size) which brings up the minExactHits
-      cmd.setFlags(SolrIndexSearcher.NO_CHECK_QCACHE | SolrIndexSearcher.NO_SET_QCACHE);
-      cmd.setQuery(new TermQuery(new Term("field2_s", "1")));
-      QueryResult qr = new QueryResult();
-      searcher.search(qr, cmd);
-      assertMatchesGraterThan(NUM_DOCS/2, qr);
+      QueryCommand cmd = createBasicQueryCommand(1, 1, "field2_s", "1");
+      assertMatchesGreaterThan(NUM_DOCS/2, searcher, cmd);
       return null;
     });
   }
-  
+
   public void testHighMinExactHitsGeneratesExactCount() throws IOException {
     h.getCore().withSearcher(searcher -> {
-      QueryCommand cmd = new QueryCommand();
-      cmd.setMinExactHits(NUM_DOCS);
-      cmd.setQuery(new TermQuery(new Term("field1_s", "foo")));
-      QueryResult qr = new QueryResult();
-      searcher.search(qr, cmd);
-      assertMatchesEqual(NUM_DOCS, qr);
+      QueryCommand cmd = createBasicQueryCommand(NUM_DOCS, 10, "field1_s", "foo");
+      assertMatchesEqual(NUM_DOCS, searcher, cmd);
       return null;
     });
     
     h.getCore().withSearcher(searcher -> {
-      QueryCommand cmd = new QueryCommand();
-      cmd.setMinExactHits(NUM_DOCS);
-      cmd.setQuery(new TermQuery(new Term("field2_s", "1")));
-      QueryResult qr = new QueryResult();
-      searcher.search(qr, cmd);
-      assertMatchesEqual(NUM_DOCS/2, qr);
+      QueryCommand cmd = createBasicQueryCommand(NUM_DOCS, 10, "field2_s", "1");
+      assertMatchesEqual(NUM_DOCS/2, searcher, cmd);
       return null;
     });
   }
+
+  
   
   public void testLowMinExactHitsWithQueryResultCache() throws IOException {
     h.getCore().withSearcher(searcher -> {
-      QueryCommand cmd = new QueryCommand();
-      cmd.setMinExactHits(NUM_DOCS / 2);
-      cmd.setQuery(new TermQuery(new Term("field1_s", "foo")));
+      QueryCommand cmd = createBasicQueryCommand(NUM_DOCS / 2, 10, "field1_s", "foo");
+      cmd.clearFlags(SolrIndexSearcher.NO_CHECK_QCACHE | SolrIndexSearcher.NO_SET_QCACHE);
       searcher.search(new QueryResult(), cmd);
-      QueryResult qr = new QueryResult();
-      searcher.search(qr, cmd);
-      assertMatchesGraterThan(NUM_DOCS, qr);
+      assertMatchesGreaterThan(NUM_DOCS, searcher, cmd);
       return null;
     });
   }
   
   public void testHighMinExactHitsWithQueryResultCache() throws IOException {
     h.getCore().withSearcher(searcher -> {
-      QueryCommand cmd = new QueryCommand();
-      cmd.setMinExactHits(NUM_DOCS);
-      cmd.setQuery(new TermQuery(new Term("field1_s", "foo")));
+      QueryCommand cmd = createBasicQueryCommand(NUM_DOCS, 2, "field1_s", "foo");
+      cmd.clearFlags(SolrIndexSearcher.NO_CHECK_QCACHE | SolrIndexSearcher.NO_SET_QCACHE);
       searcher.search(new QueryResult(), cmd);
-      QueryResult qr = new QueryResult();
-      searcher.search(qr, cmd);
-      assertMatchesEqual(NUM_DOCS, qr);
+      assertMatchesEqual(NUM_DOCS, searcher, cmd);
       return null;
     });
   }
   
   public void testMinExactHitsMoreRows() throws IOException {
     h.getCore().withSearcher(searcher -> {
-      QueryCommand cmd = new QueryCommand();
-      cmd.setMinExactHits(2);
-      cmd.setLen(NUM_DOCS);
-      cmd.setQuery(new TermQuery(new Term("field1_s", "foo")));
-      QueryResult qr = new QueryResult();
-      searcher.search(qr, cmd);
-      assertMatchesEqual(NUM_DOCS, qr);
+      QueryCommand cmd = createBasicQueryCommand(2, NUM_DOCS, "field1_s", "foo");
+      assertMatchesEqual(NUM_DOCS, searcher, cmd);
       return null;
     });
   }
   
   public void testMinExactHitsMatchWithDocSet() throws IOException {
     h.getCore().withSearcher(searcher -> {
-      QueryCommand cmd = new QueryCommand();
+      QueryCommand cmd = createBasicQueryCommand(2, 2, "field1_s", "foo");
+      assertMatchesGreaterThan(NUM_DOCS, searcher, cmd);
+      
       cmd.setNeedDocSet(true);
-      cmd.setMinExactHits(2);
-      cmd.setQuery(new TermQuery(new Term("field1_s", "foo")));
-      searcher.search(new QueryResult(), cmd);
-      QueryResult qr = new QueryResult();
-      searcher.search(qr, cmd);
-      assertMatchesEqual(NUM_DOCS, qr);
+      assertMatchesEqual(NUM_DOCS, searcher, cmd);
       return null;
     });
   }
   
   public void testMinExactHitsWithMaxScoreRequested() throws IOException {
     h.getCore().withSearcher(searcher -> {
-      QueryCommand cmd = new QueryCommand();
-      cmd.setMinExactHits(2);
+      QueryCommand cmd = createBasicQueryCommand(2, 2, "field1_s", "foo");
       cmd.setFlags(SolrIndexSearcher.GET_SCORES);
-      cmd.setQuery(new TermQuery(new Term("field1_s", "foo")));
-      searcher.search(new QueryResult(), cmd);
-      QueryResult qr = new QueryResult();
-      searcher.search(qr, cmd);
-      assertMatchesGraterThan(NUM_DOCS, qr);
+      QueryResult qr = assertMatchesGreaterThan(NUM_DOCS, searcher, cmd);
       assertNotEquals(Float.NaN, qr.getDocList().maxScore());
       return null;
     });
   }
+  
+  public void testMinExactWithFilters() throws Exception {
+    
+    h.getCore().withSearcher(searcher -> {
+      //Sanity Check - No Filter
+      QueryCommand cmd = createBasicQueryCommand(1, 1, "field4_t", "0");
+      assertMatchesGreaterThan(NUM_DOCS, searcher, cmd);
+      return null;
+    });
+    
+    
+    h.getCore().withSearcher(searcher -> {
+      QueryCommand cmd = createBasicQueryCommand(1, 1, "field4_t", "0");
+      Query filterQuery = new TermQuery(new Term("field4_t", "19"));
+      cmd.setFilterList(filterQuery);
+      assertNull(searcher.getProcessedFilter(null, cmd.getFilterList()).postFilter);
+      assertMatchesEqual(1, searcher, cmd);
+      return null;
+    });
+  }
+  
+  public void testMinExactWithPostFilters() throws Exception {
+    h.getCore().withSearcher(searcher -> {
+      //Sanity Check - No Filter
+      QueryCommand cmd = createBasicQueryCommand(1, 1, "field4_t", "0");
+      assertMatchesGreaterThan(NUM_DOCS, searcher, cmd);
+      return null;
+    });
+    
+    
+    h.getCore().withSearcher(searcher -> {
+      QueryCommand cmd = createBasicQueryCommand(1, 1, "field4_t", "0");
+      MockPostFilter filterQuery = new MockPostFilter(1, 101);
+      cmd.setFilterList(filterQuery);
+      assertNotNull(searcher.getProcessedFilter(null, cmd.getFilterList()).postFilter);
+      assertMatchesEqual(1, searcher, cmd);
+      return null;
+    });
+    
+    h.getCore().withSearcher(searcher -> {
+      QueryCommand cmd = createBasicQueryCommand(1, 1, "field4_t", "0");
+      MockPostFilter filterQuery = new MockPostFilter(100, 101);
+      cmd.setFilterList(filterQuery);
+      assertNotNull(searcher.getProcessedFilter(null, cmd.getFilterList()).postFilter);
+      assertMatchesGreaterThan(NUM_DOCS, searcher, cmd);
+      return null;
+    });
+    
+  }
+  
+  public void testMinExactWithPostFilterThatChangesScoreMode() throws Exception {
+    h.getCore().withSearcher(searcher -> {
+      QueryCommand cmd = createBasicQueryCommand(1, 1, "field4_t", "0");
+      // Use ScoreMode.COMPLETE for the PostFilter
+      MockPostFilter filterQuery = new MockPostFilter(100, 101, ScoreMode.COMPLETE);
+      cmd.setFilterList(filterQuery);
+      assertNotNull(searcher.getProcessedFilter(null, cmd.getFilterList()).postFilter);
+      assertMatchesEqual(NUM_DOCS, searcher, cmd);
+      return null;
+    });
+  }
+
+  private QueryCommand createBasicQueryCommand(int minExactHits, int length, String field, String q) {
+    QueryCommand cmd = new QueryCommand();
+    cmd.setMinExactHits(minExactHits);
+    cmd.setLen(length);
+    cmd.setFlags(SolrIndexSearcher.NO_CHECK_QCACHE | SolrIndexSearcher.NO_SET_QCACHE);
+    cmd.setQuery(new TermQuery(new Term(field, q)));
+    return cmd;
+  }
+  
+  private final static class MockPostFilter  extends TermQuery implements PostFilter {
+    
+    private final int cost;
+    private final int maxDocsToCollect;
+    private final ScoreMode scoreMode;
+    
+    public MockPostFilter(int maxDocsToCollect, int cost, ScoreMode scoreMode) {
+      super(new Term("foo", "bar"));//The term won't really be used. just the collector
+      assert cost > 100;
+      this.cost = cost;
+      this.maxDocsToCollect = maxDocsToCollect;
+      this.scoreMode = scoreMode;
+    }
+
+    public MockPostFilter(int maxDocsToCollect, int cost) {
+      this(maxDocsToCollect, cost, null);
+    }
+    
+    @Override
+    public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+      throw new UnsupportedOperationException("This class is only intended to be used as a PostFilter");
+    }
+
+    @Override
+    public boolean getCache() {
+      return false;
+    }
+
+    @Override
+    public void setCache(boolean cache) {}
+
+    @Override
+    public int getCost() {
+      return cost;
+    }
+
+    @Override
+    public void setCost(int cost) {}
+
+    @Override
+    public boolean getCacheSep() {
+      return false;
+    }
+
+    @Override
+    public void setCacheSep(boolean cacheSep) {
+    }
+
+    @Override
+    public DelegatingCollector getFilterCollector(IndexSearcher searcher) {
+      return new DelegatingCollector() {
+        private int collected = 0;
+        @Override
+        public void collect(int doc) throws IOException {
+          if (++collected <= maxDocsToCollect) {
+            super.collect(doc);
+          }
+        }
+        
+        @Override
+        public ScoreMode scoreMode() {
+          if (scoreMode != null) {
+            return scoreMode;
+          }
+          return super.scoreMode();
+        }
+      };
+    }
+    
+  }
 }
diff --git a/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java
index 09444e1..2c6f9c4 100644
--- a/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java
+++ b/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java
@@ -1039,4 +1039,33 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 {
     assertQEx("Should Fail For collapsing on Date fields", "Collapsing field should be of either String, Int or Float type",
         req("q", "*:*", "fq", "{!collapse field=group_dt}"), SolrException.ErrorCode.BAD_REQUEST);
   }
+  
+  @Test
+  public void testMinExactHitsDisabledByCollapse() throws Exception {
+    int numDocs = 10;
+    String collapseFieldInt = "field_ti_dv";
+    String collapseFieldFloat = "field_tf_dv";
+    String collapseFieldString = "field_s_dv";
+    for (int i = 0 ; i < numDocs ; i ++) {
+      assertU(adoc(
+          "id", String.valueOf(i),
+          "field_s", String.valueOf(i % 2),
+          collapseFieldInt, String.valueOf(i),
+          collapseFieldFloat, String.valueOf(i),
+          collapseFieldString, String.valueOf(i)));
+        assertU(commit());
+    }
+    
+    for (String collapseField : new String[] {collapseFieldInt, collapseFieldFloat, collapseFieldString}) {
+      assertQ(req(
+          "q", "{!cache=false}field_s:1",
+          "rows", "1",
+          "minExactHits", "1",
+          // this collapse will end up matching all docs
+          "fq", "{!collapse field=" + collapseField + " nullPolicy=expand}"// nullPolicy needed due to a bug when val=0
+          ),"//*[@numFoundExact='true']"
+          ,"//*[@numFound='" + (numDocs/2) + "']"
+          );
+    }
+  }
 }