You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2013/08/15 19:50:30 UTC

svn commit: r1514402 - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/search/ core/src/java/org/apache/solr/spelling/ core/src/test/org/apache/solr/spelling/

Author: hossman
Date: Thu Aug 15 17:50:30 2013
New Revision: 1514402

URL: http://svn.apache.org/r1514402
Log:
SOLR-5122: Fixed bug in spellcheck.collateMaxCollectDocs.  Eliminates risk of divide by zero, and makes estimated hit counts meaningful in non-optimized indexes.

Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/EarlyTerminatingCollector.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/EarlyTerminatingCollectorException.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorTest.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1514402&r1=1514401&r2=1514402&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Thu Aug 15 17:50:30 2013
@@ -126,6 +126,10 @@ Bug Fixes
 * SOLR-4764: When using NRT, just init the first reader from IndexWriter.
   (Robert Muir, Mark Miller)
   
+* SOLR-5122: Fixed bug in spellcheck.collateMaxCollectDocs.  Eliminates risk 
+  of divide by zero, and makes estimated hit counts meaningful in non-optimized
+  indexes.  (hossman)
+
 Optimizations
 ----------------------
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/EarlyTerminatingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/EarlyTerminatingCollector.java?rev=1514402&r1=1514401&r2=1514402&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/EarlyTerminatingCollector.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/EarlyTerminatingCollector.java Thu Aug 15 17:50:30 2013
@@ -29,11 +29,14 @@ import org.apache.lucene.search.Scorer;
  * </p>
  */
 public class EarlyTerminatingCollector extends Collector {
-  private int numCollected;
-  private int lastDocId = -1;
-  private int maxDocsToCollect;
-  private Collector delegate;
-  
+
+  private final int maxDocsToCollect;
+  private final Collector delegate;
+
+  private int numCollected = 0;
+  private int prevReaderCumulativeSize = 0;
+  private int currentReaderSize = 0;  
+
   /**
    * <p>
    *  Wraps a {@link Collector}, throwing {@link EarlyTerminatingCollectorException} 
@@ -44,42 +47,40 @@ public class EarlyTerminatingCollector e
    * 
    */
   public EarlyTerminatingCollector(Collector delegate, int maxDocsToCollect) {
+    assert 0 < maxDocsToCollect;
+    assert null != delegate;
+
     this.delegate = delegate;
     this.maxDocsToCollect = maxDocsToCollect;
   }
 
+  /**
+   * This collector requires that docs be collected in order, otherwise
+   * the computed number of scanned docs in the resulting 
+   * {@link EarlyTerminatingCollectorException} will be meaningless.
+   */
   @Override
   public boolean acceptsDocsOutOfOrder() {
-    return delegate.acceptsDocsOutOfOrder();
+    return false;
   }
 
   @Override
   public void collect(int doc) throws IOException {
     delegate.collect(doc);
-    lastDocId = doc;    
     numCollected++;  
-    if(numCollected==maxDocsToCollect) {
-      throw new EarlyTerminatingCollectorException(numCollected, lastDocId);
+    if(maxDocsToCollect <= numCollected) {
+      throw new EarlyTerminatingCollectorException
+        (numCollected, prevReaderCumulativeSize + (doc + 1));
     }
   }
   @Override
   public void setNextReader(AtomicReaderContext context) throws IOException {
-    delegate.setNextReader(context);    
+    prevReaderCumulativeSize += currentReaderSize; // not current any more
+    currentReaderSize = context.reader().maxDoc() - 1;
+    delegate.setNextReader(context);
   }
   @Override
   public void setScorer(Scorer scorer) throws IOException {
     delegate.setScorer(scorer);    
   }
-  public int getNumCollected() {
-    return numCollected;
-  }
-  public void setNumCollected(int numCollected) {
-    this.numCollected = numCollected;
-  }
-  public int getLastDocId() {
-    return lastDocId;
-  }
-  public void setLastDocId(int lastDocId) {
-    this.lastDocId = lastDocId;
-  }
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/EarlyTerminatingCollectorException.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/EarlyTerminatingCollectorException.java?rev=1514402&r1=1514401&r2=1514402&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/EarlyTerminatingCollectorException.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/EarlyTerminatingCollectorException.java Thu Aug 15 17:50:30 2013
@@ -23,23 +23,40 @@ package org.apache.solr.search;
  */
 public class EarlyTerminatingCollectorException extends RuntimeException {
   private static final long serialVersionUID = 5939241340763428118L;  
-  private int lastDocId = -1;
+  private int numberScanned;
   private int numberCollected;
   
-  public EarlyTerminatingCollectorException(int numberCollected, int lastDocId) {
+  public EarlyTerminatingCollectorException(int numberCollected, int numberScanned) {
+    assert numberCollected <= numberScanned : numberCollected+"<="+numberScanned;
+    assert 0 < numberCollected;
+    assert 0 < numberScanned;
+
     this.numberCollected = numberCollected;
-    this.lastDocId = lastDocId;
+    this.numberScanned = numberScanned;
   }
-  public int getLastDocId() {
-    return lastDocId;
-  }
-  public void setLastDocId(int lastDocId) {
-    this.lastDocId = lastDocId;
+  /**
+   * The total number of documents in the index that were "scanned" by 
+   * the index when collecting the {@see #getNumberCollected()} documents 
+   * that triggered this exception.
+   * <p>
+   * This number represents the sum of:
+   * </p>
+   * <ul>
+   *  <li>The total number of documents in all AtomicReaders
+   *      that were fully exhausted during collection
+   *  </li>
+   *  <li>The id of the last doc collected in the last AtomicReader
+   *      consulted during collection.
+   *  </li>
+   * </ul>
+   **/
+  public int getNumberScanned() {
+    return numberScanned;
   }
+  /**
+   * The number of documents collected that resulted in early termination
+   */
   public int getNumberCollected() {
     return numberCollected;
   }
-  public void setNumberCollected(int numberCollected) {
-    this.numberCollected = numberCollected;
-  }
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java?rev=1514402&r1=1514401&r2=1514402&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java Thu Aug 15 17:50:30 2013
@@ -147,10 +147,14 @@ public class SpellCheckCollator {
           hits = (Integer) checkResponse.rsp.getToLog().get("hits");
         } catch (EarlyTerminatingCollectorException etce) {
           assert (docCollectionLimit > 0);
-          if (etce.getLastDocId() + 1 == maxDocId) {
-            hits = docCollectionLimit;
+          assert 0 < etce.getNumberScanned();
+          assert 0 < etce.getNumberCollected();
+
+          if (etce.getNumberScanned() == maxDocId) {
+            hits = etce.getNumberCollected();
           } else {
-            hits = maxDocId / ((etce.getLastDocId() + 1) / docCollectionLimit);
+            hits = (int) ( ((float)( maxDocId * etce.getNumberCollected() )) 
+                           / (float)etce.getNumberScanned() );
           }
         } catch (Exception e) {
           LOG.warn("Exception trying to re-query to check if a spell check possibility would return any hits.", e);

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorTest.java?rev=1514402&r1=1514401&r2=1514402&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SpellCheckCollatorTest.java Thu Aug 15 17:50:30 2013
@@ -21,7 +21,9 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.lucene.util._TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.GroupParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -44,24 +46,52 @@ public class SpellCheckCollatorTest exte
   @BeforeClass
   public static void beforeClass() throws Exception {
     initCore("solrconfig-spellcheckcomponent.xml", "schema.xml");
-    assertNull(h.validateUpdate(adoc("id", "0", "lowerfilt", "faith hope and love", "teststop", "metanoia")));
-    assertNull(h.validateUpdate(adoc("id", "1", "lowerfilt", "faith hope and loaves")));
-    assertNull(h.validateUpdate(adoc("id", "2", "lowerfilt", "fat hops and loaves")));
-    assertNull(h.validateUpdate(adoc("id", "3", "lowerfilt", "faith of homer", "teststop", "metanoia")));
-    assertNull(h.validateUpdate(adoc("id", "4", "lowerfilt", "fat of homer")));
-    assertNull(h.validateUpdate(adoc("id", "5", "lowerfilt1", "peace")));
-    assertNull(h.validateUpdate(adoc("id", "6", "lowerfilt", "hyphenated word")));
-    assertNull(h.validateUpdate(adoc("id", "7", "teststop", "Jane filled out a form at Charles De Gaulle")));
-    assertNull(h.validateUpdate(adoc("id", "8", "teststop", "Dick flew from Heathrow")));
-    assertNull(h.validateUpdate(adoc("id", "9", "teststop", "Jane is stuck in customs because Spot chewed up the form")));
-    assertNull(h.validateUpdate(adoc("id", "10", "teststop", "Once in Paris Dick built a fire on the hearth")));
-    assertNull(h.validateUpdate(adoc("id", "11", "teststop", "Dick waited for Jane as he watched the sparks flow upward")));
-    assertNull(h.validateUpdate(adoc("id", "12", "teststop", "This June parisian rendez-vous is ruined because of a customs snafu")));
-    assertNull(h.validateUpdate(adoc("id", "13", "teststop", "partisan political machine", "teststop", "metanoia")));
-    assertNull(h.validateUpdate(adoc("id", "14", "teststop", "metanoia")));
-    assertNull(h.validateUpdate(adoc("id", "15", "teststop", "metanoia")));
-    assertNull(h.validateUpdate(adoc("id", "16", "teststop", "metanoia")));
-    assertNull(h.validateUpdate(commit()));
+    assertU(adoc("id", "0", 
+                 "lowerfilt", "faith hope and love", 
+                 "teststop", "metanoia"));
+    assertU(adoc("id", "1", 
+                 "lowerfilt", "faith hope and loaves",
+                 "teststop", "everyother"));
+    assertU(adoc("id", "2", 
+                 "lowerfilt", "fat hops and loaves"));
+    assertU(adoc("id", "3", 
+                 "lowerfilt", "faith of homer", 
+                 "teststop", "metanoia",
+                 "teststop", "everyother"));
+    assertU(adoc("id", "4", 
+                 "lowerfilt", "fat of homer"));
+    assertU(adoc("id", "5", 
+                 "lowerfilt1", "peace",
+                 "teststop", "everyother"));
+    assertU(adoc("id", "6", 
+                 "lowerfilt", "hyphenated word"));
+    assertU(adoc("id", "7", 
+                 "teststop", "Jane filled out a form at Charles De Gaulle",
+                 "teststop", "everyother"));
+    assertU(adoc("id", "8", 
+                 "teststop", "Dick flew from Heathrow"));
+    assertU(adoc("id", "9", 
+                 "teststop", "Jane is stuck in customs because Spot chewed up the form",
+                 "teststop", "everyother"));
+    assertU(adoc("id", "10", 
+                 "teststop", "Once in Paris Dick built a fire on the hearth"));
+    assertU(adoc("id", "11", 
+                 "teststop", "Dick waited for Jane as he watched the sparks flow upward",
+                 "teststop", "everyother"));
+    assertU(adoc("id", "12", 
+                 "teststop", "This June parisian rendez-vous is ruined because of a customs snafu"));
+    assertU(adoc("id", "13", 
+                 "teststop", "partisan political machine", 
+                 "teststop", "metanoia",
+                 "teststop", "everyother"));
+    assertU(adoc("id", "14", 
+                 "teststop", "metanoia"));
+    assertU(adoc("id", "15", 
+                 "teststop", "metanoia",
+                 "teststop", "everyother"));
+    assertU(adoc("id", "16", 
+                 "teststop", "metanoia"));
+    assertU(commit());
   }
 
   @Test
@@ -436,71 +466,66 @@ public class SpellCheckCollatorTest exte
   }
 
   @Test
-  @Ignore("SOLR-5122: estimates don't seem to make any sense")
   public void testEstimatedHitCounts() throws Exception {
-   assertQ(
-        req(
-          SpellCheckComponent.COMPONENT_NAME, "true",
-          SpellCheckComponent.SPELLCHECK_DICT, "direct",
-          SpellingParams.SPELLCHECK_COUNT, "1",   
-          SpellingParams.SPELLCHECK_COLLATE, "true",
-          SpellingParams.SPELLCHECK_MAX_COLLATION_TRIES, "1",
-          SpellingParams.SPELLCHECK_MAX_COLLATIONS, "1",
-          SpellingParams.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "true",          
-          "qt", "spellCheckCompRH",          
-          CommonParams.Q, "teststop:metnoia"
-        ),
-        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/str[@name='collationQuery']='teststop:metanoia'",
-        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/int[@name='hits']=6"        
-      );
-    assertQ(
-        req(
-          SpellCheckComponent.COMPONENT_NAME, "true",
-          SpellCheckComponent.SPELLCHECK_DICT, "direct",
-          SpellingParams.SPELLCHECK_COUNT, "1",   
-          SpellingParams.SPELLCHECK_COLLATE, "true",
-          SpellingParams.SPELLCHECK_MAX_COLLATION_TRIES, "1",
-          SpellingParams.SPELLCHECK_MAX_COLLATIONS, "1",
-          SpellingParams.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "true",
-          SpellingParams.SPELLCHECK_COLLATE_MAX_COLLECT_DOCS, "1",
-          "qt", "spellCheckCompRH",          
-          CommonParams.Q, "teststop:metnoia"
-        ),
-        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/str[@name='collationQuery']='teststop:metanoia'",
-        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/int[@name='hits']=17"        
-      );
-    assertQ(
-        req(
-          SpellCheckComponent.COMPONENT_NAME, "true",
-          SpellCheckComponent.SPELLCHECK_DICT, "direct",
-          SpellingParams.SPELLCHECK_COUNT, "1",   
-          SpellingParams.SPELLCHECK_COLLATE, "true",
-          SpellingParams.SPELLCHECK_MAX_COLLATION_TRIES, "1",
-          SpellingParams.SPELLCHECK_MAX_COLLATIONS, "1",
-          SpellingParams.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "true",
-          SpellingParams.SPELLCHECK_COLLATE_MAX_COLLECT_DOCS, "3",
-          "qt", "spellCheckCompRH",          
-          CommonParams.Q, "teststop:metnoia"
-        ),
-        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/str[@name='collationQuery']='teststop:metanoia'",
-        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/int[@name='hits']=4"        
-      );
-    assertQ(
-        req(
-          SpellCheckComponent.COMPONENT_NAME, "true",
-          SpellCheckComponent.SPELLCHECK_DICT, "direct",
-          SpellingParams.SPELLCHECK_COUNT, "1",   
-          SpellingParams.SPELLCHECK_COLLATE, "true",
-          SpellingParams.SPELLCHECK_MAX_COLLATION_TRIES, "1",
-          SpellingParams.SPELLCHECK_MAX_COLLATIONS, "1",
-          SpellingParams.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "true",
-          SpellingParams.SPELLCHECK_COLLATE_MAX_COLLECT_DOCS, "100",
-          "qt", "spellCheckCompRH",          
-          CommonParams.Q, "teststop:metnoia"
-        ),
-        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/str[@name='collationQuery']='teststop:metanoia'",
-        "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/int[@name='hits']=6"        
-      );
+    final String xpathPrefix = 
+      "//lst[@name='spellcheck']/lst[@name='suggestions']/lst[@name='collation']/";
+    final SolrParams reusedParams = params
+      (SpellCheckComponent.COMPONENT_NAME, "true",
+       SpellCheckComponent.SPELLCHECK_DICT, "direct",
+       SpellingParams.SPELLCHECK_COUNT, "1",   
+       SpellingParams.SPELLCHECK_COLLATE, "true",
+       SpellingParams.SPELLCHECK_MAX_COLLATION_TRIES, "1",
+       SpellingParams.SPELLCHECK_MAX_COLLATIONS, "1",
+       SpellingParams.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "true",          
+       "qt", "spellCheckCompRH");       
+
+    // default case, no SPELLCHECK_COLLATE_MAX_COLLECT_DOCS should be exact num hits
+    assertQ(req(reusedParams, 
+                CommonParams.Q, "teststop:metnoia")
+            , xpathPrefix + "str[@name='collationQuery']='teststop:metanoia'"
+            , xpathPrefix + "int[@name='hits']=6"        
+            );
+
+    // specifying 0 means "exact" same as default, but specifing a value greater 
+    // then the total number of docs in the index should also result in it
+    // "estimating" and getting exact number as well.
+    for (String val : new String[] { "0", "30", "100", "10000" }) {
+      assertQ(req(reusedParams,
+                  CommonParams.Q, "teststop:metnoia",
+                  SpellingParams.SPELLCHECK_COLLATE_MAX_COLLECT_DOCS, val)
+              , xpathPrefix + "str[@name='collationQuery']='teststop:metanoia'"
+              , xpathPrefix + "int[@name='hits']=6"        
+              );
+    }
+
+    // values between 0 and the num docs in the index should not error, and should 
+    // produce an estimate no more then the total number of docs
+    final int iters = atLeast(10);
+    for (int iter = 0; iter < iters; iter++) {
+      final int val = _TestUtil.nextInt(random(), 1, 17);
+      assertQ(req(reusedParams,
+                  CommonParams.Q, "teststop:metnoia",
+                  SpellingParams.SPELLCHECK_COLLATE_MAX_COLLECT_DOCS, ""+val)
+              , xpathPrefix + "str[@name='collationQuery']='teststop:metanoia'"
+              , xpathPrefix + "int[@name='hits' and . <= 17 and 0 < .]"        
+              );
+    }
+
+    // "everYother" appears in every other doc in the index, so "everother" 
+    // should produce a "decent" aproximation of the number of hits (8)
+    // for any 5 <= SPELLCHECK_COLLATE_MAX_COLLECT_DOCS
+    //
+    // (we have to be kind of flexible with our definition of "decent"
+    // since we're dealing with a fairly small index here)
+    for (int val = 5; val <= 20; val++) {
+      assertQ(req(reusedParams,
+                  CommonParams.Q, "teststop:everother",
+                  SpellingParams.SPELLCHECK_COLLATE_MAX_COLLECT_DOCS, ""+val)
+              , xpathPrefix + "str[@name='collationQuery']='teststop:everyother'"
+              , xpathPrefix + "int[@name='hits' and 6 <= . and . <= 10]"        
+              );
+    }
+
   }  
   
 }