You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2010/10/15 16:25:49 UTC

svn commit: r1022934 - in /lucene/dev/trunk/lucene: ./ contrib/memory/src/java/org/apache/lucene/index/memory/ contrib/queries/src/java/org/apache/lucene/search/ contrib/queries/src/java/org/apache/lucene/search/regex/ contrib/queries/src/test/org/apac...

Author: uschindler
Date: Fri Oct 15 14:25:48 2010
New Revision: 1022934

URL: http://svn.apache.org/viewvc?rev=1022934&view=rev
Log:
LUCENE-2690: MultiTermQuery boolean rewrites per segment

Added:
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/MIGRATE.txt
    lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java
    lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/regex/RegexQuery.java
    lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/regex/TestRegexQuery.java
    lucene/dev/trunk/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestPrefixRandom.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Fri Oct 15 14:25:48 2010
@@ -147,8 +147,10 @@ API Changes
   you also override this method on upgrade.  (Robert Muir, Mike
   McCandless)
 
-* LUCENE-2691: IndexWriter.getReader() has been made package local and is now exposed via open and reopen methods on
-  IndexReader.  The semantics of the call is the same as it was prior to the API change.  (Grant Ingersoll, Mike McCandless)
+* LUCENE-2691: IndexWriter.getReader() has been made package local and is now
+  exposed via open and reopen methods on IndexReader.  The semantics of the
+  call is the same as it was prior to the API change.
+  (Grant Ingersoll, Mike McCandless)
 
 New features
 
@@ -265,6 +267,9 @@ New features
 * LUCENE-2692: Added several new SpanQuery classes for positional checking
   (match is in a range, payload is a specific value) (Grant Ingersoll)  
   
+* LUCENE-2690: MultiTermQuery boolean rewrites per segment.
+  (Uwe Schindler, Robert Muir, Mike McCandless)
+
 Optimizations
 
 * LUCENE-2410: ~20% speedup on exact (slop=0) PhraseQuery matching.

Modified: lucene/dev/trunk/lucene/MIGRATE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/MIGRATE.txt?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/MIGRATE.txt (original)
+++ lucene/dev/trunk/lucene/MIGRATE.txt Fri Oct 15 14:25:48 2010
@@ -316,3 +316,11 @@ LUCENE-1458, LUCENE-2111: Flexible Index
 * LUCENE-2691: The near-real-time API has moved from IndexWriter to
   IndexReader.  Instead of IndexWriter.getReader(), call
   IndexReader.open(IndexWriter) or IndexReader.reopen(IndexWriter).
+
+* LUCENE-2690: MultiTermQuery boolean rewrites per segment.
+  Also MultiTermQuery.getTermsEnum() now takes an AttributeSource. FuzzyTermsEnum
+  is both consumer and producer of attributes: MTQ.BoostAttribute is
+  added to the FuzzyTermsEnum and MTQ's rewrite mode consumes it.
+  The other way round MTQ.TopTermsBooleanQueryRewrite supplys a
+  global AttributeSource to each segments TermsEnum. The TermsEnum is consumer
+  and gets the current minimum competitive boosts (MTQ.MaxNonCompetitiveBoostAttribute).

Modified: lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Fri Oct 15 14:25:48 2010
@@ -886,7 +886,7 @@ public class MemoryIndex implements Seri
 
       @Override
       public int docFreq() {
-        return info.sortedTerms[termUpto].getValue().size();
+        return 1;
       }
 
       @Override

Modified: lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java (original)
+++ lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java Fri Oct 15 14:25:48 2010
@@ -29,6 +29,7 @@ import org.apache.lucene.analysis.TokenS
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.PriorityQueue;
 
@@ -199,7 +200,10 @@ public class FuzzyLikeThisQuery extends 
                   ScoreTermQueue variantsQ=new ScoreTermQueue(MAX_VARIANTS_PER_TERM); //maxNum variants considered for any one term
                   float minScore=0;
                   Term startTerm=internSavingTemplateTerm.createTerm(term);
-                  FuzzyTermsEnum fe = new FuzzyTermsEnum(reader, startTerm, f.minSimilarity, f.prefixLength);
+                  AttributeSource atts = new AttributeSource();
+                  MultiTermQuery.MaxNonCompetitiveBoostAttribute maxBoostAtt =
+                    atts.addAttribute(MultiTermQuery.MaxNonCompetitiveBoostAttribute.class);
+                  FuzzyTermsEnum fe = new FuzzyTermsEnum(reader, atts, startTerm, f.minSimilarity, f.prefixLength);
                   //store the df so all variants use same idf
                   int df = reader.docFreq(startTerm);
                   int numVariants=0;
@@ -217,7 +221,7 @@ public class FuzzyLikeThisQuery extends 
                           variantsQ.insertWithOverflow(st);
                           minScore = variantsQ.top().score; // maintain minScore
                         }
-                        boostAtt.setMaxNonCompetitiveBoost(variantsQ.size() >= MAX_VARIANTS_PER_TERM ? minScore : Float.NEGATIVE_INFINITY);
+                        maxBoostAtt.setMaxNonCompetitiveBoost(variantsQ.size() >= MAX_VARIANTS_PER_TERM ? minScore : Float.NEGATIVE_INFINITY);
                       }
                     }
 

Modified: lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/regex/RegexQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/regex/RegexQuery.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/regex/RegexQuery.java (original)
+++ lucene/dev/trunk/lucene/contrib/queries/src/java/org/apache/lucene/search/regex/RegexQuery.java Fri Oct 15 14:25:48 2010
@@ -21,6 +21,7 @@ import org.apache.lucene.search.MultiTer
 import org.apache.lucene.search.FilteredTermsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.ToStringUtils;
 
 import java.io.IOException;
@@ -60,7 +61,7 @@ public class RegexQuery extends MultiTer
   }
 
   @Override
-  protected FilteredTermsEnum getTermsEnum(IndexReader reader) throws IOException {
+  protected FilteredTermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException {
     return new RegexTermsEnum(reader, term, regexImpl);
   }
 

Modified: lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/regex/TestRegexQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/regex/TestRegexQuery.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/regex/TestRegexQuery.java (original)
+++ lucene/dev/trunk/lucene/contrib/queries/src/test/org/apache/lucene/search/regex/TestRegexQuery.java Fri Oct 15 14:25:48 2010
@@ -28,6 +28,7 @@ import org.apache.lucene.index.TermsEnum
 
 import org.apache.lucene.search.spans.SpanNearQuery;
 import org.apache.lucene.search.spans.SpanQuery;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.LuceneTestCase;
 
 public class TestRegexQuery extends LuceneTestCase {
@@ -78,7 +79,7 @@ public class TestRegexQuery extends Luce
   }
 
   public void testMatchAll() throws Exception {
-    TermsEnum terms = new RegexQuery(new Term(FN, "jum.")).getTermsEnum(searcher.getIndexReader());
+    TermsEnum terms = new RegexQuery(new Term(FN, "jum.")).getTermsEnum(searcher.getIndexReader(), new AttributeSource() /*dummy*/);
     // no term should match
     assertNull(terms.next());
   }

Modified: lucene/dev/trunk/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java (original)
+++ lucene/dev/trunk/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java Fri Oct 15 14:25:48 2010
@@ -30,6 +30,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.FuzzyTermsEnum;
 import org.apache.lucene.search.MultiTermQuery;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
 
@@ -387,7 +388,10 @@ public class DirectSpellChecker {
   private Collection<ScoreTerm> suggestSimilar(Term term, int numSug, 
       IndexReader ir, int docfreq, int editDistance, float accuracy) throws IOException {
     
-    FuzzyTermsEnum e = new FuzzyTermsEnum(ir, term, editDistance, Math.max(minPrefix, editDistance-1));
+    AttributeSource atts = new AttributeSource();
+    MultiTermQuery.MaxNonCompetitiveBoostAttribute maxBoostAtt =
+      atts.addAttribute(MultiTermQuery.MaxNonCompetitiveBoostAttribute.class);
+    FuzzyTermsEnum e = new FuzzyTermsEnum(ir, atts, term, editDistance, Math.max(minPrefix, editDistance-1));
     final PriorityQueue<ScoreTerm> stQueue = new PriorityQueue<ScoreTerm>();
     
     BytesRef queryTerm = new BytesRef(term.text());
@@ -435,7 +439,7 @@ public class DirectSpellChecker {
       stQueue.offer(st);
       // possibly drop entries from queue
       st = (stQueue.size() > numSug) ? stQueue.poll() : new ScoreTerm();
-      boostAtt.setMaxNonCompetitiveBoost((stQueue.size() >= numSug) ? stQueue.peek().boost : Float.NEGATIVE_INFINITY);
+      maxBoostAtt.setMaxNonCompetitiveBoost((stQueue.size() >= numSug) ? stQueue.peek().boost : Float.NEGATIVE_INFINITY);
     }
       
     return stQueue;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java Fri Oct 15 14:25:48 2010
@@ -24,6 +24,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.util.ToStringUtils;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
@@ -85,7 +86,7 @@ public class AutomatonQuery extends Mult
   }
 
   @Override
-  protected TermsEnum getTermsEnum(IndexReader reader) throws IOException {
+  protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException {
     // matches nothing
     if (BasicOperations.isEmpty(automaton)) {
       return TermsEnum.EMPTY;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java Fri Oct 15 14:25:48 2010
@@ -20,6 +20,7 @@ package org.apache.lucene.search;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
 
@@ -135,11 +136,11 @@ public class FuzzyQuery extends MultiTer
   }
 
   @Override
-  protected TermsEnum getTermsEnum(IndexReader reader) throws IOException {
+  protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException {
     if (!termLongEnough) {  // can only match if it's exact
       return new SingleTermsEnum(reader, term);
     }
-    return new FuzzyTermsEnum(reader, getTerm(), minimumSimilarity, prefixLength);
+    return new FuzzyTermsEnum(reader, atts, getTerm(), minimumSimilarity, prefixLength);
   }
   
   /**

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Fri Oct 15 14:25:48 2010
@@ -22,6 +22,7 @@ import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
@@ -51,7 +52,12 @@ public final class FuzzyTermsEnum extend
   private final MultiTermQuery.BoostAttribute boostAtt =
     attributes().addAttribute(MultiTermQuery.BoostAttribute.class);
   
-  private float bottom = boostAtt.getMaxNonCompetitiveBoost();
+  private final MultiTermQuery.MaxNonCompetitiveBoostAttribute maxBoostAtt;
+  
+  private float bottom;
+  private BytesRef bottomTerm;
+  // nocommit: chicken-and-egg
+  private final Comparator<BytesRef> termComparator = BytesRef.getUTF8SortedAsUnicodeComparator();
   
   private final float minSimilarity;
   private final float scale_factor;
@@ -82,7 +88,7 @@ public final class FuzzyTermsEnum extend
    * @param prefixLength Length of required common prefix. Default value is 0.
    * @throws IOException
    */
-  public FuzzyTermsEnum(IndexReader reader, Term term, 
+  public FuzzyTermsEnum(IndexReader reader, AttributeSource atts, Term term, 
       final float minSimilarity, final int prefixLength) throws IOException {
     if (minSimilarity >= 1.0f && minSimilarity != (int)minSimilarity)
       throw new IllegalArgumentException("fractional edit distances are not allowed");
@@ -116,9 +122,10 @@ public final class FuzzyTermsEnum extend
     }
     this.scale_factor = 1.0f / (1.0f - this.minSimilarity);
 
-    TermsEnum subEnum = getAutomatonEnum(maxEdits, null);
-    setEnum(subEnum != null ? subEnum : 
-      new LinearFuzzyTermsEnum());
+    this.maxBoostAtt = atts.addAttribute(MultiTermQuery.MaxNonCompetitiveBoostAttribute.class);
+    bottom = maxBoostAtt.getMaxNonCompetitiveBoost();
+    bottomTerm = maxBoostAtt.getCompetitiveTerm();
+    bottomChanged(null, true);
   }
   
   /**
@@ -169,19 +176,24 @@ public final class FuzzyTermsEnum extend
    * fired when the max non-competitive boost has changed. this is the hook to
    * swap in a smarter actualEnum
    */
-  private void bottomChanged(float boostValue, BytesRef lastTerm)
+  private void bottomChanged(BytesRef lastTerm, boolean init)
       throws IOException {
     int oldMaxEdits = maxEdits;
     
+    // true if the last term encountered is lexicographically equal or after the bottom term in the PQ
+    boolean termAfter = bottomTerm == null || (lastTerm != null && termComparator.compare(lastTerm, bottomTerm) >= 0);
+
     // as long as the max non-competitive boost is >= the max boost
     // for some edit distance, keep dropping the max edit distance.
-    while (maxEdits > 0 && boostValue >= calculateMaxBoost(maxEdits))
+    while (maxEdits > 0 && (termAfter ? bottom >= calculateMaxBoost(maxEdits) : bottom > calculateMaxBoost(maxEdits)))
       maxEdits--;
     
-    if (oldMaxEdits != maxEdits) { // the maximum n has changed
+    if (oldMaxEdits != maxEdits || init) { // the maximum n has changed
       TermsEnum newEnum = getAutomatonEnum(maxEdits, lastTerm);
       if (newEnum != null) {
         setEnum(newEnum);
+      } else if (init) {
+        setEnum(new LinearFuzzyTermsEnum());      
       }
     }
   }
@@ -202,16 +214,18 @@ public final class FuzzyTermsEnum extend
   @Override
   public BytesRef next() throws IOException {
     if (queuedBottom != null) {
-      bottomChanged(bottom, queuedBottom);
+      bottomChanged(queuedBottom, false);
       queuedBottom = null;
     }
     
     BytesRef term = actualEnum.next();
     boostAtt.setBoost(actualBoostAtt.getBoost());
     
-    final float bottom = boostAtt.getMaxNonCompetitiveBoost();
-    if (bottom != this.bottom && term != null) {
+    final float bottom = maxBoostAtt.getMaxNonCompetitiveBoost();
+    final BytesRef bottomTerm = maxBoostAtt.getCompetitiveTerm();
+    if (term != null && (bottom != this.bottom || bottomTerm != this.bottomTerm)) {
       this.bottom = bottom;
+      this.bottomTerm = bottomTerm;
       // clone the term before potentially doing something with it
       // this is a rare but wonderful occurrence anyway
       queuedBottom = new BytesRef(term);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java Fri Oct 15 14:25:48 2010
@@ -19,19 +19,30 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.PriorityQueue;
+import java.util.Comparator;
 
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.MultiFields;
-import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.Terms;
-import org.apache.lucene.queryParser.QueryParser; // for javadoc
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.queryParser.QueryParser;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeImpl;
-import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefHash;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.ReaderUtil;
+import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
 
 /**
  * An abstract {@link Query} that matches documents
@@ -39,7 +50,7 @@ import org.apache.lucene.util.PagedBytes
  * FilteredTermsEnum} enumeration.
  *
  * <p>This query cannot be used directly; you must subclass
- * it and define {@link #getTermsEnum} to provide a {@link
+ * it and define {@link #getTermsEnum(IndexReader,AttributeSource)} to provide a {@link
  * FilteredTermsEnum} that iterates through the terms to be
  * matched.
  *
@@ -71,34 +82,25 @@ public abstract class MultiTermQuery ext
   protected RewriteMethod rewriteMethod = CONSTANT_SCORE_AUTO_REWRITE_DEFAULT;
   transient int numberOfTerms = 0;
   
-  /** Add this {@link Attribute} to a {@link TermsEnum} returned by {@link #getTermsEnum}
+  /** Add this {@link Attribute} to a {@link TermsEnum} returned by {@link #getTermsEnum(IndexReader,AttributeSource)}
    * and update the boost on each returned term. This enables to control the boost factor
    * for each matching term in {@link #SCORING_BOOLEAN_QUERY_REWRITE} or
    * {@link TopTermsBooleanQueryRewrite} mode.
    * {@link FuzzyQuery} is using this to take the edit distance into account.
+   * <p><b>Please note:</b> This attribute is intended to be added only by the TermsEnum
+   * to itsself in its constructor and consumed by the {@link RewriteMethod}.
+   * @lucene.internal
    */
   public static interface BoostAttribute extends Attribute {
     /** Sets the boost in this attribute */
     public void setBoost(float boost);
     /** Retrieves the boost, default is {@code 1.0f}. */
     public float getBoost();
-    /** Sets the maximum boost for terms that would never get
-     * into the priority queue of {@link MultiTermQuery.TopTermsBooleanQueryRewrite}.
-     * This value is not changed by {@link AttributeImpl#clear}
-     * and not used in {@code equals()} and {@code hashCode()}.
-     * Do not change the value in the {@link TermsEnum}!
-     */
-    public void setMaxNonCompetitiveBoost(float maxNonCompetitiveBoost);
-    /** Retrieves the maximum boost that is not competitive,
-     * default is megative infinity. You can use this boost value
-     * as a hint when writing the {@link TermsEnum}.
-     */
-    public float getMaxNonCompetitiveBoost();
   }
 
   /** Implementation class for {@link BoostAttribute}. */
   public static final class BoostAttributeImpl extends AttributeImpl implements BoostAttribute {
-    private float boost = 1.0f, maxNonCompetitiveBoost = Float.NEGATIVE_INFINITY;
+    private float boost = 1.0f;
   
     public void setBoost(float boost) {
       this.boost = boost;
@@ -107,8 +109,61 @@ public abstract class MultiTermQuery ext
     public float getBoost() {
       return boost;
     }
+
+    @Override
+    public void clear() {
+      boost = 1.0f;
+    }
+
+    @Override
+    public boolean equals(Object other) {
+      if (this == other)
+        return true;
+      if (other instanceof BoostAttributeImpl)
+        return ((BoostAttributeImpl) other).boost == boost;
+      return false;
+    }
+
+    @Override
+    public int hashCode() {
+      return Float.floatToIntBits(boost);
+    }
+    
+    @Override
+    public void copyTo(AttributeImpl target) {
+      ((BoostAttribute) target).setBoost(boost);
+    }
+  }
+
+  /** Add this {@link Attribute} to a fresh {@link AttributeSource} before calling
+   * {@link #getTermsEnum(IndexReader,AttributeSource)}.
+   * {@link FuzzyQuery} is using this to control its internal behaviour
+   * to only return competitive terms.
+   * <p><b>Please note:</b> This attribute is intended to be added by the {@link RewriteMethod}
+   * to an empty {@link AttributeSource} that is shared for all segments
+   * during query rewrite. This attribute source is passed to all segment enums
+   * on {@link #getTermsEnum(IndexReader,AttributeSource)}.
+   * {@link TopTermsBooleanQueryRewrite} uses this attribute to
+   * inform all enums about the current boost, that is not competitive.
+   * @lucene.internal
+   */
+  public static interface MaxNonCompetitiveBoostAttribute extends Attribute {
+    /** This is the maximum boost that would not be competitive. */
+    public void setMaxNonCompetitiveBoost(float maxNonCompetitiveBoost);
+    /** This is the maximum boost that would not be competitive. Default is negative infinity, which means every term is competitive. */
+    public float getMaxNonCompetitiveBoost();
+    /** This is the term or <code>null<code> of the term that triggered the boost change. */
+    public void setCompetitiveTerm(BytesRef competitiveTerm);
+    /** This is the term or <code>null<code> of the term that triggered the boost change. Default is <code>null</code>, which means every term is competitoive. */
+    public BytesRef getCompetitiveTerm();
+  }
+
+  /** Implementation class for {@link MaxNonCompetitiveBoostAttribute}. */
+  public static final class MaxNonCompetitiveBoostAttributeImpl extends AttributeImpl implements MaxNonCompetitiveBoostAttribute {
+    private float maxNonCompetitiveBoost = Float.NEGATIVE_INFINITY;
+    private BytesRef competitiveTerm = null;
   
-    public void setMaxNonCompetitiveBoost(float maxNonCompetitiveBoost) {
+    public void setMaxNonCompetitiveBoost(final float maxNonCompetitiveBoost) {
       this.maxNonCompetitiveBoost = maxNonCompetitiveBoost;
     }
     
@@ -116,28 +171,44 @@ public abstract class MultiTermQuery ext
       return maxNonCompetitiveBoost;
     }
 
+    public void setCompetitiveTerm(final BytesRef competitiveTerm) {
+      this.competitiveTerm = competitiveTerm;
+    }
+    
+    public BytesRef getCompetitiveTerm() {
+      return competitiveTerm;
+    }
+
     @Override
     public void clear() {
-      boost = 1.0f;
+      maxNonCompetitiveBoost = Float.NEGATIVE_INFINITY;
+      competitiveTerm = null;
     }
 
     @Override
     public boolean equals(Object other) {
       if (this == other)
         return true;
-      if (other instanceof BoostAttributeImpl)
-        return ((BoostAttributeImpl) other).boost == boost;
+      if (other instanceof MaxNonCompetitiveBoostAttributeImpl) {
+        final MaxNonCompetitiveBoostAttributeImpl o = (MaxNonCompetitiveBoostAttributeImpl) other;
+        return (o.maxNonCompetitiveBoost == maxNonCompetitiveBoost)
+          && (o.competitiveTerm == null ? competitiveTerm == null : o.competitiveTerm.equals(competitiveTerm));
+      }
       return false;
     }
 
     @Override
     public int hashCode() {
-      return Float.floatToIntBits(boost);
+      int hash = Float.floatToIntBits(maxNonCompetitiveBoost);
+      if (competitiveTerm != null) hash = 31 * hash + competitiveTerm.hashCode();
+      return hash;
     }
     
     @Override
     public void copyTo(AttributeImpl target) {
-      ((BoostAttribute) target).setBoost(boost);
+      final MaxNonCompetitiveBoostAttributeImpl t = (MaxNonCompetitiveBoostAttributeImpl) target;
+      t.setMaxNonCompetitiveBoost(maxNonCompetitiveBoost);
+      t.setCompetitiveTerm(competitiveTerm);
     }
   }
 
@@ -177,69 +248,85 @@ public abstract class MultiTermQuery ext
   private abstract static class BooleanQueryRewrite extends RewriteMethod {
   
     protected final int collectTerms(IndexReader reader, MultiTermQuery query, TermCollector collector) throws IOException {
-      final Fields fields = MultiFields.getFields(reader);
-      if (fields == null) {
-        // reader has no fields
-        return 0;
-      }
-
-      final Terms terms = fields.terms(query.field);
-      if (terms == null) {
-        // field does not exist
-        return 0;
-      }
-
-      final TermsEnum termsEnum = query.getTermsEnum(reader);
-      assert termsEnum != null;
-
-      if (termsEnum == TermsEnum.EMPTY)
-        return 0;
-      final BoostAttribute boostAtt =
-        termsEnum.attributes().addAttribute(BoostAttribute.class);
-      collector.boostAtt = boostAtt;
+      final List<IndexReader> subReaders = new ArrayList<IndexReader>();
+      ReaderUtil.gatherSubReaders(subReaders, reader);
       int count = 0;
-      BytesRef bytes;
-      while ((bytes = termsEnum.next()) != null) {
-        if (collector.collect(termsEnum, bytes, boostAtt.getBoost())) {
-          termsEnum.cacheCurrentTerm();
-          count++;
-        } else {
-          break;
+      Comparator<BytesRef> lastTermComp = null;
+      
+      for (IndexReader r : subReaders) {
+        final Fields fields = r.fields();
+        if (fields == null) {
+          // reader has no fields
+          continue;
+        }
+
+        final Terms terms = fields.terms(query.field);
+        if (terms == null) {
+          // field does not exist
+          continue;
+        }
+
+        final TermsEnum termsEnum = query.getTermsEnum(r, collector.attributes);
+        assert termsEnum != null;
+
+        if (termsEnum == TermsEnum.EMPTY)
+          continue;
+        
+        // Check comparator compatibility:
+        final Comparator<BytesRef> newTermComp = termsEnum.getComparator();
+        if (lastTermComp != null && newTermComp != lastTermComp)
+          throw new RuntimeException("term comparator should not change between segments: "+lastTermComp+" != "+newTermComp);
+        lastTermComp = newTermComp;
+        
+        collector.setNextEnum(termsEnum);
+        BytesRef bytes;
+        while ((bytes = termsEnum.next()) != null) {
+          if (collector.collect(bytes)) {
+            termsEnum.cacheCurrentTerm();
+            count++;
+          } else {
+            return count; // interrupt whole term collection, so also don't iterate other subReaders
+          }
         }
       }
-      collector.boostAtt = null;
       return count;
     }
     
     protected static abstract class TermCollector {
-      private BoostAttribute boostAtt = null;
+      /** attributes used for communication with the enum */
+      public final AttributeSource attributes = new AttributeSource();
     
       /** return false to stop collecting */
-      public abstract boolean collect(TermsEnum termsEnum, BytesRef bytes, float boost) throws IOException;
+      public abstract boolean collect(BytesRef bytes) throws IOException;
       
-      /** set the minimum boost as a hint for the term producer */
-      protected final void setMaxNonCompetitiveBoost(float maxNonCompetitiveBoost) {
-        assert boostAtt != null;
-        boostAtt.setMaxNonCompetitiveBoost(maxNonCompetitiveBoost);
-      }
+      /** the next segment's {@link TermsEnum} that is used to collect terms */
+      public abstract void setNextEnum(TermsEnum termsEnum) throws IOException;
     }
   }
   
   private static class ScoringBooleanQueryRewrite extends BooleanQueryRewrite {
     @Override
     public Query rewrite(final IndexReader reader, final MultiTermQuery query) throws IOException {
-      final BooleanQuery result = new BooleanQuery(true);
+      final ParallelArraysTermCollector col = new ParallelArraysTermCollector();
+      collectTerms(reader, query, col);
+      
       final Term placeholderTerm = new Term(query.field);
-      query.incTotalNumberOfTerms(collectTerms(reader, query, new TermCollector() {
-        @Override
-        public boolean collect(TermsEnum termsEnum, BytesRef bytes, float boost) {
-          // add new TQ, we must clone the term, else it may get overwritten!
-          TermQuery tq = new TermQuery(placeholderTerm.createTerm(new BytesRef(bytes)), termsEnum.docFreq());
-          tq.setBoost(query.getBoost() * boost); // set the boost
-          result.add(tq, BooleanClause.Occur.SHOULD); // add to query
-          return true;
+      final BooleanQuery result = new BooleanQuery(true);
+      final int size = col.terms.size();
+      if (size > 0) {
+        final int sort[] = col.terms.sort(col.termsEnum.getComparator());
+        final int[] docFreq = col.array.docFreq;
+        final float[] boost = col.array.boost;
+        for (int i = 0; i < size; i++) {
+          final int pos = sort[i];
+          final Term term = placeholderTerm.createTerm(col.terms.get(pos, new BytesRef()));
+          assert reader.docFreq(term) == docFreq[pos];
+          final TermQuery tq = new TermQuery(term, docFreq[pos]);
+          tq.setBoost(query.getBoost() * boost[pos]);
+          result.add(tq, BooleanClause.Occur.SHOULD);
         }
-      }));
+      }
+      query.incTotalNumberOfTerms(size);
       return result;
     }
 
@@ -247,6 +334,75 @@ public abstract class MultiTermQuery ext
     protected Object readResolve() {
       return SCORING_BOOLEAN_QUERY_REWRITE;
     }
+    
+    static final class ParallelArraysTermCollector extends TermCollector {
+      final TermFreqBoostByteStart array = new TermFreqBoostByteStart(16);
+      final BytesRefHash terms = new BytesRefHash(new ByteBlockPool(new ByteBlockPool.DirectAllocator()), 16, array);
+      TermsEnum termsEnum;
+
+      private BoostAttribute boostAtt;
+    
+      @Override
+      public void setNextEnum(TermsEnum termsEnum) throws IOException {
+        this.termsEnum = termsEnum;
+        this.boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class);
+      }
+    
+      @Override
+      public boolean collect(BytesRef bytes) {
+        final int e = terms.add(bytes);
+        if (e < 0 ) {
+          // duplicate term: update docFreq
+          final int pos = (-e)-1;
+          array.docFreq[pos] += termsEnum.docFreq();
+          assert array.boost[pos] == boostAtt.getBoost() : "boost should be equal in all segment TermsEnums";
+        } else {
+          // new entry: we populate the entry initially
+          array.docFreq[e] = termsEnum.docFreq();
+          array.boost[e] = boostAtt.getBoost();
+        }
+        // if the new entry reaches the max clause count, we exit early
+        if (e >= BooleanQuery.getMaxClauseCount())
+          throw new BooleanQuery.TooManyClauses();
+        return true;
+      }
+    }
+    
+    /** Special implementation of BytesStartArray that keeps parallel arrays for boost and docFreq */
+    static final class TermFreqBoostByteStart extends DirectBytesStartArray  {
+      int[] docFreq;
+      float[] boost;
+      
+      public TermFreqBoostByteStart(int initSize) {
+        super(initSize);
+      }
+
+      @Override
+      public int[] init() {
+        final int[] ord = super.init();
+        boost = new float[ArrayUtil.oversize(ord.length, RamUsageEstimator.NUM_BYTES_FLOAT)];
+        docFreq = new int[ArrayUtil.oversize(ord.length, RamUsageEstimator.NUM_BYTES_INT)];
+        assert boost.length >= ord.length && docFreq.length >= ord.length;
+        return ord;
+      }
+
+      @Override
+      public int[] grow() {
+        final int[] ord = super.grow();
+        docFreq = ArrayUtil.grow(docFreq, ord.length);
+        boost = ArrayUtil.grow(boost, ord.length);
+        assert boost.length >= ord.length && docFreq.length >= ord.length;
+        return ord;
+      }
+
+      @Override
+      public int[] clear() {
+       boost = null;
+       docFreq = null;
+       return super.clear();
+      }
+      
+    }
   }
 
   /** A rewrite method that first translates each term into
@@ -291,44 +447,92 @@ public abstract class MultiTermQuery ext
       final int maxSize = Math.min(size, BooleanQuery.getMaxClauseCount());
       final PriorityQueue<ScoreTerm> stQueue = new PriorityQueue<ScoreTerm>();
       collectTerms(reader, query, new TermCollector() {
+        private final MaxNonCompetitiveBoostAttribute maxBoostAtt =
+          attributes.addAttribute(MaxNonCompetitiveBoostAttribute.class);
+        
+        private final Map<BytesRef,ScoreTerm> visitedTerms = new HashMap<BytesRef,ScoreTerm>();
+        
+        private TermsEnum termsEnum;
+        private Comparator<BytesRef> termComp;
+        private BoostAttribute boostAtt;        
+        private ScoreTerm st;
+        
         @Override
-        public boolean collect(TermsEnum termsEnum, BytesRef bytes, float boost) {
+        public void setNextEnum(TermsEnum termsEnum) throws IOException {
+          this.termsEnum = termsEnum;
+          this.termComp = termsEnum.getComparator();
+          // lazy init the initial ScoreTerm because comparator is not known on ctor:
+          if (st == null)
+            st = new ScoreTerm(this.termComp);
+          boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class);
+        }
+      
+        @Override
+        public boolean collect(BytesRef bytes) {
+          final float boost = boostAtt.getBoost();
           // ignore uncompetetive hits
-          if (stQueue.size() >= maxSize && boost <= stQueue.peek().boost)
-            return true;
-          // add new entry in PQ, we must clone the term, else it may get overwritten!
-          st.bytes.copy(bytes);
-          st.boost = boost;
-          st.docFreq = termsEnum.docFreq();
-          stQueue.offer(st);
-          // possibly drop entries from queue
-          st = (stQueue.size() > maxSize) ? stQueue.poll() : new ScoreTerm();
-          setMaxNonCompetitiveBoost((stQueue.size() >= maxSize) ? stQueue.peek().boost : Float.NEGATIVE_INFINITY);
+          if (stQueue.size() == maxSize) {
+            final ScoreTerm t = stQueue.peek();
+            if (boost < t.boost)
+              return true;
+            if (boost == t.boost && termComp.compare(bytes, t.bytes) > 0)
+              return true;
+          }
+          ScoreTerm t = visitedTerms.get(bytes);
+          if (t != null) {
+            // if the term is already in the PQ, only update docFreq of term in PQ
+            t.docFreq += termsEnum.docFreq();
+            assert t.boost == boost : "boost should be equal in all segment TermsEnums";
+          } else {
+            // add new entry in PQ, we must clone the term, else it may get overwritten!
+            st.bytes.copy(bytes);
+            st.boost = boost;
+            st.docFreq = termsEnum.docFreq();
+            visitedTerms.put(st.bytes, st);
+            stQueue.offer(st);
+            // possibly drop entries from queue
+            if (stQueue.size() > maxSize) {
+              st = stQueue.poll();
+              visitedTerms.remove(st.bytes);
+            } else {
+              st = new ScoreTerm(termComp);
+            }
+            assert stQueue.size() <= maxSize : "the PQ size must be limited to maxSize";
+            // set maxBoostAtt with values to help FuzzyTermsEnum to optimize
+            if (stQueue.size() == maxSize) {
+              t = stQueue.peek();
+              maxBoostAtt.setMaxNonCompetitiveBoost(t.boost);
+              maxBoostAtt.setCompetitiveTerm(t.bytes);
+            }
+          }
           return true;
         }
-        
-        // reusable instance
-        private ScoreTerm st = new ScoreTerm();
       });
       
       final Term placeholderTerm = new Term(query.field);
       final BooleanQuery bq = new BooleanQuery(true);
-      for (final ScoreTerm st : stQueue) {
-        // add new query, we must clone the term, else it may get overwritten!
-        Query tq = getQuery(placeholderTerm.createTerm(st.bytes), st.docFreq);
+      final ScoreTerm[] scoreTerms = stQueue.toArray(new ScoreTerm[stQueue.size()]);
+      Arrays.sort(scoreTerms, new Comparator<ScoreTerm>() {
+        public int compare(ScoreTerm st1, ScoreTerm st2) {
+          assert st1.termComp == st2.termComp :
+            "term comparator should not change between segments";
+          return st1.termComp.compare(st1.bytes, st2.bytes);
+        }
+      });
+      for (final ScoreTerm st : scoreTerms) {
+        final Term term = placeholderTerm.createTerm(st.bytes);
+        assert reader.docFreq(term) == st.docFreq;
+        Query tq = getQuery(term, st.docFreq);
         tq.setBoost(query.getBoost() * st.boost); // set the boost
         bq.add(tq, BooleanClause.Occur.SHOULD);   // add to query
       }
-      query.incTotalNumberOfTerms(bq.clauses().size());
+      query.incTotalNumberOfTerms(scoreTerms.length);
       return bq;
     }
   
     @Override
     public int hashCode() {
-      final int prime = 17;
-      int result = 1;
-      result = prime * result + size;
-      return result;
+      return 31 * size;
     }
 
     @Override
@@ -341,15 +545,20 @@ public abstract class MultiTermQuery ext
       return true;
     }
   
-    private static class ScoreTerm implements Comparable<ScoreTerm> {
+    static final class ScoreTerm implements Comparable<ScoreTerm> {
+      public final Comparator<BytesRef> termComp;
+
       public final BytesRef bytes = new BytesRef();
       public float boost;
       public int docFreq;
       
+      public ScoreTerm(Comparator<BytesRef> termComp) {
+        this.termComp = termComp;
+      }
+      
       public int compareTo(ScoreTerm other) {
         if (this.boost == other.boost)
-          // TODO: is it OK to use default compare here?
-          return other.bytes.compareTo(this.bytes);
+          return termComp.compare(other.bytes, this.bytes);
         else
           return Float.compare(this.boost, other.boost);
       }
@@ -362,8 +571,8 @@ public abstract class MultiTermQuery ext
    * scores as computed by the query.
    * 
    * <p>
-   * This rewrite mode only uses the top scoring terms so it will not overflow
-   * the boolean max clause count. It is the default rewrite mode for
+   * This rewrite method only uses the top scoring terms so it will not overflow
+   * the boolean max clause count. It is the default rewrite method for
    * {@link FuzzyQuery}.
    * 
    * @see #setRewriteMethod
@@ -510,63 +719,61 @@ public abstract class MultiTermQuery ext
       final int docCountCutoff = (int) ((docCountPercent / 100.) * reader.maxDoc());
       final int termCountLimit = Math.min(BooleanQuery.getMaxClauseCount(), termCountCutoff);
 
-      final CutOffTermCollector col = new CutOffTermCollector(reader, query.field, docCountCutoff, termCountLimit);
+      final CutOffTermCollector col = new CutOffTermCollector(docCountCutoff, termCountLimit);
       collectTerms(reader, query, col);
-      
+      final int size = col.pendingTerms.size();
       if (col.hasCutOff) {
         return CONSTANT_SCORE_FILTER_REWRITE.rewrite(reader, query);
-      } else if (col.termCount == 0) {
+      } else if (size == 0) {
         return new BooleanQuery(true);
       } else {
-        final PagedBytes.Reader bytesReader = col.pendingTerms.freeze(false);
-        try {
-          final BooleanQuery bq = new BooleanQuery(true);
-          final Term placeholderTerm = new Term(query.field);
-          long start = col.startOffset;
-          for(int i = 0; i < col.termCount; i++) {
-            final BytesRef bytes = new BytesRef();
-            start = bytesReader.fillUsingLengthPrefix3(bytes, start);
-            bq.add(new TermQuery(placeholderTerm.createTerm(bytes)), BooleanClause.Occur.SHOULD);
-          }
-          // Strip scores
-          final Query result = new ConstantScoreQuery(new QueryWrapperFilter(bq));
-          result.setBoost(query.getBoost());
-          query.incTotalNumberOfTerms(col.termCount);
-          return result;
-        } finally {
-          bytesReader.close();
+        final BooleanQuery bq = new BooleanQuery(true);
+        final Term placeholderTerm = new Term(query.field);
+        final BytesRefHash pendingTerms = col.pendingTerms;
+        final int sort[] = pendingTerms.sort(col.termsEnum.getComparator());
+        for(int i = 0; i < size; i++) {
+          // docFreq is not used for constant score here, we pass 1
+          // to explicitely set a fake value, so it's not calculated
+          bq.add(new TermQuery(
+            placeholderTerm.createTerm(pendingTerms.get(sort[i], new BytesRef())), 1
+          ), BooleanClause.Occur.SHOULD);
         }
+        // Strip scores
+        final Query result = new ConstantScoreQuery(new QueryWrapperFilter(bq));
+        result.setBoost(query.getBoost());
+        query.incTotalNumberOfTerms(size);
+        return result;
       }
     }
     
-    private static final class CutOffTermCollector extends TermCollector {
-      CutOffTermCollector(IndexReader reader, String field, int docCountCutoff, int termCountLimit) {
-        this.reader = reader;
-        this.field = field;
+    static final class CutOffTermCollector extends TermCollector {
+      CutOffTermCollector(int docCountCutoff, int termCountLimit) {
         this.docCountCutoff = docCountCutoff;
         this.termCountLimit = termCountLimit;
       }
     
-      public boolean collect(TermsEnum termsEnum, BytesRef bytes, float boost) throws IOException {
-        termCount++;
-        if (termCount >= termCountLimit || docVisitCount >= docCountCutoff) {
+      @Override
+      public void setNextEnum(TermsEnum termsEnum) throws IOException {
+        this.termsEnum = termsEnum;
+      }
+        
+      @Override
+      public boolean collect(BytesRef bytes) throws IOException {
+        if (pendingTerms.size() >= termCountLimit || docVisitCount >= docCountCutoff) {
           hasCutOff = true;
           return false;
         }
-        pendingTerms.copyUsingLengthPrefix(bytes);
+        pendingTerms.add(bytes);
         docVisitCount += termsEnum.docFreq();
         return true;
       }
       
       int docVisitCount = 0;
       boolean hasCutOff = false;
-      int termCount = 0;
-      
-      final IndexReader reader;
-      final String field;
+      TermsEnum termsEnum;
+
       final int docCountCutoff, termCountLimit;
-      final PagedBytes pendingTerms = new PagedBytes(15); // max term size is 32 KiB
-      final long startOffset = pendingTerms.getPointer();
+      final BytesRefHash pendingTerms = new BytesRefHash();
     }
 
     @Override
@@ -644,8 +851,20 @@ public abstract class MultiTermQuery ext
    *  field does exist).  This method should not return null
    *  (should instead return {@link TermsEnum#EMPTY} if no
    *  terms match).  The TermsEnum must already be
-   *  positioned to the first matching term. */
-  protected abstract TermsEnum getTermsEnum(IndexReader reader) throws IOException;
+   *  positioned to the first matching term.
+   * The given {@link AttributeSource} is passed by the {@link RewriteMethod} to
+   * provide attributes, the rewrite method uses to inform about e.g. maximum competitive boosts.
+   * This is currently only used by {@link TopTermsBooleanQueryRewrite}
+   */
+  protected abstract TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException;
+
+  /** Convenience method, if no attributes are needed:
+   * This simply passes empty attributes and is equal to:
+   * <code>getTermsEnum(reader, new AttributeSource())</code>
+   */
+  protected final TermsEnum getTermsEnum(IndexReader reader) throws IOException {
+    return getTermsEnum(reader, new AttributeSource());
+  }
 
   /**
    * Expert: Return the number of unique terms visited during execution of the query.

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java Fri Oct 15 14:25:48 2010
@@ -26,6 +26,7 @@ import org.apache.lucene.document.Numeri
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.TermsEnum;
 
@@ -301,7 +302,7 @@ public final class NumericRangeQuery<T e
   }
 
   @Override @SuppressWarnings("unchecked")
-  protected TermsEnum getTermsEnum(final IndexReader reader) throws IOException {
+  protected TermsEnum getTermsEnum(final IndexReader reader, AttributeSource atts) throws IOException {
     // very strange: java.lang.Number itsself is not Comparable, but all subclasses used here are
     return (min != null && max != null && ((Comparable<T>) min).compareTo(max) > 0) ?
       TermsEnum.EMPTY :

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixQuery.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixQuery.java Fri Oct 15 14:25:48 2010
@@ -24,6 +24,7 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.ToStringUtils;
 
 /** A Query that matches documents containing terms with a specified prefix. A PrefixQuery
@@ -45,7 +46,7 @@ public class PrefixQuery extends MultiTe
   public Term getPrefix() { return prefix; }
   
   @Override  
-  protected TermsEnum getTermsEnum(IndexReader reader) throws IOException {
+  protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException {
     if (prefix.bytes().length == 0) {
       // no prefix -- match all terms for this field:
       // NOTE: for now, MultiTermQuery enums terms at the

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java Fri Oct 15 14:25:48 2010
@@ -33,7 +33,7 @@ import org.apache.lucene.index.IndexRead
  * once per day.
  */
 public class QueryWrapperFilter extends Filter {
-  private Query query;
+  private final Query query;
 
   /** Constructs a filter which only matches documents matching
    * <code>query</code>.
@@ -41,6 +41,11 @@ public class QueryWrapperFilter extends 
   public QueryWrapperFilter(Query query) {
     this.query = query;
   }
+  
+  /** returns the inner Query */
+  public final Query getQuery() {
+    return query;
+  }
 
   @Override
   public DocIdSet getDocIdSet(final IndexReader reader) throws IOException {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java Fri Oct 15 14:25:48 2010
@@ -25,6 +25,7 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.util.ToStringUtils;
+import org.apache.lucene.util.AttributeSource;
 
 /**
  * A Query that matches documents within an range of terms.
@@ -130,7 +131,7 @@ public class TermRangeQuery extends Mult
   public Collator getCollator() { return collator; }
   
   @Override
-  protected TermsEnum getTermsEnum(IndexReader reader) throws IOException {
+  protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException {
     if (collator == null && lowerTerm != null && upperTerm != null && lowerTerm.compareTo(upperTerm) > 0) {
       return TermsEnum.EMPTY;
     }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java Fri Oct 15 14:25:48 2010
@@ -27,14 +27,7 @@ import java.util.Collection;
 
 public final class ArrayUtil {
 
-  /**
-   * @deprecated This constructor was not intended to be public and should not be used.
-   *  This class contains solely a static utility methods.
-   *  It will be made private in Lucene 4.0
-   */
-  // make private in 4.0!
-  @Deprecated
-  public ArrayUtil() {} // no instance
+  private ArrayUtil() {} // no instance
 
   /*
      Begin Apache Harmony code
@@ -247,6 +240,19 @@ public final class ArrayUtil {
   public static short[] grow(short[] array) {
     return grow(array, 1 + array.length);
   }
+  
+  public static float[] grow(float[] array, int minSize) {
+    if (array.length < minSize) {
+      float[] newArray = new float[oversize(minSize, RamUsageEstimator.NUM_BYTES_FLOAT)];
+      System.arraycopy(array, 0, newArray, 0, array.length);
+      return newArray;
+    } else
+      return array;
+  }
+
+  public static float[] grow(float[] array) {
+    return grow(array, 1 + array.length);
+  }
 
   public static short[] shrink(short[] array, int targetSize) {
     final int newSize = getShrinkSize(array.length, targetSize, RamUsageEstimator.NUM_BYTES_SHORT);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java Fri Oct 15 14:25:48 2010
@@ -16,8 +16,12 @@ package org.apache.lucene.util;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+import java.util.Arrays;
+import java.util.List;
+import static org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_OBJECT_REF;
 
-/* Class that Posting and PostingVector use to write byte
+/** 
+ * Class that Posting and PostingVector use to write byte
  * streams into shared fixed-size byte[] arrays.  The idea
  * is to allocate slices of increasing lengths For
  * example, the first slice is 5 bytes, the next slice is
@@ -31,14 +35,10 @@ package org.apache.lucene.util;
  * the end with a non-zero byte.  This way the methods
  * that are writing into the slice don't need to record
  * its length and instead allocate a new slice once they
- * hit a non-zero byte. */
-
-import java.util.Arrays;
-
-
-import java.util.List;
-import static org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_OBJECT_REF;
-
+ * hit a non-zero byte. 
+ * 
+ * @lucene.internal
+ **/
 public final class ByteBlockPool {
   public final static int BYTE_BLOCK_SHIFT = 15;
   public final static int BYTE_BLOCK_SIZE = 1 << BYTE_BLOCK_SHIFT;
@@ -62,6 +62,22 @@ public final class ByteBlockPool {
       return new byte[blockSize];
     }
   }
+  
+  public static final class DirectAllocator extends Allocator {
+    
+    public DirectAllocator() {
+      this(BYTE_BLOCK_SIZE);
+    }
+
+    public DirectAllocator(int blockSize) {
+      super(blockSize);
+    }
+
+    @Override
+    public void recycleByteBlocks(byte[][] blocks, int start, int end) {
+    }
+    
+  }
 
   public byte[][] buffers = new byte[10][];
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRefHash.java Fri Oct 15 14:25:48 2010
@@ -17,13 +17,15 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
+import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
+import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SHIFT;
+import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
+
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.concurrent.atomic.AtomicLong;
 
-import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
-import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
-import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SHIFT;
+import org.apache.lucene.util.ByteBlockPool.DirectAllocator;
 
 /**
  * {@link BytesRefHash} is a special purpose hash-map like data-structure
@@ -54,6 +56,14 @@ public final class BytesRefHash {
   public static final int DEFAULT_CAPACITY = 16;
   private final BytesStartArray bytesStartArray;
   private AtomicLong bytesUsed;
+  
+  /**
+   * Creates a new {@link BytesRefHash} with a {@link ByteBlockPool} using a
+   * {@link DirectAllocator}.
+   */
+  public BytesRefHash() { 
+    this(new ByteBlockPool(new DirectAllocator()));
+  }
 
   /**
    * Creates a new {@link BytesRefHash}
@@ -75,7 +85,7 @@ public final class BytesRefHash {
     Arrays.fill(ords, -1);
     this.bytesStartArray = bytesStartArray;
     bytesStart = bytesStartArray.init();
-    bytesUsed = bytesStartArray.bytesUsed();
+    bytesUsed = bytesStartArray.bytesUsed() == null? new AtomicLong(0) : bytesStartArray.bytesUsed();;
     bytesUsed.addAndGet(hashSize * RamUsageEstimator.NUM_BYTES_INT);
   }
 
@@ -143,7 +153,6 @@ public final class BytesRefHash {
    *          the {@link Comparator} used for sorting
    */
   public int[] sort(Comparator<BytesRef> comp) {
-    assert bytesStart != null : "Bytesstart is null - not initialized";
     final int[] compact = compact();
     quickSort(comp, compact, 0, count - 1);
     return compact;
@@ -536,13 +545,13 @@ public final class BytesRefHash {
     public abstract AtomicLong bytesUsed();
   }
 
-  static class DirectBytesStartArray extends BytesStartArray {
+  public static class DirectBytesStartArray extends BytesStartArray {
 
-    private final int initSize;
+    protected final int initSize;
     private int[] bytesStart;
     private final AtomicLong bytesUsed = new AtomicLong(0);
 
-    DirectBytesStartArray(int initSize) {
+    public DirectBytesStartArray(int initSize) {
       this.initSize = initSize;
     }
 

Added: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java?rev=1022934&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java (added)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java Fri Oct 15 14:25:48 2010
@@ -0,0 +1,186 @@
+package org.apache.lucene.search;
+
+/**
+ * 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 org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.MultiReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+public class TestMultiTermQueryRewrites extends LuceneTestCase {
+
+  static Directory dir, sdir1, sdir2;
+  static IndexReader reader, multiReader;
+  static IndexSearcher searcher, multiSearcher;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    dir = newDirectory();
+    sdir1 = newDirectory();
+    sdir2 = newDirectory();
+    final RandomIndexWriter writer = new RandomIndexWriter(random, dir, new MockAnalyzer());
+    final RandomIndexWriter swriter1 = new RandomIndexWriter(random, sdir1, new MockAnalyzer());
+    final RandomIndexWriter swriter2 = new RandomIndexWriter(random, sdir2, new MockAnalyzer());
+
+    for (int i = 0; i < 10; i++) {
+      Document doc = new Document();
+      doc.add(newField("data", Integer.toString(i), Field.Store.NO, Field.Index.NOT_ANALYZED));
+      writer.addDocument(doc);
+      ((i % 2 == 0) ? swriter1 : swriter2).addDocument(doc);
+    }
+    writer.optimize(); swriter1.optimize(); swriter2.optimize();
+    writer.close(); swriter1.close(); swriter2.close();
+    
+    reader = IndexReader.open(dir, true);
+    searcher = new IndexSearcher(reader);
+    multiReader = new MultiReader(new IndexReader[] {
+      IndexReader.open(sdir1, true), IndexReader.open(sdir2, true) 
+    }, true);
+    multiSearcher = new IndexSearcher(multiReader);
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    reader.close();
+    multiReader.close();
+    dir.close(); sdir1.close(); sdir2.close();
+    reader = multiReader = null;
+    searcher = multiSearcher = null;
+    dir = sdir1 = sdir2 = null;
+  }
+  
+  private Query extractInnerQuery(Query q) {
+    if (q instanceof ConstantScoreQuery) {
+      // wrapped as ConstantScoreQuery using QueryWrapperFilter
+      q = ((QueryWrapperFilter) ((ConstantScoreQuery) q).getFilter()).getQuery();
+    }
+    return q;
+  }
+  
+  private Term extractTerm(Query q) {
+    q = extractInnerQuery(q);
+    return ((TermQuery) q).getTerm();
+  }
+  
+  private void checkBooleanQueryOrder(Query q) {
+    q = extractInnerQuery(q);
+    final BooleanQuery bq = (BooleanQuery) q;
+    Term last = null, act;
+    for (BooleanClause clause : bq.clauses()) {
+      act = extractTerm(clause.getQuery());
+      if (last != null) {
+        assertTrue("sort order of terms in BQ violated", last.compareTo(act) < 0);
+      }
+      last = act;
+    }
+  }
+  
+  private void checkDuplicateTerms(MultiTermQuery.RewriteMethod method) throws Exception {
+    final MultiTermQuery mtq = new TermRangeQuery("data", "2", "7", true, true);
+    mtq.setRewriteMethod(method);
+    final Query q1 = searcher.rewrite(mtq);
+    final Query q2 = multiSearcher.rewrite(mtq);
+    if (VERBOSE) {
+      System.out.println();
+      System.out.println("single segment: " + q1);
+      System.out.println(" multi segment: " + q2);
+    }
+    assertEquals("The multi-segment case must produce same rewritten query", q1, q2);
+    checkBooleanQueryOrder(q1);
+    checkBooleanQueryOrder(q2);
+  }
+  
+  public void testRewritesWithDuplicateTerms() throws Exception {
+    checkDuplicateTerms(MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE);
+    
+    checkDuplicateTerms(MultiTermQuery.CONSTANT_SCORE_BOOLEAN_QUERY_REWRITE);
+    
+    // use a large PQ here to only test duplicate terms and dont mix up when all scores are equal
+    checkDuplicateTerms(new MultiTermQuery.TopTermsScoringBooleanQueryRewrite(1024));
+    checkDuplicateTerms(new MultiTermQuery.TopTermsBoostOnlyBooleanQueryRewrite(1024));
+    
+    // Test auto rewrite (but only boolean mode), so we set the limits to large values to always get a BQ
+    final MultiTermQuery.ConstantScoreAutoRewrite rewrite = new MultiTermQuery.ConstantScoreAutoRewrite();
+    rewrite.setTermCountCutoff(Integer.MAX_VALUE);
+    rewrite.setDocCountPercent(100.);
+    checkDuplicateTerms(rewrite);
+  }
+  
+  private void checkBooleanQueryBoosts(BooleanQuery bq) {
+    for (BooleanClause clause : bq.clauses()) {
+      final TermQuery mtq = (TermQuery) clause.getQuery();
+      assertEquals("Parallel sorting of boosts in rewrite mode broken",
+        Float.parseFloat(mtq.getTerm().text()), mtq.getBoost());
+    }
+  }
+  
+  private void checkBoosts(MultiTermQuery.RewriteMethod method) throws Exception {
+    final MultiTermQuery mtq = new MultiTermQuery("data") {
+      @Override
+      protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException {
+        return new TermRangeTermsEnum(reader, field, "2", "7", true, true, null) {
+          final MultiTermQuery.BoostAttribute boostAtt =
+            attributes().addAttribute(MultiTermQuery.BoostAttribute.class);
+        
+          @Override
+          protected AcceptStatus accept(BytesRef term) {
+            boostAtt.setBoost(Float.parseFloat(term.utf8ToString()));
+            return super.accept(term);
+          }
+        };
+      }
+      
+      @Override
+      public String toString(String field) {
+        return "dummy";
+      }
+    };
+    mtq.setRewriteMethod(method);
+    final Query q1 = searcher.rewrite(mtq);
+    final Query q2 = multiSearcher.rewrite(mtq);
+    if (VERBOSE) {
+      System.out.println();
+      System.out.println("single segment: " + q1);
+      System.out.println(" multi segment: " + q2);
+    }
+    assertEquals("The multi-segment case must produce same rewritten query", q1, q2);
+    checkBooleanQueryBoosts((BooleanQuery) q1);
+    checkBooleanQueryBoosts((BooleanQuery) q2);
+  }
+  
+  public void testBoosts() throws Exception {
+    checkBoosts(MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE);
+
+    // use a large PQ here to only test boosts and dont mix up when all scores are equal
+    checkBoosts(new MultiTermQuery.TopTermsScoringBooleanQueryRewrite(1024));
+  }
+  
+}

Propchange: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestPrefixRandom.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestPrefixRandom.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestPrefixRandom.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestPrefixRandom.java Fri Oct 15 14:25:48 2010
@@ -29,6 +29,7 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
@@ -85,7 +86,7 @@ public class TestPrefixRandom extends Lu
     }
     
     @Override
-    protected TermsEnum getTermsEnum(IndexReader reader) throws IOException {
+    protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException {
       return new SimplePrefixTermsEnum(reader, field, prefix);
     }
 

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java?rev=1022934&r1=1022933&r2=1022934&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java Fri Oct 15 14:25:48 2010
@@ -36,6 +36,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util._TestUtil;
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.AutomatonTestUtil;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
@@ -103,7 +104,7 @@ public class TestRegexpRandom2 extends L
     }
     
     @Override
-    protected TermsEnum getTermsEnum(IndexReader reader) throws IOException {
+    protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException {
       return new SimpleAutomatonTermsEnum(reader, field);
     }