You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2011/04/20 05:43:27 UTC

svn commit: r1095260 - in /lucene/dev/trunk: lucene/contrib/ lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/ lucene/contrib/highlighter/src/test/org/apache/lucene/search/highlight/ solr/ solr/src/java/org/apache/solr/highlight/

Author: markrmiller
Date: Wed Apr 20 03:43:27 2011
New Revision: 1095260

URL: http://svn.apache.org/viewvc?rev=1095260&view=rev
Log:
LUCENE-2939: Highlighter should try and use maxDocCharsToAnalyze in WeightedSpanTermExtractor when adding a new field to MemoryIndex as well as when using CachingTokenStream
SOLR-2390: Performance of usePhraseHighlighter is terrible on very large Documents, regardless of hl.maxDocCharsToAnalyze

Added:
    lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/OffsetLimitTokenFilter.java   (with props)
    lucene/dev/trunk/lucene/contrib/highlighter/src/test/org/apache/lucene/search/highlight/OffsetLimitTokenFilterTest.java   (with props)
Modified:
    lucene/dev/trunk/lucene/contrib/CHANGES.txt
    lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/Highlighter.java
    lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/QueryScorer.java
    lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java

Modified: lucene/dev/trunk/lucene/contrib/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/CHANGES.txt?rev=1095260&r1=1095259&r2=1095260&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/contrib/CHANGES.txt Wed Apr 20 03:43:27 2011
@@ -45,10 +45,15 @@ API Changes
 
 ======================= Lucene 3.x (not yet released) =======================
 
-Bug fixes
+Bug Fixes
 
  * LUCENE-3026: SmartChineseAnalyzer's WordTokenFilter threw NullPointerException
    on sentences longer than 32,767 characters.  (wangzhenghang via Robert Muir)
+   
+ * LUCENE-2939: Highlighter should try and use maxDocCharsToAnalyze in 
+   WeightedSpanTermExtractor when adding a new field to MemoryIndex as well as 
+   when using CachingTokenStream. This can be a significant performance bug for
+   large documents. (Mark Miller)
 
 New Features
 

Modified: lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/Highlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/Highlighter.java?rev=1095260&r1=1095259&r2=1095260&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/Highlighter.java (original)
+++ lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/Highlighter.java Wed Apr 20 03:43:27 2011
@@ -197,6 +197,11 @@ public class Highlighter
 	    tokenStream.reset();
 	    
 		TextFragment currentFrag =	new TextFragment(newText,newText.length(), docFrags.size());
+		
+    if (fragmentScorer instanceof QueryScorer) {
+      ((QueryScorer) fragmentScorer).setMaxDocCharsToAnalyze(maxDocCharsToAnalyze);
+    }
+    
 		TokenStream newStream = fragmentScorer.init(tokenStream);
 		if(newStream != null) {
 		  tokenStream = newStream;

Added: lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/OffsetLimitTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/OffsetLimitTokenFilter.java?rev=1095260&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/OffsetLimitTokenFilter.java (added)
+++ lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/OffsetLimitTokenFilter.java Wed Apr 20 03:43:27 2011
@@ -0,0 +1 @@
+package org.apache.lucene.search.highlight;

/**
 * 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.
 */

import java.io.IOException;

import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.TokenStream;
import org.apache
 .lucene.analysis.tokenattributes.OffsetAttribute;

/**
 * This TokenFilter limits the number of tokens while indexing by adding up the
 * current offset.
 */
public final class OffsetLimitTokenFilter extends TokenFilter {
  
  private int offsetCount;
  private OffsetAttribute offsetAttrib = getAttribute(OffsetAttribute.class);
  private int offsetLimit;
  
  public OffsetLimitTokenFilter(TokenStream input, int offsetLimit) {
    super(input);
    this.offsetLimit = offsetLimit;
  }
  
  @Override
  public boolean incrementToken() throws IOException {
    if (offsetCount < offsetLimit && input.incrementToken()) {
      int offsetLength = offsetAttrib.endOffset() - offsetAttrib.startOffset();
      offsetCount += offsetLength;
      return true;
    }
    return false;
  }
  
  @Override
  public void reset() throws IOException {
    super.reset();
    offsetCount = 0;
  }
  
}
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/QueryScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/QueryScorer.java?rev=1095260&r1=1095259&r2=1095260&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/QueryScorer.java (original)
+++ lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/QueryScorer.java Wed Apr 20 03:43:27 2011
@@ -54,6 +54,7 @@ public class QueryScorer implements Scor
   private IndexReader reader;
   private boolean skipInitExtractor;
   private boolean wrapToCaching = true;
+  private int maxCharsToAnalyze;
 
   /**
    * @param query Query to use for highlighting
@@ -209,7 +210,7 @@ public class QueryScorer implements Scor
   private TokenStream initExtractor(TokenStream tokenStream) throws IOException {
     WeightedSpanTermExtractor qse = defaultField == null ? new WeightedSpanTermExtractor()
         : new WeightedSpanTermExtractor(defaultField);
-
+    qse.setMaxDocCharsToAnalyze(maxCharsToAnalyze);
     qse.setExpandMultiTermQuery(expandMultiTermQuery);
     qse.setWrapIfNotCachingTokenFilter(wrapToCaching);
     if (reader == null) {
@@ -265,4 +266,8 @@ public class QueryScorer implements Scor
   public void setWrapIfNotCachingTokenFilter(boolean wrap) {
     this.wrapToCaching = wrap;
   }
+
+  public void setMaxDocCharsToAnalyze(int maxDocCharsToAnalyze) {
+    this.maxCharsToAnalyze = maxDocCharsToAnalyze;
+  }
 }

Modified: lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java?rev=1095260&r1=1095259&r2=1095260&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java (original)
+++ lucene/dev/trunk/lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java Wed Apr 20 03:43:27 2011
@@ -56,6 +56,7 @@ public class WeightedSpanTermExtractor {
   private boolean expandMultiTermQuery;
   private boolean cachedTokenStream;
   private boolean wrapToCaching = true;
+  private int maxDocCharsToAnalyze;
 
   public WeightedSpanTermExtractor() {
   }
@@ -320,13 +321,13 @@ public class WeightedSpanTermExtractor {
 
   private AtomicReaderContext getLeafContextForField(String field) throws IOException {
     if(wrapToCaching && !cachedTokenStream && !(tokenStream instanceof CachingTokenFilter)) {
-      tokenStream = new CachingTokenFilter(tokenStream);
+      tokenStream = new CachingTokenFilter(new OffsetLimitTokenFilter(tokenStream, maxDocCharsToAnalyze));
       cachedTokenStream = true;
     }
     AtomicReaderContext context = readers.get(field);
     if (context == null) {
       MemoryIndex indexer = new MemoryIndex();
-      indexer.addField(field, tokenStream);
+      indexer.addField(field, new OffsetLimitTokenFilter(tokenStream, maxDocCharsToAnalyze));
       tokenStream.reset();
       IndexSearcher searcher = indexer.createSearcher();
       // MEM index has only atomic ctx
@@ -545,4 +546,8 @@ public class WeightedSpanTermExtractor {
   public void setWrapIfNotCachingTokenFilter(boolean wrap) {
     this.wrapToCaching = wrap;
   }
+
+  protected final void setMaxDocCharsToAnalyze(int maxDocCharsToAnalyze) {
+    this.maxDocCharsToAnalyze = maxDocCharsToAnalyze;
+  }
 }

Added: lucene/dev/trunk/lucene/contrib/highlighter/src/test/org/apache/lucene/search/highlight/OffsetLimitTokenFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/highlighter/src/test/org/apache/lucene/search/highlight/OffsetLimitTokenFilterTest.java?rev=1095260&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/contrib/highlighter/src/test/org/apache/lucene/search/highlight/OffsetLimitTokenFilterTest.java (added)
+++ lucene/dev/trunk/lucene/contrib/highlighter/src/test/org/apache/lucene/search/highlight/OffsetLimitTokenFilterTest.java Wed Apr 20 03:43:27 2011
@@ -0,0 +1 @@
+package org.apache.lucene.search.highlight;

/**
 * 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.
 */

import java.io.Reader;
import java.io.StringReader;

import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseToken
 StreamTestCase;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.TokenStream;

public class OffsetLimitTokenFilterTest extends BaseTokenStreamTestCase {
  
  public void testFilter() throws Exception {
    TokenStream stream = new MockTokenizer(new StringReader(
        "short toolong evenmuchlongertext a ab toolong foo"),
        MockTokenizer.WHITESPACE, false);
    OffsetLimitTokenFilter filter = new OffsetLimitTokenFilter(stream, 10);
    assertTokenStreamContents(filter, new String[] {"short", "toolong"});
    
    stream = new MockTokenizer(new StringReader(
    "short toolong evenmuchlongertext a ab toolong foo"),
    MockTokenizer.WHITESPACE, false);
    filter = new OffsetLimitTokenFilter(stream, 12);
    assertTokenStreamContents(filter, new String[] {"short", "toolong"});
    
    stream = new MockTokenizer(new StringReader(
        "short toolong evenmuchlongertext a ab toolong foo"),
        MockTokenizer.WHITESPACE, false);
   
  filter = new OffsetLimitTokenFilter(stream, 30);
    assertTokenStreamContents(filter, new String[] {"short", "toolong",
        "evenmuchlongertext"});
    
    
    checkOneTermReuse(new Analyzer() {
      
      @Override
      public TokenStream tokenStream(String fieldName, Reader reader) {
        return new OffsetLimitTokenFilter(new MockTokenizer(reader,
            MockTokenizer.WHITESPACE, false), 10);
      }
    }, "llenges", "llenges");
  }
}
\ No newline at end of file

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1095260&r1=1095259&r2=1095260&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Wed Apr 20 03:43:27 2011
@@ -278,6 +278,9 @@ Bug Fixes
 
 * SOLR-2333: The "rename" core admin action does not persist the new name to solr.xml
   (Rasmus Hahn, Paul R. Brown via Mark Miller)
+  
+* SOLR-2390: Performance of usePhraseHighlighter is terrible on very large Documents, 
+  regardless of hl.maxDocCharsToAnalyze. (Mark Miller)
 
 Other Changes
 ----------------------

Modified: lucene/dev/trunk/solr/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java?rev=1095260&r1=1095259&r2=1095260&view=diff
==============================================================================
--- lucene/dev/trunk/solr/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java (original)
+++ lucene/dev/trunk/solr/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java Wed Apr 20 03:43:27 2011
@@ -435,12 +435,20 @@ public class DefaultSolrHighlighter exte
         // fall back to analyzer
         tstream = createAnalyzerTStream(schema, fieldName, docTexts[j]);
       }
-                   
+      
+      int maxCharsToAnalyze = params.getFieldInt(fieldName,
+          HighlightParams.MAX_CHARS,
+          Highlighter.DEFAULT_MAX_CHARS_TO_ANALYZE);
+      
       Highlighter highlighter;
       if (Boolean.valueOf(req.getParams().get(HighlightParams.USE_PHRASE_HIGHLIGHTER, "true"))) {
         // TODO: this is not always necessary - eventually we would like to avoid this wrap
         //       when it is not needed.
-        tstream = new CachingTokenFilter(tstream);
+        if (maxCharsToAnalyze < 0) {
+          tstream = new CachingTokenFilter(tstream);
+        } else {
+          tstream = new CachingTokenFilter(new OffsetLimitTokenFilter(tstream, maxCharsToAnalyze));
+        }
         
         // get highlighter
         highlighter = getPhraseHighlighter(query, fieldName, req, (CachingTokenFilter) tstream);
@@ -453,9 +461,6 @@ public class DefaultSolrHighlighter exte
         highlighter = getHighlighter(query, fieldName, req);
       }
       
-      int maxCharsToAnalyze = params.getFieldInt(fieldName,
-          HighlightParams.MAX_CHARS,
-          Highlighter.DEFAULT_MAX_CHARS_TO_ANALYZE);
       if (maxCharsToAnalyze < 0) {
         highlighter.setMaxDocCharsToAnalyze(docTexts[j].length());
       } else {