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

svn commit: r1036080 [2/4] - in /lucene/dev/branches/docvalues: ./ lucene/ lucene/contrib/ lucene/contrib/highlighter/src/test/ lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/ lucene/contrib/misc/src/java/org/apache/lucene/in...

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java Wed Nov 17 15:43:06 2010
@@ -84,7 +84,7 @@ public class StandardCodec extends Codec
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postings = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize);
+    PostingsReaderBase postings = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize, state.codecId);
     TermsIndexReaderBase indexReader;
 
     boolean success = false;
@@ -93,7 +93,8 @@ public class StandardCodec extends Codec
                                                        state.fieldInfos,
                                                        state.segmentInfo.name,
                                                        state.termsIndexDivisor,
-                                                       BytesRef.getUTF8SortedAsUnicodeComparator());
+                                                       BytesRef.getUTF8SortedAsUnicodeComparator(),
+                                                       state.codecId);
       success = true;
     } finally {
       if (!success) {
@@ -110,7 +111,8 @@ public class StandardCodec extends Codec
                                                        postings,
                                                        state.readBufferSize,
                                                        BytesRef.getUTF8SortedAsUnicodeComparator(),
-                                                       TERMS_CACHE_SIZE);
+                                                       TERMS_CACHE_SIZE,
+                                                       state.codecId);
       success = true;
       return ret;
     } finally {
@@ -131,10 +133,10 @@ public class StandardCodec extends Codec
   static final String PROX_EXTENSION = "prx";
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) throws IOException {
-    StandardPostingsReader.files(dir, segmentInfo, files);
-    PrefixCodedTermsReader.files(dir, segmentInfo, files);
-    FixedGapTermsIndexReader.files(dir, segmentInfo, files);
+  public void files(Directory dir, SegmentInfo segmentInfo, String id, Set<String> files) throws IOException {
+    StandardPostingsReader.files(dir, segmentInfo, id, files);
+    PrefixCodedTermsReader.files(dir, segmentInfo, id, files);
+    FixedGapTermsIndexReader.files(dir, segmentInfo, id, files);
   }
 
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java Wed Nov 17 15:43:06 2010
@@ -45,13 +45,13 @@ public class StandardPostingsReader exte
   int skipInterval;
   int maxSkipLevels;
 
-  public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, int readBufferSize) throws IOException {
-    freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, "", StandardCodec.FREQ_EXTENSION),
+  public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, int readBufferSize, String codecId) throws IOException {
+    freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.FREQ_EXTENSION),
                            readBufferSize);
     if (segmentInfo.getHasProx()) {
       boolean success = false;
       try {
-        proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, "", StandardCodec.PROX_EXTENSION),
+        proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.PROX_EXTENSION),
                                readBufferSize);
         success = true;
       } finally {
@@ -64,10 +64,10 @@ public class StandardPostingsReader exte
     }
   }
 
-  public static void files(Directory dir, SegmentInfo segmentInfo, Collection<String> files) throws IOException {
-    files.add(IndexFileNames.segmentFileName(segmentInfo.name, "", StandardCodec.FREQ_EXTENSION));
+  public static void files(Directory dir, SegmentInfo segmentInfo, String id, Collection<String> files) throws IOException {
+    files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, StandardCodec.FREQ_EXTENSION));
     if (segmentInfo.getHasProx()) {
-      files.add(IndexFileNames.segmentFileName(segmentInfo.name, "", StandardCodec.PROX_EXTENSION));
+      files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, StandardCodec.PROX_EXTENSION));
     }
   }
 
@@ -175,19 +175,37 @@ public class StandardPostingsReader exte
     if (fieldInfo.omitTermFreqAndPositions) {
       return null;
     }
-    SegmentDocsAndPositionsEnum docsEnum;
-    if (reuse == null || !(reuse instanceof SegmentDocsAndPositionsEnum)) {
-      docsEnum = new SegmentDocsAndPositionsEnum(freqIn, proxIn);
+    
+    // TODO: refactor
+    if (fieldInfo.storePayloads) {
+      SegmentDocsAndPositionsAndPayloadsEnum docsEnum;
+      if (reuse == null || !(reuse instanceof SegmentDocsAndPositionsAndPayloadsEnum)) {
+        docsEnum = new SegmentDocsAndPositionsAndPayloadsEnum(freqIn, proxIn);
+      } else {
+        docsEnum = (SegmentDocsAndPositionsAndPayloadsEnum) reuse;
+        if (docsEnum.startFreqIn != freqIn) {
+          // If you are using ParellelReader, and pass in a
+          // reused DocsEnum, it could have come from another
+          // reader also using standard codec
+          docsEnum = new SegmentDocsAndPositionsAndPayloadsEnum(freqIn, proxIn);
+        }
+      }
+      return docsEnum.reset(fieldInfo, (DocTermState) termState, skipDocs);
     } else {
-      docsEnum = (SegmentDocsAndPositionsEnum) reuse;
-      if (docsEnum.startFreqIn != freqIn) {
-        // If you are using ParellelReader, and pass in a
-        // reused DocsEnum, it could have come from another
-        // reader also using standard codec
+      SegmentDocsAndPositionsEnum docsEnum;
+      if (reuse == null || !(reuse instanceof SegmentDocsAndPositionsEnum)) {
         docsEnum = new SegmentDocsAndPositionsEnum(freqIn, proxIn);
+      } else {
+        docsEnum = (SegmentDocsAndPositionsEnum) reuse;
+        if (docsEnum.startFreqIn != freqIn) {
+          // If you are using ParellelReader, and pass in a
+          // reused DocsEnum, it could have come from another
+          // reader also using standard codec
+          docsEnum = new SegmentDocsAndPositionsEnum(freqIn, proxIn);
+        }
       }
+      return docsEnum.reset(fieldInfo, (DocTermState) termState, skipDocs);
     }
-    return docsEnum.reset(fieldInfo, (DocTermState) termState, skipDocs);
   }
 
   // Decodes only docs
@@ -360,13 +378,195 @@ public class StandardPostingsReader exte
     }
   }
 
-  // Decodes docs & positions
+  // Decodes docs & positions. payloads are not present.
   private class SegmentDocsAndPositionsEnum extends DocsAndPositionsEnum {
     final IndexInput startFreqIn;
     private final IndexInput freqIn;
     private final IndexInput proxIn;
 
-    boolean storePayloads;                        // does current field store payloads?
+    int limit;                                    // number of docs in this posting
+    int ord;                                      // how many docs we've read
+    int doc;                                      // doc we last read
+    int freq;                                     // freq we last read
+    int position;
+
+    Bits skipDocs;
+
+    long freqOffset;
+    int skipOffset;
+    long proxOffset;
+
+    int posPendingCount;
+
+    boolean skipped;
+    DefaultSkipListReader skipper;
+    private long lazyProxPointer;
+
+    public SegmentDocsAndPositionsEnum(IndexInput freqIn, IndexInput proxIn) throws IOException {
+      startFreqIn = freqIn;
+      this.freqIn = (IndexInput) freqIn.clone();
+      this.proxIn = (IndexInput) proxIn.clone();
+    }
+
+    public SegmentDocsAndPositionsEnum reset(FieldInfo fieldInfo, DocTermState termState, Bits skipDocs) throws IOException {
+      assert !fieldInfo.omitTermFreqAndPositions;
+      assert !fieldInfo.storePayloads;
+
+      this.skipDocs = skipDocs;
+
+      // TODO: for full enum case (eg segment merging) this
+      // seek is unnecessary; maybe we can avoid in such
+      // cases
+      freqIn.seek(termState.freqOffset);
+      lazyProxPointer = termState.proxOffset;
+
+      limit = termState.docFreq;
+      ord = 0;
+      doc = 0;
+      position = 0;
+
+      skipped = false;
+      posPendingCount = 0;
+
+      freqOffset = termState.freqOffset;
+      proxOffset = termState.proxOffset;
+      skipOffset = termState.skipOffset;
+
+      return this;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      while(true) {
+        if (ord == limit) {
+          return doc = NO_MORE_DOCS;
+        }
+
+        ord++;
+
+        // Decode next doc/freq pair
+        final int code = freqIn.readVInt();
+
+        doc += code >>> 1;              // shift off low bit
+        if ((code & 1) != 0) {          // if low bit is set
+          freq = 1;                     // freq is one
+        } else {
+          freq = freqIn.readVInt();     // else read freq
+        }
+        posPendingCount += freq;
+
+        if (skipDocs == null || !skipDocs.get(doc)) {
+          break;
+        }
+      }
+
+      position = 0;
+
+      return doc;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public int freq() {
+      return freq;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+
+      // TODO: jump right to next() if target is < X away
+      // from where we are now?
+
+      if (skipOffset > 0) {
+
+        // There are enough docs in the posting to have
+        // skip data
+
+        if (skipper == null) {
+          // This is the first time this enum has ever been used for skipping -- do lazy init
+          skipper = new DefaultSkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
+        }
+
+        if (!skipped) {
+
+          // This is the first time this posting has
+          // skipped, since reset() was called, so now we
+          // load the skip data for this posting
+
+          skipper.init(freqOffset+skipOffset,
+                       freqOffset, proxOffset,
+                       limit, false);
+
+          skipped = true;
+        }
+
+        final int newOrd = skipper.skipTo(target); 
+
+        if (newOrd > ord) {
+          // Skipper moved
+          ord = newOrd;
+          doc = skipper.getDoc();
+          freqIn.seek(skipper.getFreqPointer());
+          lazyProxPointer = skipper.getProxPointer();
+          posPendingCount = 0;
+          position = 0;
+        }
+      }
+        
+      // Now, linear scan for the rest:
+      do {
+        nextDoc();
+      } while (target > doc);
+
+      return doc;
+    }
+
+    public int nextPosition() throws IOException {
+
+      if (lazyProxPointer != -1) {
+        proxIn.seek(lazyProxPointer);
+        lazyProxPointer = -1;
+      }
+
+      // scan over any docs that were iterated without their positions
+      if (posPendingCount > freq) {
+        position = 0;
+        while(posPendingCount != freq) {
+          if ((proxIn.readByte() & 0x80) == 0) {
+            posPendingCount--;
+          }
+        }
+      }
+
+      position += proxIn.readVInt();
+
+      posPendingCount--;
+
+      assert posPendingCount >= 0: "nextPosition() was called too many times (more than freq() times) posPendingCount=" + posPendingCount;
+
+      return position;
+    }
+
+    /** Returns the payload at this position, or null if no
+     *  payload was indexed. */
+    public BytesRef getPayload() throws IOException {
+      throw new IOException("No payloads exist for this field!");
+    }
+
+    public boolean hasPayload() {
+      return false;
+    }
+  }
+  
+  // Decodes docs & positions & payloads
+  private class SegmentDocsAndPositionsAndPayloadsEnum extends DocsAndPositionsEnum {
+    final IndexInput startFreqIn;
+    private final IndexInput freqIn;
+    private final IndexInput proxIn;
 
     int limit;                                    // number of docs in this posting
     int ord;                                      // how many docs we've read
@@ -389,16 +589,16 @@ public class StandardPostingsReader exte
     private BytesRef payload;
     private long lazyProxPointer;
 
-    public SegmentDocsAndPositionsEnum(IndexInput freqIn, IndexInput proxIn) throws IOException {
+    public SegmentDocsAndPositionsAndPayloadsEnum(IndexInput freqIn, IndexInput proxIn) throws IOException {
       startFreqIn = freqIn;
       this.freqIn = (IndexInput) freqIn.clone();
       this.proxIn = (IndexInput) proxIn.clone();
     }
 
-    public SegmentDocsAndPositionsEnum reset(FieldInfo fieldInfo, DocTermState termState, Bits skipDocs) throws IOException {
+    public SegmentDocsAndPositionsAndPayloadsEnum reset(FieldInfo fieldInfo, DocTermState termState, Bits skipDocs) throws IOException {
       assert !fieldInfo.omitTermFreqAndPositions;
-      storePayloads = fieldInfo.storePayloads;
-      if (storePayloads && payload == null) {
+      assert fieldInfo.storePayloads;
+      if (payload == null) {
         payload = new BytesRef();
         payload.bytes = new byte[1];
       }
@@ -491,7 +691,7 @@ public class StandardPostingsReader exte
 
           skipper.init(freqOffset+skipOffset,
                        freqOffset, proxOffset,
-                       limit, storePayloads);
+                       limit, true);
 
           skipped = true;
         }
@@ -537,15 +737,14 @@ public class StandardPostingsReader exte
 
         final int code = proxIn.readVInt();
 
-        if (storePayloads) {
-          if ((code & 1) != 0) {
-            // new payload length
-            payloadLength = proxIn.readVInt();
-            assert payloadLength >= 0;
-          }
-          assert payloadLength != -1;
-          proxIn.seek(proxIn.getFilePointer() + payloadLength);
+        if ((code & 1) != 0) {
+          // new payload length
+          payloadLength = proxIn.readVInt();
+          assert payloadLength >= 0;
         }
+        
+        assert payloadLength != -1;
+        proxIn.seek(proxIn.getFilePointer() + payloadLength);
 
         posPendingCount--;
         position = 0;
@@ -553,26 +752,21 @@ public class StandardPostingsReader exte
       }
 
       // read next position
-      if (storePayloads) {
-
-        if (payloadPending && payloadLength > 0) {
-          // payload wasn't retrieved for last position
-          proxIn.seek(proxIn.getFilePointer()+payloadLength);
-        }
+      if (payloadPending && payloadLength > 0) {
+        // payload wasn't retrieved for last position
+        proxIn.seek(proxIn.getFilePointer()+payloadLength);
+      }
 
-        final int code = proxIn.readVInt();
-        if ((code & 1) != 0) {
-          // new payload length
-          payloadLength = proxIn.readVInt();
-          assert payloadLength >= 0;
-        }
-        assert payloadLength != -1;
-          
-        payloadPending = true;
-        position += code >>> 1;
-      } else {
-        position += proxIn.readVInt();
+      final int code = proxIn.readVInt();
+      if ((code & 1) != 0) {
+        // new payload length
+        payloadLength = proxIn.readVInt();
+        assert payloadLength >= 0;
       }
+      assert payloadLength != -1;
+          
+      payloadPending = true;
+      position += code >>> 1;
 
       posPendingCount--;
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java Wed Nov 17 15:43:06 2010
@@ -60,14 +60,14 @@ public final class StandardPostingsWrite
 
   public StandardPostingsWriter(SegmentWriteState state) throws IOException {
     super();
-    String fileName = IndexFileNames.segmentFileName(state.segmentName, "", StandardCodec.FREQ_EXTENSION);
+    String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.FREQ_EXTENSION);
     state.flushedFiles.add(fileName);
     freqOut = state.directory.createOutput(fileName);
 
     if (state.fieldInfos.hasProx()) {
       // At least one field does not omit TF, so create the
       // prox file
-      fileName = IndexFileNames.segmentFileName(state.segmentName, "", StandardCodec.PROX_EXTENSION);
+      fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION);
       state.flushedFiles.add(fileName);
       proxOut = state.directory.createOutput(fileName);
     } else {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanQuery.java Wed Nov 17 15:43:06 2010
@@ -39,10 +39,8 @@ public class BooleanQuery extends Query 
    * is expanded to many terms during search. 
    */
   public static class TooManyClauses extends RuntimeException {
-    public TooManyClauses() {}
-    @Override
-    public String getMessage() {
-      return "maxClauseCount is set to " + maxClauseCount;
+    public TooManyClauses() {
+      super("maxClauseCount is set to " + maxClauseCount);
     }
   }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FieldComparator.java Wed Nov 17 15:43:06 2010
@@ -1251,12 +1251,11 @@ public abstract class FieldComparator {
             bottomOrd = index;
             // exact value match
             bottomSameReader = true;
+            readerGen[bottomSlot] = currentReaderGen;            
+            ords[bottomSlot] = bottomOrd;
           }
         }
       }
-      if (bottomSameReader) {
-        readerGen[bottomSlot] = currentReaderGen;
-      }
     }
 
     @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Wed Nov 17 15:43:06 2010
@@ -49,12 +49,12 @@ import java.util.List;
  */
 public final class FuzzyTermsEnum extends TermsEnum {
   private TermsEnum actualEnum;
-  private MultiTermQuery.BoostAttribute actualBoostAtt;
+  private BoostAttribute actualBoostAtt;
   
-  private final MultiTermQuery.BoostAttribute boostAtt =
-    attributes().addAttribute(MultiTermQuery.BoostAttribute.class);
+  private final BoostAttribute boostAtt =
+    attributes().addAttribute(BoostAttribute.class);
   
-  private final MultiTermQuery.MaxNonCompetitiveBoostAttribute maxBoostAtt;
+  private final MaxNonCompetitiveBoostAttribute maxBoostAtt;
   private final LevenshteinAutomataAttribute dfaAtt;
   
   private float bottom;
@@ -128,7 +128,7 @@ public final class FuzzyTermsEnum extend
     }
     this.scale_factor = 1.0f / (1.0f - this.minSimilarity);
 
-    this.maxBoostAtt = atts.addAttribute(MultiTermQuery.MaxNonCompetitiveBoostAttribute.class);
+    this.maxBoostAtt = atts.addAttribute(MaxNonCompetitiveBoostAttribute.class);
     bottom = maxBoostAtt.getMaxNonCompetitiveBoost();
     bottomTerm = maxBoostAtt.getCompetitiveTerm();
     bottomChanged(null, true);
@@ -174,8 +174,7 @@ public final class FuzzyTermsEnum extend
   /** swap in a new actual enum to proxy to */
   private void setEnum(TermsEnum actualEnum) {
     this.actualEnum = actualEnum;
-    this.actualBoostAtt = actualEnum.attributes().addAttribute(
-        MultiTermQuery.BoostAttribute.class);
+    this.actualBoostAtt = actualEnum.attributes().addAttribute(BoostAttribute.class);
   }
   
   /**
@@ -300,8 +299,8 @@ public final class FuzzyTermsEnum extend
     private final BytesRef termRef;
     
     private final BytesRef lastTerm;
-    private final MultiTermQuery.BoostAttribute boostAtt =
-      attributes().addAttribute(MultiTermQuery.BoostAttribute.class);
+    private final BoostAttribute boostAtt =
+      attributes().addAttribute(BoostAttribute.class);
     
     public AutomatonFuzzyTermsEnum(ByteRunAutomaton matchers[], 
         BytesRef lastTerm) throws IOException {
@@ -363,8 +362,8 @@ public final class FuzzyTermsEnum extend
     // this is the text, minus the prefix
     private final int[] text;
     
-    private final MultiTermQuery.BoostAttribute boostAtt =
-      attributes().addAttribute(MultiTermQuery.BoostAttribute.class);
+    private final BoostAttribute boostAtt =
+      attributes().addAttribute(BoostAttribute.class);
     
     /**
      * Constructor for enumeration of all terms from specified <code>reader</code> which share a prefix of

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java Wed Nov 17 15:43:06 2010
@@ -19,29 +19,12 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 import java.io.Serializable;
-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.index.Terms;
 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.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
@@ -80,156 +63,12 @@ public abstract class MultiTermQuery ext
   protected final String field;
   protected RewriteMethod rewriteMethod = CONSTANT_SCORE_AUTO_REWRITE_DEFAULT;
   transient int numberOfTerms = 0;
-  
-  /** 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();
-  }
-
-  /** Implementation class for {@link BoostAttribute}. */
-  public static final class BoostAttributeImpl extends AttributeImpl implements BoostAttribute {
-    private float boost = 1.0f;
-  
-    public void setBoost(float boost) {
-      this.boost = boost;
-    }
-    
-    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(final float maxNonCompetitiveBoost) {
-      this.maxNonCompetitiveBoost = maxNonCompetitiveBoost;
-    }
-    
-    public float getMaxNonCompetitiveBoost() {
-      return maxNonCompetitiveBoost;
-    }
-
-    public void setCompetitiveTerm(final BytesRef competitiveTerm) {
-      this.competitiveTerm = competitiveTerm;
-    }
-    
-    public BytesRef getCompetitiveTerm() {
-      return competitiveTerm;
-    }
-
-    @Override
-    public void clear() {
-      maxNonCompetitiveBoost = Float.NEGATIVE_INFINITY;
-      competitiveTerm = null;
-    }
-
-    @Override
-    public boolean equals(Object other) {
-      if (this == other)
-        return true;
-      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() {
-      int hash = Float.floatToIntBits(maxNonCompetitiveBoost);
-      if (competitiveTerm != null) hash = 31 * hash + competitiveTerm.hashCode();
-      return hash;
-    }
-    
-    @Override
-    public void copyTo(AttributeImpl target) {
-      final MaxNonCompetitiveBoostAttributeImpl t = (MaxNonCompetitiveBoostAttributeImpl) target;
-      t.setMaxNonCompetitiveBoost(maxNonCompetitiveBoost);
-      t.setCompetitiveTerm(competitiveTerm);
-    }
-  }
 
   /** Abstract class that defines how the query is rewritten. */
   public static abstract class RewriteMethod implements Serializable {
     public abstract Query rewrite(IndexReader reader, MultiTermQuery query) throws IOException;
   }
 
-  private static final class ConstantScoreFilterRewrite extends RewriteMethod {
-    @Override
-    public Query rewrite(IndexReader reader, MultiTermQuery query) {
-      Query result = new ConstantScoreQuery(new MultiTermQueryWrapperFilter<MultiTermQuery>(query));
-      result.setBoost(query.getBoost());
-      return result;
-    }
-
-    // Make sure we are still a singleton even after deserializing
-    protected Object readResolve() {
-      return CONSTANT_SCORE_FILTER_REWRITE;
-    }
-  }
-
   /** A rewrite method that first creates a private Filter,
    *  by visiting each term in sequence and marking all docs
    *  for that term.  Matching documents are assigned a
@@ -242,162 +81,19 @@ public abstract class MultiTermQuery ext
    *  exception.
    *
    *  @see #setRewriteMethod */
-  public final static RewriteMethod CONSTANT_SCORE_FILTER_REWRITE = new ConstantScoreFilterRewrite();
-
-  private abstract static class BooleanQueryRewrite extends RewriteMethod {
-  
-    protected final void collectTerms(IndexReader reader, MultiTermQuery query, TermCollector collector) throws IOException {
-      final List<IndexReader> subReaders = new ArrayList<IndexReader>();
-      ReaderUtil.gatherSubReaders(subReaders, reader);
-      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) {
-          termsEnum.cacheCurrentTerm();
-          if (!collector.collect(bytes))
-            return; // interrupt whole term collection, so also don't iterate other subReaders
-        }
-      }
-    }
-    
-    protected static abstract class TermCollector {
-      /** attributes used for communication with the enum */
-      public final AttributeSource attributes = new AttributeSource();
-    
-      /** return false to stop collecting */
-      public abstract boolean collect(BytesRef bytes) throws IOException;
-      
-      /** 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 {
+  public static final RewriteMethod CONSTANT_SCORE_FILTER_REWRITE = new RewriteMethod() {
     @Override
-    public Query rewrite(final IndexReader reader, final MultiTermQuery query) throws IOException {
-      final ParallelArraysTermCollector col = new ParallelArraysTermCollector();
-      collectTerms(reader, query, col);
-      
-      final Term placeholderTerm = new Term(query.field);
-      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);
+    public Query rewrite(IndexReader reader, MultiTermQuery query) {
+      Query result = new ConstantScoreQuery(new MultiTermQueryWrapperFilter<MultiTermQuery>(query));
+      result.setBoost(query.getBoost());
       return result;
     }
 
     // Make sure we are still a singleton even after deserializing
     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();
-      }
-      
+      return CONSTANT_SCORE_FILTER_REWRITE;
     }
-  }
+  };
 
   /** A rewrite method that first translates each term into
    *  {@link BooleanClause.Occur#SHOULD} clause in a
@@ -412,155 +108,19 @@ public abstract class MultiTermQuery ext
    *  exceeds {@link BooleanQuery#getMaxClauseCount}.
    *
    *  @see #setRewriteMethod */
-  public final static RewriteMethod SCORING_BOOLEAN_QUERY_REWRITE = new ScoringBooleanQueryRewrite();
-
-
-  /**
-   * Base rewrite method for collecting only the top terms
-   * via a priority queue.
-   */
-  public static abstract class TopTermsBooleanQueryRewrite extends BooleanQueryRewrite {
-    private final int size;
-    
-    /** 
-     * Create a TopTermsBooleanQueryRewrite for 
-     * at most <code>size</code> terms.
-     * <p>
-     * NOTE: if {@link BooleanQuery#getMaxClauseCount} is smaller than 
-     * <code>size</code>, then it will be used instead. 
-     */
-    public TopTermsBooleanQueryRewrite(int size) {
-      this.size = size;
-    }
-    
-    /** Return a suitable Query for a MultiTermQuery term. */
-    protected abstract Query getQuery(Term term, int docCount);
-
-    @Override
-    public Query rewrite(final IndexReader reader, final MultiTermQuery query) throws IOException {
-      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 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) {
-            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;
-        }
-      });
-      
-      final Term placeholderTerm = new Term(query.field);
-      final BooleanQuery bq = new BooleanQuery(true);
-      final ScoreTerm[] scoreTerms = stQueue.toArray(new ScoreTerm[stQueue.size()]);
-      ArrayUtil.quickSort(scoreTerms, scoreTermSortByTermComp);
-      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(scoreTerms.length);
-      return bq;
-    }
+  public final static RewriteMethod SCORING_BOOLEAN_QUERY_REWRITE = ScoringRewrite.SCORING_BOOLEAN_QUERY_REWRITE;
   
-    @Override
-    public int hashCode() {
-      return 31 * size;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (this == obj) return true;
-      if (obj == null) return false;
-      if (getClass() != obj.getClass()) return false;
-      TopTermsBooleanQueryRewrite other = (TopTermsBooleanQueryRewrite) obj;
-      if (size != other.size) return false;
-      return true;
-    }
-    
-    private static final Comparator<ScoreTerm> scoreTermSortByTermComp = 
-      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);
-        }
-      };
-  
-    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)
-          return termComp.compare(other.bytes, this.bytes);
-        else
-          return Float.compare(this.boost, other.boost);
-      }
-    }
-  }
+  /** Like {@link #SCORING_BOOLEAN_QUERY_REWRITE} except
+   *  scores are not computed.  Instead, each matching
+   *  document receives a constant score equal to the
+   *  query's boost.
+   * 
+   *  <p><b>NOTE</b>: This rewrite method will hit {@link
+   *  BooleanQuery.TooManyClauses} if the number of terms
+   *  exceeds {@link BooleanQuery#getMaxClauseCount}.
+   *
+   *  @see #setRewriteMethod */
+  public final static RewriteMethod CONSTANT_SCORE_BOOLEAN_QUERY_REWRITE = ScoringRewrite.CONSTANT_SCORE_BOOLEAN_QUERY_REWRITE;
 
   /**
    * A rewrite method that first translates each term into
@@ -574,8 +134,7 @@ public abstract class MultiTermQuery ext
    * 
    * @see #setRewriteMethod
    */
-  public static final class TopTermsScoringBooleanQueryRewrite extends
-      TopTermsBooleanQueryRewrite {
+  public static final class TopTermsScoringBooleanQueryRewrite extends TopTermsRewrite<BooleanQuery> {
 
     /** 
      * Create a TopTermsScoringBooleanQueryRewrite for 
@@ -589,8 +148,20 @@ public abstract class MultiTermQuery ext
     }
     
     @Override
-    protected Query getQuery(Term term, int docFreq) {
-      return new TermQuery(term, docFreq);
+    protected int getMaxSize() {
+      return BooleanQuery.getMaxClauseCount();
+    }
+    
+    @Override
+    protected BooleanQuery getTopLevelQuery() {
+      return new BooleanQuery(true);
+    }
+    
+    @Override
+    protected void addClause(BooleanQuery topLevel, Term term, int docCount, float boost) {
+      final TermQuery tq = new TermQuery(term, docCount);
+      tq.setBoost(boost);
+      topLevel.add(tq, BooleanClause.Occur.SHOULD);
     }
   }
   
@@ -604,8 +175,7 @@ public abstract class MultiTermQuery ext
    * 
    * @see #setRewriteMethod
    */
-  public static final class TopTermsBoostOnlyBooleanQueryRewrite extends
-      TopTermsBooleanQueryRewrite {
+  public static final class TopTermsBoostOnlyBooleanQueryRewrite extends TopTermsRewrite<BooleanQuery> {
     
     /** 
      * Create a TopTermsBoostOnlyBooleanQueryRewrite for 
@@ -619,45 +189,23 @@ public abstract class MultiTermQuery ext
     }
     
     @Override
-    protected Query getQuery(Term term, int docFreq) {
-      return new ConstantScoreQuery(new QueryWrapperFilter(new TermQuery(term, docFreq)));
+    protected int getMaxSize() {
+      return BooleanQuery.getMaxClauseCount();
     }
-  }
-  
-  private static class ConstantScoreBooleanQueryRewrite extends ScoringBooleanQueryRewrite implements Serializable {
+    
     @Override
-    public Query rewrite(IndexReader reader, MultiTermQuery query) throws IOException {
-      Query result = super.rewrite(reader, query);
-      assert result instanceof BooleanQuery;
-      // TODO: if empty boolean query return NullQuery?
-      if (!((BooleanQuery) result).clauses().isEmpty()) {
-        // strip the scores off
-        result = new ConstantScoreQuery(new QueryWrapperFilter(result));
-        result.setBoost(query.getBoost());
-      }
-      return result;
+    protected BooleanQuery getTopLevelQuery() {
+      return new BooleanQuery(true);
     }
-
-    // Make sure we are still a singleton even after deserializing
+    
     @Override
-    protected Object readResolve() {
-      return CONSTANT_SCORE_BOOLEAN_QUERY_REWRITE;
+    protected void addClause(BooleanQuery topLevel, Term term, int docFreq, float boost) {
+      final Query q = new ConstantScoreQuery(new QueryWrapperFilter(new TermQuery(term, docFreq)));
+      q.setBoost(boost);
+      topLevel.add(q, BooleanClause.Occur.SHOULD);
     }
   }
-
-  /** Like {@link #SCORING_BOOLEAN_QUERY_REWRITE} except
-   *  scores are not computed.  Instead, each matching
-   *  document receives a constant score equal to the
-   *  query's boost.
-   * 
-   *  <p><b>NOTE</b>: This rewrite method will hit {@link
-   *  BooleanQuery.TooManyClauses} if the number of terms
-   *  exceeds {@link BooleanQuery#getMaxClauseCount}.
-   *
-   *  @see #setRewriteMethod */
-  public final static RewriteMethod CONSTANT_SCORE_BOOLEAN_QUERY_REWRITE = new ConstantScoreBooleanQueryRewrite();
-
-
+    
   /** A rewrite method that tries to pick the best
    *  constant-score rewrite method based on term and
    *  document counts from the query.  If both the number of
@@ -666,140 +214,7 @@ public abstract class MultiTermQuery ext
    *  Otherwise, {@link #CONSTANT_SCORE_FILTER_REWRITE} is
    *  used.
    */
-  public static class ConstantScoreAutoRewrite extends BooleanQueryRewrite {
-
-    // Defaults derived from rough tests with a 20.0 million
-    // doc Wikipedia index.  With more than 350 terms in the
-    // query, the filter method is fastest:
-    public static int DEFAULT_TERM_COUNT_CUTOFF = 350;
-
-    // If the query will hit more than 1 in 1000 of the docs
-    // in the index (0.1%), the filter method is fastest:
-    public static double DEFAULT_DOC_COUNT_PERCENT = 0.1;
-
-    private int termCountCutoff = DEFAULT_TERM_COUNT_CUTOFF;
-    private double docCountPercent = DEFAULT_DOC_COUNT_PERCENT;
-
-    /** If the number of terms in this query is equal to or
-     *  larger than this setting then {@link
-     *  #CONSTANT_SCORE_FILTER_REWRITE} is used. */
-    public void setTermCountCutoff(int count) {
-      termCountCutoff = count;
-    }
-
-    /** @see #setTermCountCutoff */
-    public int getTermCountCutoff() {
-      return termCountCutoff;
-    }
-
-    /** If the number of documents to be visited in the
-     *  postings exceeds this specified percentage of the
-     *  maxDoc() for the index, then {@link
-     *  #CONSTANT_SCORE_FILTER_REWRITE} is used.
-     *  @param percent 0.0 to 100.0 */
-    public void setDocCountPercent(double percent) {
-      docCountPercent = percent;
-    }
-
-    /** @see #setDocCountPercent */
-    public double getDocCountPercent() {
-      return docCountPercent;
-    }
-
-    @Override
-    public Query rewrite(final IndexReader reader, final MultiTermQuery query) throws IOException {
-
-      // Get the enum and start visiting terms.  If we
-      // exhaust the enum before hitting either of the
-      // cutoffs, we use ConstantBooleanQueryRewrite; else,
-      // ConstantFilterRewrite:
-      final int docCountCutoff = (int) ((docCountPercent / 100.) * reader.maxDoc());
-      final int termCountLimit = Math.min(BooleanQuery.getMaxClauseCount(), termCountCutoff);
-
-      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 (size == 0) {
-        return new BooleanQuery(true);
-      } else {
-        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;
-      }
-    }
-    
-    static final class CutOffTermCollector extends TermCollector {
-      CutOffTermCollector(int docCountCutoff, int termCountLimit) {
-        this.docCountCutoff = docCountCutoff;
-        this.termCountLimit = termCountLimit;
-      }
-    
-      @Override
-      public void setNextEnum(TermsEnum termsEnum) throws IOException {
-        this.termsEnum = termsEnum;
-      }
-        
-      @Override
-      public boolean collect(BytesRef bytes) throws IOException {
-        pendingTerms.add(bytes);
-        docVisitCount += termsEnum.docFreq();
-        if (pendingTerms.size() >= termCountLimit || docVisitCount >= docCountCutoff) {
-          hasCutOff = true;
-          return false;
-        }
-        return true;
-      }
-      
-      int docVisitCount = 0;
-      boolean hasCutOff = false;
-      TermsEnum termsEnum;
-
-      final int docCountCutoff, termCountLimit;
-      final BytesRefHash pendingTerms = new BytesRefHash();
-    }
-
-    @Override
-    public int hashCode() {
-      final int prime = 1279;
-      return (int) (prime * termCountCutoff + Double.doubleToLongBits(docCountPercent));
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (this == obj)
-        return true;
-      if (obj == null)
-        return false;
-      if (getClass() != obj.getClass())
-        return false;
-
-      ConstantScoreAutoRewrite other = (ConstantScoreAutoRewrite) obj;
-      if (other.termCountCutoff != termCountCutoff) {
-        return false;
-      }
-
-      if (Double.doubleToLongBits(other.docCountPercent) != Double.doubleToLongBits(docCountPercent)) {
-        return false;
-      }
-      
-      return true;
-    }
-  }
+  public static class ConstantScoreAutoRewrite extends org.apache.lucene.search.ConstantScoreAutoRewrite {}
 
   /** Read-only default instance of {@link
    *  ConstantScoreAutoRewrite}, with {@link
@@ -851,7 +266,7 @@ public abstract class MultiTermQuery ext
    *  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}
+   * This is currently only used by {@link TopTermsRewrite}
    */
   protected abstract TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException;
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanFirstQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanFirstQuery.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanFirstQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanFirstQuery.java Wed Nov 17 15:43:06 2010
@@ -37,8 +37,14 @@ public class SpanFirstQuery extends Span
   }
 
   @Override
-  protected boolean acceptPosition(Spans spans) throws IOException {
-    return spans.end() <= end;
+  protected AcceptStatus acceptPosition(Spans spans) throws IOException {
+    assert spans.start() != spans.end();
+    if (spans.start() >= end)
+      return AcceptStatus.NO_AND_ADVANCE;
+    else if (spans.end() <= end)
+      return AcceptStatus.YES;
+    else
+      return AcceptStatus.NO;
   }
 
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanNearPayloadCheckQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanNearPayloadCheckQuery.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanNearPayloadCheckQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanNearPayloadCheckQuery.java Wed Nov 17 15:43:06 2010
@@ -43,7 +43,7 @@ public class SpanNearPayloadCheckQuery e
   }
 
   @Override
-  protected boolean acceptPosition(Spans spans) throws IOException {
+  protected AcceptStatus acceptPosition(Spans spans) throws IOException {
     boolean result = spans.isPayloadAvailable();
     if (result == true) {
       Collection<byte[]> candidate = spans.getPayload();
@@ -62,15 +62,15 @@ public class SpanNearPayloadCheckQuery e
         }
         if (matches == payloadToMatch.size()){
           //we've verified all the bytes
-          return true;
+          return AcceptStatus.YES;
         } else {
-          return false;
+          return AcceptStatus.NO;
         }
       } else {
-        return false;
+        return AcceptStatus.NO;
       }
     }
-    return false;
+    return AcceptStatus.NO;
   }
 
   public String toString(String field) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java Wed Nov 17 15:43:06 2010
@@ -42,16 +42,20 @@ public class SpanOrQuery extends SpanQue
     // copy clauses array into an ArrayList
     this.clauses = new ArrayList<SpanQuery>(clauses.length);
     for (int i = 0; i < clauses.length; i++) {
-      SpanQuery clause = clauses[i];
-      if (i == 0) {                               // check field
-        field = clause.getField();
-      } else if (!clause.getField().equals(field)) {
-        throw new IllegalArgumentException("Clauses must have same field.");
-      }
-      this.clauses.add(clause);
+      addClause(clauses[i]);
     }
   }
 
+  /** Adds a clause to this query */
+  public final void addClause(SpanQuery clause) {
+    if (field == null) {
+      field = clause.getField();
+    } else if (!clause.getField().equals(field)) {
+      throw new IllegalArgumentException("Clauses must have same field.");
+    }
+    this.clauses.add(clause);
+  }
+  
   /** Return the clauses whose spans are matched. */
   public SpanQuery[] getClauses() {
     return clauses.toArray(new SpanQuery[clauses.size()]);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanPayloadCheckQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanPayloadCheckQuery.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanPayloadCheckQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanPayloadCheckQuery.java Wed Nov 17 15:43:06 2010
@@ -50,7 +50,7 @@ public class SpanPayloadCheckQuery exten
   }
 
   @Override
-  protected boolean acceptPosition(Spans spans) throws IOException {
+  protected AcceptStatus acceptPosition(Spans spans) throws IOException {
     boolean result = spans.isPayloadAvailable();
     if (result == true){
       Collection<byte[]> candidate = spans.getPayload();
@@ -62,16 +62,16 @@ public class SpanPayloadCheckQuery exten
         for (byte[] candBytes : candidate) {
           //if one is a mismatch, then return false
           if (Arrays.equals(candBytes, toMatchIter.next()) == false){
-            return false;
+            return AcceptStatus.NO;
           }
         }
         //we've verified all the bytes
-        return true;
+        return AcceptStatus.YES;
       } else {
-        return false;
+        return AcceptStatus.NO;
       }
     }
-    return result;
+    return AcceptStatus.YES;
   } 
 
   public String toString(String field) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java Wed Nov 17 15:43:06 2010
@@ -57,6 +57,12 @@ public abstract class SpanPositionCheckQ
 	    match.extractTerms(terms);
   }
 
+  /** Return value if the match should be accepted {@code YES}, rejected {@code NO},
+   * or rejected and enumeration should advance to the next document {@code NO_AND_ADVANCE}.
+   * @see #acceptPosition(Spans)
+   */
+  protected static enum AcceptStatus { YES, NO, NO_AND_ADVANCE };
+  
   /**
    * Implementing classes are required to return whether the current position is a match for the passed in
    * "match" {@link org.apache.lucene.search.spans.SpanQuery}.
@@ -66,36 +72,12 @@ public abstract class SpanPositionCheckQ
    *
    *
    * @param spans The {@link org.apache.lucene.search.spans.Spans} instance, positioned at the spot to check
-   * @return true if it is a match, else false.
+   * @return whether the match is accepted, rejected, or rejected and should move to the next doc.
    *
    * @see org.apache.lucene.search.spans.Spans#next()
    *
    */
-  protected abstract boolean acceptPosition(Spans spans) throws IOException;
-
-  /**
-   * Implementing classes are required to return whether the position at the target is someplace that
-   * can be skipped to.  For instance, the {@link org.apache.lucene.search.spans.SpanFirstQuery} returns
-   * false if the target position is beyond the maximum position allowed or if {@link Spans#next()} is true.
-   * <p/>
-   * Note, this method is only called if the underlying match {@link org.apache.lucene.search.spans.SpanQuery} can
-   * skip to the target.
-   * <p/>
-   * It is safe to assume that the passed in {@link org.apache.lucene.search.spans.Spans} object for the underlying {@link org.apache.lucene.search.spans.SpanQuery} is
-   * positioned at the target.
-   * <p/>
-   * The default implementation is to return true if either {@link #acceptPosition(Spans)} or {@link org.apache.lucene.search.spans.Spans#next()} is true for the
-   * passed in instance of Spans.
-   *<p/>
-   * @param spans The {@link org.apache.lucene.search.spans.Spans} to check
-   * @return true if the instance can skip to this position
-   *
-   * @see Spans#skipTo(int)
-   * @throws java.io.IOException if there is a low-level IO error
-   */
-  protected boolean acceptSkipTo(Spans spans) throws IOException{
-    return acceptPosition(spans) || spans.next();
-  }
+  protected abstract AcceptStatus acceptPosition(Spans spans) throws IOException;
 
   @Override
   public Spans getSpans(final IndexReader reader) throws IOException {
@@ -123,21 +105,16 @@ public abstract class SpanPositionCheckQ
   protected class PositionCheckSpan extends Spans {
     private Spans spans;
 
-    private final IndexReader reader;
-
     public PositionCheckSpan(IndexReader reader) throws IOException {
-      this.reader = reader;
       spans = match.getSpans(reader);
     }
 
     @Override
     public boolean next() throws IOException {
-      //TODO: optimize to skip ahead to start
-      while (spans.next()) {                  // scan to next match
-        if (acceptPosition(this))
-          return true;
-      }
-      return false;
+      if (!spans.next())
+        return false;
+      
+      return doNext();
     }
 
     @Override
@@ -145,8 +122,23 @@ public abstract class SpanPositionCheckQ
       if (!spans.skipTo(target))
         return false;
 
-      return acceptSkipTo(this);
-
+      return doNext();
+    }
+    
+    protected boolean doNext() throws IOException {
+      for (;;) {
+        switch(acceptPosition(this)) {
+          case YES: return true;
+          case NO: 
+            if (!spans.next()) 
+              return false;
+            break;
+          case NO_AND_ADVANCE: 
+            if (!spans.skipTo(spans.doc()+1)) 
+              return false;
+            break;
+        }
+      }
     }
 
     @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanPositionRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanPositionRangeQuery.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanPositionRangeQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/spans/SpanPositionRangeQuery.java Wed Nov 17 15:43:06 2010
@@ -39,8 +39,14 @@ public class SpanPositionRangeQuery exte
 
 
   @Override
-  protected boolean acceptPosition(Spans spans) throws IOException {
-    return spans.start() >= start && spans.end() <= end;
+  protected AcceptStatus acceptPosition(Spans spans) throws IOException {
+    assert spans.start() != spans.end();
+    if (spans.start() >= end)
+      return AcceptStatus.NO_AND_ADVANCE;
+    else if (spans.start() >= start && spans.end() <= end)
+      return AcceptStatus.YES;
+    else
+      return AcceptStatus.NO;
   }
 
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/DataInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/DataInput.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/DataInput.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/DataInput.java Wed Nov 17 15:43:06 2010
@@ -82,7 +82,7 @@ public abstract class DataInput implemen
    * supported.
    * @see DataOutput#writeVInt(int)
    */
-  public int readVInt() throws IOException {
+  public final int readVInt() throws IOException {
     byte b = readByte();
     int i = b & 0x7F;
     for (int shift = 7; (b & 0x80) != 0; shift += 7) {
@@ -102,7 +102,7 @@ public abstract class DataInput implemen
   /** Reads a long stored in variable-length format.  Reads between one and
    * nine bytes.  Smaller values take fewer bytes.  Negative numbers are not
    * supported. */
-  public long readVLong() throws IOException {
+  public final long readVLong() throws IOException {
     byte b = readByte();
     long i = b & 0x7F;
     for (int shift = 7; (b & 0x80) != 0; shift += 7) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/DataOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/DataOutput.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/DataOutput.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/store/DataOutput.java Wed Nov 17 15:43:06 2010
@@ -66,7 +66,7 @@ public abstract class DataOutput {
    * supported.
    * @see DataInput#readVInt()
    */
-  public void writeVInt(int i) throws IOException {
+  public final void writeVInt(int i) throws IOException {
     while ((i & ~0x7F) != 0) {
       writeByte((byte)((i & 0x7f) | 0x80));
       i >>>= 7;
@@ -87,7 +87,7 @@ public abstract class DataOutput {
    * supported.
    * @see DataInput#readVLong()
    */
-  public void writeVLong(long i) throws IOException {
+  public final void writeVLong(long i) throws IOException {
     while ((i & ~0x7F) != 0) {
       writeByte((byte)((i & 0x7f) | 0x80));
       i >>>= 7;

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestExternalCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestExternalCodecs.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestExternalCodecs.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/TestExternalCodecs.java Wed Nov 17 15:43:06 2010
@@ -494,7 +494,7 @@ public class TestExternalCodecs extends 
     }
 
     @Override
-    public void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) {
+    public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) {
     }
   }
 
@@ -563,7 +563,7 @@ public class TestExternalCodecs extends 
     @Override
     public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
 
-      PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize);
+      PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize, state.codecId);
       PostingsReaderBase pulsingReader = new PulsingPostingsReaderImpl(docsReader);
 
       // Terms dict index reader
@@ -575,7 +575,8 @@ public class TestExternalCodecs extends 
                                                          state.fieldInfos,
                                                          state.segmentInfo.name,
                                                          state.termsIndexDivisor,
-                                                         reverseUnicodeComparator);
+                                                         reverseUnicodeComparator,
+                                                         state.codecId);
         success = true;
       } finally {
         if (!success) {
@@ -593,7 +594,8 @@ public class TestExternalCodecs extends 
                                                          pulsingReader,
                                                          state.readBufferSize,
                                                          reverseUnicodeComparator,
-                                                         StandardCodec.TERMS_CACHE_SIZE);
+                                                         StandardCodec.TERMS_CACHE_SIZE,
+                                                         state.codecId);
         success = true;
         return ret;
       } finally {
@@ -608,10 +610,10 @@ public class TestExternalCodecs extends 
     }
 
     @Override
-    public void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) throws IOException {
-      StandardPostingsReader.files(dir, segmentInfo, files);
-      PrefixCodedTermsReader.files(dir, segmentInfo, files);
-      FixedGapTermsIndexReader.files(dir, segmentInfo, files);
+    public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) throws IOException {
+      StandardPostingsReader.files(dir, segmentInfo, codecId, files);
+      PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files);
+      FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
     }
 
     @Override

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Wed Nov 17 15:43:06 2010
@@ -366,7 +366,7 @@ public class TestAddIndexes extends Luce
         .setOpenMode(OpenMode.APPEND).setMaxBufferedDocs(10));
     ((LogMergePolicy) writer.getConfig().getMergePolicy()).setMergeFactor(4);
 
-    writer.addIndexes(new Directory[] { aux, new MockDirectoryWrapper(new RAMDirectory(aux)) });
+    writer.addIndexes(new Directory[] { aux, new MockDirectoryWrapper(random, new RAMDirectory(aux)) });
     assertEquals(1060, writer.maxDoc());
     assertEquals(1000, writer.getDocCount(0));
     writer.close();
@@ -398,7 +398,7 @@ public class TestAddIndexes extends Luce
         .setOpenMode(OpenMode.APPEND).setMaxBufferedDocs(4));
     ((LogMergePolicy) writer.getConfig().getMergePolicy()).setMergeFactor(4);
 
-    writer.addIndexes(new Directory[] { aux, new MockDirectoryWrapper(new RAMDirectory(aux)) });
+    writer.addIndexes(new Directory[] { aux, new MockDirectoryWrapper(random, new RAMDirectory(aux)) });
     assertEquals(1060, writer.maxDoc());
     assertEquals(1000, writer.getDocCount(0));
     writer.close();
@@ -592,7 +592,7 @@ public class TestAddIndexes extends Luce
 
     public RunAddIndexesThreads(int numCopy) throws Throwable {
       NUM_COPY = numCopy;
-      dir = new MockDirectoryWrapper(new RAMDirectory());
+      dir = new MockDirectoryWrapper(random, new RAMDirectory());
       IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(
           TEST_VERSION_CURRENT, new MockAnalyzer())
           .setMaxBufferedDocs(2));
@@ -600,7 +600,7 @@ public class TestAddIndexes extends Luce
         addDoc(writer);
       writer.close();
 
-      dir2 = new MockDirectoryWrapper(new RAMDirectory());
+      dir2 = new MockDirectoryWrapper(random, new RAMDirectory());
       writer2 = new IndexWriter(dir2, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()));
       writer2.commit();
 
@@ -619,7 +619,7 @@ public class TestAddIndexes extends Luce
 
                 final Directory[] dirs = new Directory[NUM_COPY];
                 for(int k=0;k<NUM_COPY;k++)
-                  dirs[k] = new MockDirectoryWrapper(new RAMDirectory(dir));
+                  dirs[k] = new MockDirectoryWrapper(random, new RAMDirectory(dir));
 
                 int j=0;
 
@@ -677,20 +677,35 @@ public class TestAddIndexes extends Luce
     void doBody(int j, Directory[] dirs) throws Throwable {
       switch(j%5) {
       case 0:
+        if (VERBOSE) {
+          System.out.println(Thread.currentThread().getName() + ": TEST: addIndexes(Dir[]) then optimize");
+        }
         writer2.addIndexes(dirs);
         writer2.optimize();
         break;
       case 1:
+        if (VERBOSE) {
+          System.out.println(Thread.currentThread().getName() + ": TEST: addIndexes(Dir[])");
+        }
         writer2.addIndexes(dirs);
         break;
       case 2:
+        if (VERBOSE) {
+          System.out.println(Thread.currentThread().getName() + ": TEST: addIndexes(IndexReader[])");
+        }
         writer2.addIndexes(readers);
         break;
       case 3:
+        if (VERBOSE) {
+          System.out.println(Thread.currentThread().getName() + ": TEST: addIndexes(Dir[]) then maybeMerge");
+        }
         writer2.addIndexes(dirs);
         writer2.maybeMerge();
         break;
       case 4:
+        if (VERBOSE) {
+          System.out.println(Thread.currentThread().getName() + ": TEST: commit");
+        }
         writer2.commit();
       }
     }
@@ -703,6 +718,7 @@ public class TestAddIndexes extends Luce
     final int NUM_ITER = 15;
     final int NUM_COPY = 3;
     CommitAndAddIndexes c = new CommitAndAddIndexes(NUM_COPY);
+    c.writer2.setInfoStream(VERBOSE ? System.out : null);
     c.launchThreads(NUM_ITER);
 
     for(int i=0;i<100;i++)

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java Wed Nov 17 15:43:06 2010
@@ -189,7 +189,7 @@ public class TestAtomicUpdate extends Lu
     Directory directory;
 
     // First in a RAM directory:
-    directory = new MockDirectoryWrapper(new RAMDirectory());
+    directory = new MockDirectoryWrapper(random, new RAMDirectory());
     runTest(directory);
     directory.close();
 

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexReader.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexReader.java Wed Nov 17 15:43:06 2010
@@ -895,7 +895,7 @@ public class TestIndexReader extends Luc
 
       // Iterate w/ ever increasing free disk space:
       while(!done) {
-        MockDirectoryWrapper dir = new MockDirectoryWrapper(new RAMDirectory(startDir));
+        MockDirectoryWrapper dir = new MockDirectoryWrapper(random, new RAMDirectory(startDir));
 
         // If IndexReader hits disk full, it can write to
         // the same files again.
@@ -941,7 +941,7 @@ public class TestIndexReader extends Luc
           }
 
           dir.setMaxSizeInBytes(thisDiskFree);
-          dir.setRandomIOExceptionRate(rate, diskFree);
+          dir.setRandomIOExceptionRate(rate);
 
           try {
             if (0 == x) {

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java Wed Nov 17 15:43:06 2010
@@ -905,7 +905,7 @@ public class TestIndexWriter extends Luc
       final class MyRAMDirectory extends MockDirectoryWrapper {
         private LockFactory myLockFactory;
         MyRAMDirectory(Directory delegate) {
-          super(delegate);
+          super(random, delegate);
           lockFactory = null;
           myLockFactory = new SingleInstanceLockFactory();
         }
@@ -2083,7 +2083,7 @@ public class TestIndexWriter extends Luc
     @Override
     public void run() {
       // LUCENE-2239: won't work with NIOFS/MMAP
-      Directory dir = new MockDirectoryWrapper(new RAMDirectory()); 
+      Directory dir = new MockDirectoryWrapper(random, new RAMDirectory()); 
       IndexWriter w = null;
       while(!finish) {
         try {

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java Wed Nov 17 15:43:06 2010
@@ -439,7 +439,7 @@ public class TestIndexWriterDelete exten
 
     // Iterate w/ ever increasing free disk space:
     while (!done) {
-      MockDirectoryWrapper dir = new MockDirectoryWrapper(new RAMDirectory(startDir));
+      MockDirectoryWrapper dir = new MockDirectoryWrapper(random, new RAMDirectory(startDir));
       dir.setPreventDoubleWrite(false);
       IndexWriter modifier = new IndexWriter(dir,
                                              newIndexWriterConfig(
@@ -488,7 +488,7 @@ public class TestIndexWriterDelete exten
         }
 
         dir.setMaxSizeInBytes(thisDiskFree);
-        dir.setRandomIOExceptionRate(rate, diskFree);
+        dir.setRandomIOExceptionRate(rate);
 
         try {
           if (0 == x) {

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java Wed Nov 17 15:43:06 2010
@@ -810,18 +810,18 @@ public class TestIndexWriterExceptions e
     w.close();
 
     for(int i=0;i<200;i++) {
-      MockDirectoryWrapper dir = new MockDirectoryWrapper(new RAMDirectory(startDir));
+      MockDirectoryWrapper dir = new MockDirectoryWrapper(random, new RAMDirectory(startDir));
       conf = newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setMergeScheduler(new ConcurrentMergeScheduler());
       ((ConcurrentMergeScheduler) conf.getMergeScheduler()).setSuppressExceptions();
       w = new IndexWriter(dir, conf);
-      dir.setRandomIOExceptionRate(0.5, 100);
+      dir.setRandomIOExceptionRate(0.5);
       try {
         w.optimize();
       } catch (IOException ioe) {
         if (ioe.getCause() == null)
           fail("optimize threw IOException without root cause");
       }
-      dir.setRandomIOExceptionRate(0, 0);
+      dir.setRandomIOExceptionRate(0);
       w.close();
       dir.close();
     }

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java Wed Nov 17 15:43:06 2010
@@ -54,7 +54,7 @@ public class TestIndexWriterOnDiskFull e
       while(true) {
         if (VERBOSE)
           System.out.println("TEST: cycle: diskFree=" + diskFree);
-        MockDirectoryWrapper dir = new MockDirectoryWrapper(new RAMDirectory());
+        MockDirectoryWrapper dir = new MockDirectoryWrapper(random, new RAMDirectory());
         dir.setMaxSizeInBytes(diskFree);
         IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
         MergeScheduler ms = writer.getConfig().getMergeScheduler();
@@ -213,7 +213,7 @@ public class TestIndexWriterOnDiskFull e
       while(!done) {
         
         // Make a new dir that will enforce disk usage:
-        MockDirectoryWrapper dir = new MockDirectoryWrapper(new RAMDirectory(startDir));
+        MockDirectoryWrapper dir = new MockDirectoryWrapper(random, new RAMDirectory(startDir));
         writer = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()).setOpenMode(OpenMode.APPEND));
         IOException err = null;
         
@@ -263,7 +263,7 @@ public class TestIndexWriterOnDiskFull e
           
           dir.setTrackDiskUsage(true);
           dir.setMaxSizeInBytes(thisDiskFree);
-          dir.setRandomIOExceptionRate(rate, diskFree);
+          dir.setRandomIOExceptionRate(rate);
           
           try {
             
@@ -390,7 +390,7 @@ public class TestIndexWriterOnDiskFull e
         
         // Make sure we don't hit disk full during close below:
         dir.setMaxSizeInBytes(0);
-        dir.setRandomIOExceptionRate(0.0, 0);
+        dir.setRandomIOExceptionRate(0.0);
         
         writer.close();
         

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterReader.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestIndexWriterReader.java Wed Nov 17 15:43:06 2010
@@ -67,6 +67,8 @@ public class TestIndexWriterReader exten
 
     Directory dir1 = newDirectory();
     IndexWriter writer = new IndexWriter(dir1, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
+    // test relies on no merges happening below:
+    ((LogMergePolicy) writer.getMergePolicy()).setMergeFactor(10);
 
     // create the index
     createIndexNoClose(!optimize, "index1", writer);
@@ -128,6 +130,9 @@ public class TestIndexWriterReader exten
 
     Directory dir1 = newDirectory();
     IndexWriter writer = new IndexWriter(dir1, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer()));
+    // test relies on no merges happening below:
+    ((LogMergePolicy) writer.getMergePolicy()).setMergeFactor(10);
+
     writer.setInfoStream(infoStream);
     // create the index
     createIndexNoClose(!optimize, "index1", writer);
@@ -350,7 +355,7 @@ public class TestIndexWriterReader exten
             try {
               final Directory[] dirs = new Directory[numDirs];
               for (int k = 0; k < numDirs; k++)
-                dirs[k] = new MockDirectoryWrapper(new RAMDirectory(addDir));
+                dirs[k] = new MockDirectoryWrapper(random, new RAMDirectory(addDir));
               //int j = 0;
               //while (true) {
                 // System.out.println(Thread.currentThread().getName() + ": iter
@@ -626,7 +631,7 @@ public class TestIndexWriterReader exten
 
     final Directory[] dirs = new Directory[10];
     for (int i=0;i<10;i++) {
-      dirs[i] = new MockDirectoryWrapper(new RAMDirectory(dir1));
+      dirs[i] = new MockDirectoryWrapper(random, new RAMDirectory(dir1));
     }
 
     IndexReader r = writer.getReader();

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestLazyProxSkipping.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestLazyProxSkipping.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestLazyProxSkipping.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestLazyProxSkipping.java Wed Nov 17 15:43:06 2010
@@ -50,7 +50,7 @@ public class TestLazyProxSkipping extend
 
     private class SeekCountingDirectory extends MockDirectoryWrapper {
       public SeekCountingDirectory(Directory delegate) {
-        super(delegate);
+        super(random, delegate);
       }
 
       @Override

Modified: lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java?rev=1036080&r1=1036079&r2=1036080&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test/org/apache/lucene/index/TestMultiLevelSkipList.java Wed Nov 17 15:43:06 2010
@@ -50,7 +50,7 @@ public class TestMultiLevelSkipList exte
   
   class CountingRAMDirectory extends MockDirectoryWrapper {
     public CountingRAMDirectory(Directory delegate) {
-      super(delegate);
+      super(random, delegate);
     }
 
     public IndexInput openInput(String fileName) throws IOException {