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 2011/05/02 15:51:22 UTC

svn commit: r1098566 [9/22] - in /lucene/dev/branches/docvalues: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/ant/ dev-tools/idea/lucene/contrib/db/bdb-je/ dev-tools/idea/lucene/contrib/db/bdb/ dev-tools/idea/lucene/contrib...

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Mon May  2 13:50:57 2011
@@ -19,7 +19,6 @@ package org.apache.lucene.index.codecs.s
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.codecs.FieldsProducer;
-import org.apache.lucene.index.values.DocValues;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.Terms;
@@ -113,12 +112,6 @@ class SimpleTextFieldsReader extends Fie
     public TermsEnum terms() throws IOException {
       return SimpleTextFieldsReader.this.terms(current).iterator();
     }
-
-    @Override
-    public DocValues docValues() throws IOException {
-      // TODO Auto-generated method stub
-      return null;
-    }
   }
 
   private class SimpleTextTermsEnum extends TermsEnum {

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=1098566&r1=1098565&r2=1098566&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 Mon May  2 13:50:57 2011
@@ -50,12 +50,13 @@ public final class StandardPostingsWrite
    * smaller indexes, greater acceleration, but fewer accelerable cases, while
    * smaller values result in bigger indexes, less acceleration and more
    * accelerable cases. More detailed experiments would be useful here. */
-  final int skipInterval = 16;
+  static final int DEFAULT_SKIP_INTERVAL = 16;
+  final int skipInterval;
   
   /**
    * Expert: minimum docFreq to write any skip data at all
    */
-  final int skipMinimum = skipInterval;
+  final int skipMinimum;
 
   /** Expert: The maximum number of skip levels. Smaller values result in 
    * slightly smaller indexes, but slower skipping in big posting lists.
@@ -82,7 +83,12 @@ public final class StandardPostingsWrite
   private RAMOutputStream bytesWriter = new RAMOutputStream();
 
   public StandardPostingsWriter(SegmentWriteState state) throws IOException {
+    this(state, DEFAULT_SKIP_INTERVAL);
+  }
+  public StandardPostingsWriter(SegmentWriteState state, int skipInterval) throws IOException {
     super();
+    this.skipInterval = skipInterval;
+    this.skipMinimum = skipInterval; /* set to the same for now */
     //this.segment = state.segmentName;
     String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecIdAsString(), StandardCodec.FREQ_EXTENSION);
     freqOut = state.directory.createOutput(fileName);

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=1098566&r1=1098565&r2=1098566&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 Mon May  2 13:50:57 2011
@@ -367,8 +367,12 @@ public class BooleanQuery extends Query 
         Query query = c.getQuery().rewrite(reader);    // rewrite first
 
         if (getBoost() != 1.0f) {                 // incorporate boost
-          if (query == c.getQuery())                   // if rewrite was no-op
+          if (query == c.getQuery()) {                   // if rewrite was no-op
             query = (Query)query.clone();         // then clone before boost
+          }
+          // Since the BooleanQuery only has 1 clause, the BooleanQuery will be
+          // written out. Therefore the rewritten Query's boost must incorporate both
+          // the clause's boost, and the boost of the BooleanQuery itself
           query.setBoost(getBoost() * query.getBoost());
         }
 
@@ -381,8 +385,12 @@ public class BooleanQuery extends Query 
       BooleanClause c = clauses.get(i);
       Query query = c.getQuery().rewrite(reader);
       if (query != c.getQuery()) {                     // clause rewrote: must clone
-        if (clone == null)
+        if (clone == null) {
+          // The BooleanQuery clone is lazily initialized so only initialize
+          // it if a rewritten clause differs from the original clause (and hasn't been
+          // initialized already).  If nothing differs, the clone isn't needlessly created
           clone = (BooleanQuery)this.clone();
+        }
         clone.clauses.set(i, new BooleanClause(query, c.getOccur()));
       }
     }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Mon May  2 13:50:57 2011
@@ -857,4 +857,9 @@ public class IndexSearcher {
       this.leaves = leaves;
     }
   }
+
+  @Override
+  public String toString() {
+    return "IndexSearcher(" + reader + ")";
+  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/AttributeSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/AttributeSource.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/AttributeSource.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/AttributeSource.java Mon May  2 13:50:57 2011
@@ -93,10 +93,33 @@ public class AttributeSource {
     }
   }
       
+  /**
+   * This class holds the state of an AttributeSource.
+   * @see #captureState
+   * @see #restoreState
+   */
+  public static final class State implements Cloneable {
+    AttributeImpl attribute;
+    State next;
+    
+    @Override
+    public Object clone() {
+      State clone = new State();
+      clone.attribute = (AttributeImpl) attribute.clone();
+      
+      if (next != null) {
+        clone.next = (State) next.clone();
+      }
+      
+      return clone;
+    }
+  }
+    
   // These two maps must always be in sync!!!
   // So they are private, final and read-only from the outside (read-only iterators)
   private final Map<Class<? extends Attribute>, AttributeImpl> attributes;
   private final Map<Class<? extends AttributeImpl>, AttributeImpl> attributeImpls;
+  private final State[] currentState;
 
   private AttributeFactory factory;
   
@@ -116,6 +139,7 @@ public class AttributeSource {
     }
     this.attributes = input.attributes;
     this.attributeImpls = input.attributeImpls;
+    this.currentState = input.currentState;
     this.factory = input.factory;
   }
   
@@ -125,6 +149,7 @@ public class AttributeSource {
   public AttributeSource(AttributeFactory factory) {
     this.attributes = new LinkedHashMap<Class<? extends Attribute>, AttributeImpl>();
     this.attributeImpls = new LinkedHashMap<Class<? extends AttributeImpl>, AttributeImpl>();
+    this.currentState = new State[1];
     this.factory = factory;
   }
   
@@ -147,11 +172,8 @@ public class AttributeSource {
    * if one instance implements more than one Attribute interface.
    */
   public final Iterator<AttributeImpl> getAttributeImplsIterator() {
-    if (hasAttributes()) {
-      if (currentState == null) {
-        computeCurrentState();
-      }
-      final State initState = currentState;
+    final State initState = getCurrentState();
+    if (initState != null) {
       return new Iterator<AttributeImpl>() {
         private State state = initState;
       
@@ -225,7 +247,7 @@ public class AttributeSource {
       // Attribute is a superclass of this interface
       if (!attributes.containsKey(curInterface)) {
         // invalidate state to force recomputation in captureState()
-        this.currentState = null;
+        this.currentState[0] = null;
         attributes.put(curInterface, att);
         attributeImpls.put(clazz, att);
       }
@@ -283,41 +305,21 @@ public class AttributeSource {
     }
     return attClass.cast(attImpl);
   }
-  
-  /**
-   * This class holds the state of an AttributeSource.
-   * @see #captureState
-   * @see #restoreState
-   */
-  public static final class State implements Cloneable {
-    AttributeImpl attribute;
-    State next;
     
-    @Override
-    public Object clone() {
-      State clone = new State();
-      clone.attribute = (AttributeImpl) attribute.clone();
-      
-      if (next != null) {
-        clone.next = (State) next.clone();
-      }
-      
-      return clone;
+  private State getCurrentState() {
+    State s  = currentState[0];
+    if (s != null || !hasAttributes()) {
+      return s;
     }
-  }
-  
-  private State currentState = null;
-  
-  private void computeCurrentState() {
-    currentState = new State();
-    State c = currentState;
+    State c = s = currentState[0] = new State();
     final Iterator<AttributeImpl> it = attributeImpls.values().iterator();
     c.attribute = it.next();
     while (it.hasNext()) {
       c.next = new State();
       c = c.next;
       c.attribute = it.next();
-    }        
+    }
+    return s;
   }
   
   /**
@@ -325,13 +327,8 @@ public class AttributeSource {
    * {@link AttributeImpl#clear()} on each Attribute implementation.
    */
   public final void clearAttributes() {
-    if (hasAttributes()) {
-      if (currentState == null) {
-        computeCurrentState();
-      }
-      for (State state = currentState; state != null; state = state.next) {
-        state.attribute.clear();
-      }
+    for (State state = getCurrentState(); state != null; state = state.next) {
+      state.attribute.clear();
     }
   }
   
@@ -340,14 +337,8 @@ public class AttributeSource {
    * {@link #restoreState} to restore the state of this or another AttributeSource.
    */
   public final State captureState() {
-    if (!hasAttributes()) {
-      return null;
-    }
-      
-    if (currentState == null) {
-      computeCurrentState();
-    }
-    return (State) this.currentState.clone();
+    final State state = this.getCurrentState();
+    return (state == null) ? null : (State) state.clone();
   }
   
   /**
@@ -382,15 +373,9 @@ public class AttributeSource {
   @Override
   public int hashCode() {
     int code = 0;
-    if (hasAttributes()) {
-      if (currentState == null) {
-        computeCurrentState();
-      }
-      for (State state = currentState; state != null; state = state.next) {
-        code = code * 31 + state.attribute.hashCode();
-      }
+    for (State state = getCurrentState(); state != null; state = state.next) {
+      code = code * 31 + state.attribute.hashCode();
     }
-    
     return code;
   }
   
@@ -413,14 +398,8 @@ public class AttributeSource {
         }
   
         // it is only equal if all attribute impls are the same in the same order
-        if (this.currentState == null) {
-          this.computeCurrentState();
-        }
-        State thisState = this.currentState;
-        if (other.currentState == null) {
-          other.computeCurrentState();
-        }
-        State otherState = other.currentState;
+        State thisState = this.getCurrentState();
+        State otherState = other.getCurrentState();
         while (thisState != null && otherState != null) {
           if (otherState.attribute.getClass() != thisState.attribute.getClass() || !otherState.attribute.equals(thisState.attribute)) {
             return false;
@@ -473,13 +452,8 @@ public class AttributeSource {
    * @see AttributeImpl#reflectWith
    */
   public final void reflectWith(AttributeReflector reflector) {
-    if (hasAttributes()) {
-      if (currentState == null) {
-        computeCurrentState();
-      }
-      for (State state = currentState; state != null; state = state.next) {
-        state.attribute.reflectWith(reflector);
-      }
+    for (State state = getCurrentState(); state != null; state = state.next) {
+      state.attribute.reflectWith(reflector);
     }
   }
 
@@ -495,10 +469,7 @@ public class AttributeSource {
     
     if (hasAttributes()) {
       // first clone the impls
-      if (currentState == null) {
-        computeCurrentState();
-      }
-      for (State state = currentState; state != null; state = state.next) {
+      for (State state = getCurrentState(); state != null; state = state.next) {
         clone.attributeImpls.put(state.attribute.getClass(), (AttributeImpl) state.attribute.clone());
       }
       
@@ -520,18 +491,13 @@ public class AttributeSource {
    * {@link #cloneAttributes} instead of {@link #captureState}.
    */
   public final void copyTo(AttributeSource target) {
-    if (hasAttributes()) {
-      if (currentState == null) {
-        computeCurrentState();
-      }
-      for (State state = currentState; state != null; state = state.next) {
-        final AttributeImpl targetImpl = target.attributeImpls.get(state.attribute.getClass());
-        if (targetImpl == null) {
-          throw new IllegalArgumentException("This AttributeSource contains AttributeImpl of type " +
-            state.attribute.getClass().getName() + " that is not in the target");
-        }
-        state.attribute.copyTo(targetImpl);
+    for (State state = getCurrentState(); state != null; state = state.next) {
+      final AttributeImpl targetImpl = target.attributeImpls.get(state.attribute.getClass());
+      if (targetImpl == null) {
+        throw new IllegalArgumentException("This AttributeSource contains AttributeImpl of type " +
+          state.attribute.getClass().getName() + " that is not in the target");
       }
+      state.attribute.copyTo(targetImpl);
     }
   }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/CodecUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/CodecUtil.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/CodecUtil.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/CodecUtil.java Mon May  2 13:50:57 2011
@@ -18,8 +18,8 @@ package org.apache.lucene.util;
  */
 
 
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexFormatTooNewException;
 import org.apache.lucene.index.IndexFormatTooOldException;
@@ -35,18 +35,16 @@ public final class CodecUtil {
 
   private final static int CODEC_MAGIC = 0x3fd76c17;
 
-  public static IndexOutput writeHeader(IndexOutput out, String codec, int version)
+  public static DataOutput writeHeader(DataOutput out, String codec, int version)
     throws IOException {
-    final long start = out.getFilePointer();
+    BytesRef bytes = new BytesRef(codec);
+    if (bytes.length != codec.length() || bytes.length >= 128) {
+      throw new IllegalArgumentException("codec must be simple ASCII, less than 128 characters in length [got " + codec + "]");
+    }
     out.writeInt(CODEC_MAGIC);
     out.writeString(codec);
     out.writeInt(version);
 
-    // We require this so we can easily pre-compute header length
-    if (out.getFilePointer()-start != codec.length()+9) {
-      throw new IllegalArgumentException("codec must be simple ASCII, less than 128 characters in length [got " + codec + "]");
-    }
-
     return out;
   }
 
@@ -54,7 +52,7 @@ public final class CodecUtil {
     return 9+codec.length();
   }
 
-  public static int checkHeader(IndexInput in, String codec, int minVersion, int maxVersion)
+  public static int checkHeader(DataInput in, String codec, int minVersion, int maxVersion)
     throws IOException {
 
     // Safety to guard against reading a bogus string:

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/PriorityQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/PriorityQueue.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/PriorityQueue.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/PriorityQueue.java Mon May  2 13:50:57 2011
@@ -21,8 +21,10 @@ package org.apache.lucene.util;
  * least element can always be found in constant time.  Put()'s and pop()'s
  * require log(size) time.
  *
- * <p><b>NOTE</b>: This class pre-allocates a full array of
- * length <code>maxSize+1</code>, in {@link #initialize}.
+ * <p><b>NOTE</b>: This class will pre-allocate a full array of
+ * length <code>maxSize+1</code> if instantiated via the
+ * {@link #PriorityQueue(int,boolean)} constructor with
+ * <code>prepopulate</code> set to <code>true</code>.
  * 
  * @lucene.internal
 */
@@ -83,9 +85,10 @@ public abstract class PriorityQueue<T> {
 
   /**
    * This method can be overridden by extending classes to return a sentinel
-   * object which will be used by {@link #initialize(int)} to fill the queue, so
-   * that the code which uses that queue can always assume it's full and only
-   * change the top without attempting to insert any new object.<br>
+   * object which will be used by the {@link PriorityQueue#PriorityQueue(int,boolean)} 
+   * constructor to fill the queue, so that the code which uses that queue can always
+   * assume it's full and only change the top without attempting to insert any new
+   * object.<br>
    * 
    * Those sentinel values should always compare worse than any non-sentinel
    * value (i.e., {@link #lessThan} should always favor the
@@ -111,11 +114,11 @@ public abstract class PriorityQueue<T> {
    * </pre>
    * 
    * <b>NOTE:</b> if this method returns a non-null value, it will be called by
-   * {@link #initialize(int)} {@link #size()} times, relying on a new object to
-   * be returned and will not check if it's null again. Therefore you should
-   * ensure any call to this method creates a new instance and behaves
-   * consistently, e.g., it cannot return null if it previously returned
-   * non-null.
+   * the {@link PriorityQueue#PriorityQueue(int,boolean)} constructor 
+   * {@link #size()} times, relying on a new object to be returned and will not
+   * check if it's null again. Therefore you should ensure any call to this
+   * method creates a new instance and behaves consistently, e.g., it cannot
+   * return null if it previously returned non-null.
    * 
    * @return the sentinel object to use to pre-populate the queue, or null if
    *         sentinel objects are not supported.

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java Mon May  2 13:50:57 2011
@@ -180,7 +180,13 @@ public class Builder<T> {
           compileAllTargets(node);
         }
         final T nextFinalOutput = node.output;
-        final boolean isFinal = node.isFinal;
+
+        // We "fake" the node as being final if it has no
+        // outgoing arcs; in theory we could leave it
+        // as non-final (the FST can represent this), but
+        // FSTEnum, Util, etc., have trouble w/ non-final
+        // dead-end states:
+        final boolean isFinal = node.isFinal || node.numArcs == 0;
 
         if (doCompile) {
           // this node makes it and we now compile it.  first,
@@ -219,7 +225,7 @@ public class Builder<T> {
     add(scratchIntsRef, output);
   }
 
-  /** Sugar: adds the UTF32 chars from char[] slice.  FST
+  /** Sugar: adds the UTF32 codepoints from char[] slice.  FST
    *  must be FST.INPUT_TYPE.BYTE4! */
   public void add(char[] s, int offset, int length, T output) throws IOException {
     assert fst.getInputType() == FST.INPUT_TYPE.BYTE4;
@@ -237,7 +243,7 @@ public class Builder<T> {
     add(scratchIntsRef, output);
   }
 
-  /** Sugar: adds the UTF32 chars from CharSequence.  FST
+  /** Sugar: adds the UTF32 codepoints from CharSequence.  FST
    *  must be FST.INPUT_TYPE.BYTE4! */
   public void add(CharSequence s, T output) throws IOException {
     assert fst.getInputType() == FST.INPUT_TYPE.BYTE4;
@@ -268,6 +274,7 @@ public class Builder<T> {
       // 'finalness' is stored on the incoming arc, not on
       // the node
       frontier[0].inputCount++;
+      frontier[0].isFinal = true;
       fst.setEmptyOutput(output);
       return;
     }
@@ -388,6 +395,10 @@ public class Builder<T> {
       if (!arc.target.isCompiled()) {
         // not yet compiled
         @SuppressWarnings("unchecked") final UnCompiledNode<T> n = (UnCompiledNode<T>) arc.target;
+        if (n.numArcs == 0) {
+          //System.out.println("seg=" + segment + "        FORCE final arc=" + (char) arc.label);
+          arc.isFinal = n.isFinal = true;
+        }
         arc.target = compileNode(n);
       }
     }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java Mon May  2 13:50:57 2011
@@ -21,12 +21,14 @@ import java.io.IOException;
 
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.util.automaton.fst.Builder.UnCompiledNode;
 
+// NOTE: while the FST is able to represent a non-final
+// dead-end state (NON_FINAL_END_NODE=0), the layres above
+// (FSTEnum, Util) have problems with this!!
+
 /** Represents an FST using a compact byte[] format.
  *  <p> The format is similar to what's used by Morfologik
  *  (http://sourceforge.net/projects/morfologik).
@@ -168,7 +170,7 @@ public class FST<T> {
   }
 
   // create an existing FST
-  public FST(IndexInput in, Outputs<T> outputs) throws IOException {
+  public FST(DataInput in, Outputs<T> outputs) throws IOException {
     this.outputs = outputs;
     writer = null;
     CodecUtil.checkHeader(in, FILE_FORMAT_NAME, VERSION_START, VERSION_START);
@@ -216,6 +218,9 @@ public class FST<T> {
   }
 
   void finish(int startNode) {
+    if (startNode == FINAL_END_NODE && emptyOutput != null) {
+      startNode = 0;
+    }
     if (this.startNode != -1) {
       throw new IllegalStateException("already finished");
     }
@@ -250,11 +255,13 @@ public class FST<T> {
     writer.posWrite = posSave;
   }
 
-  public void save(IndexOutput out) throws IOException {
+  public void save(DataOutput out) throws IOException {
     if (startNode == -1) {
       throw new IllegalStateException("call finish first");
     }
     CodecUtil.writeHeader(out, FILE_FORMAT_NAME, VERSION_CURRENT);
+    // TODO: really we should encode this as an arc, arriving
+    // to the root node, instead of special casing here:
     if (emptyOutput != null) {
       out.writeByte((byte) 1);
       out.writeVInt(emptyOutputBytes.length);
@@ -468,7 +475,9 @@ public class FST<T> {
       arc.nextFinalOutput = emptyOutput;
     } else {
       arc.flags = BIT_LAST_ARC;
+      arc.nextFinalOutput = NO_OUTPUT;
     }
+    arc.output = NO_OUTPUT;
 
     // If there are no nodes, ie, the FST only accepts the
     // empty string, then startNode is 0, and then readFirstTargetArc
@@ -585,12 +594,11 @@ public class FST<T> {
    * expanded array format.
    */
   boolean isExpandedTarget(Arc<T> follow) throws IOException {
-    if (follow.isFinal()) {
+    if (!targetHasArcs(follow)) {
       return false;
     } else {
       final BytesReader in = getBytesReader(follow.target);
       final byte b = in.readByte();
-      
       return (b & BIT_ARCS_AS_FIXED_ARRAY) != 0;
     }
   }
@@ -669,8 +677,11 @@ public class FST<T> {
     }
 
     if (arc.flag(BIT_STOP_NODE)) {
-      arc.target = FINAL_END_NODE;
-      arc.flags |= BIT_FINAL_ARC;
+      if (arc.flag(BIT_FINAL_ARC)) {
+        arc.target = FINAL_END_NODE;
+      } else {
+        arc.target = NON_FINAL_END_NODE;
+      }
       arc.nextArc = in.pos;
     } else if (arc.flag(BIT_TARGET_NEXT)) {
       arc.nextArc = in.pos;

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Mon May  2 13:50:57 2011
@@ -19,11 +19,15 @@ package org.apache.lucene.analysis;
 
 import java.io.StringReader;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
  
 import org.apache.lucene.analysis.tokenattributes.*;
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 /** 
  * Base class for all Lucene unit tests that use TokenStreams.  
@@ -117,11 +121,24 @@ public abstract class BaseTokenStreamTes
         assertEquals("type "+i, types[i], typeAtt.type());
       if (posIncrements != null)
         assertEquals("posIncrement "+i, posIncrements[i], posIncrAtt.getPositionIncrement());
+      
+      // we can enforce some basic things about a few attributes even if the caller doesn't check:
+      if (offsetAtt != null) {
+        assertTrue("startOffset must be >= 0", offsetAtt.startOffset() >= 0);
+        assertTrue("endOffset must be >= 0", offsetAtt.endOffset() >= 0);
+        assertTrue("endOffset must be >= startOffset", offsetAtt.endOffset() >= offsetAtt.startOffset());
+      }
+      if (posIncrAtt != null) {
+        assertTrue("posIncrement must be >= 0", posIncrAtt.getPositionIncrement() >= 0);
+      }
     }
     assertFalse("end of stream", ts.incrementToken());
     ts.end();
     if (finalOffset != null)
       assertEquals("finalOffset ", finalOffset.intValue(), offsetAtt.endOffset());
+    if (offsetAtt != null) {
+      assertTrue("finalOffset must be >= 0", offsetAtt.endOffset() >= 0);
+    }
     ts.close();
   }
   
@@ -216,4 +233,39 @@ public abstract class BaseTokenStreamTes
     assertAnalyzesToReuse(a, input, new String[]{expected});
   }
   
+  // simple utility method for blasting tokenstreams with data to make sure they don't do anything crazy
+
+  public static void checkRandomData(Random random, Analyzer a, int iterations) throws IOException {
+    checkRandomData(random, a, iterations, 20);
+  }
+
+  public static void checkRandomData(Random random, Analyzer a, int iterations, int maxWordLength) throws IOException {
+    for (int i = 0; i < iterations; i++) {
+      String text;
+      switch(_TestUtil.nextInt(random, 0, 3)) {
+        case 0: 
+          text = _TestUtil.randomSimpleString(random);
+          break;
+        case 1:
+          text = _TestUtil.randomRealisticUnicodeString(random, maxWordLength);
+          break;
+        default:
+          text = _TestUtil.randomUnicodeString(random, maxWordLength);
+      }
+      
+      TokenStream ts = a.reusableTokenStream("dummy", new StringReader(text));
+      assertTrue("has no CharTermAttribute", ts.hasAttribute(CharTermAttribute.class));
+      CharTermAttribute termAtt = ts.getAttribute(CharTermAttribute.class);
+      List<String> tokens = new ArrayList<String>();
+      ts.reset();
+      while (ts.incrementToken()) {
+        tokens.add(termAtt.toString());
+        // TODO: we could collect offsets etc here for better checking that reset() really works.
+      }
+      ts.close();
+      // verify reusing is "reproducable" and also get the normal tokenstream sanity checks
+      if (!tokens.isEmpty())
+        assertAnalyzesToReuse(a, text, tokens.toArray(new String[tokens.size()]));
+    }
+  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java Mon May  2 13:50:57 2011
@@ -19,10 +19,10 @@ package org.apache.lucene.analysis;
 
 import java.io.IOException;
 import java.io.Reader;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
 
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
-import org.apache.lucene.index.Payload;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 
 /**
@@ -33,67 +33,50 @@ public final class MockAnalyzer extends 
   private final boolean lowerCase;
   private final CharacterRunAutomaton filter;
   private final boolean enablePositionIncrements;
-  private final boolean payload;
   private int positionIncrementGap;
-
-  /**
-   * Calls {@link #MockAnalyzer(CharacterRunAutomaton, boolean, CharacterRunAutomaton, boolean, boolean) 
-   * MockAnalyzer(runAutomaton, lowerCase, filter, enablePositionIncrements, true}).
-   */
-  public MockAnalyzer(CharacterRunAutomaton runAutomaton, boolean lowerCase, CharacterRunAutomaton filter, boolean enablePositionIncrements) {
-    this(runAutomaton, lowerCase, filter, enablePositionIncrements, true);    
-  }
+  private final Random random;
+  private Map<String,Integer> previousMappings = new HashMap<String,Integer>();
 
   /**
    * Creates a new MockAnalyzer.
    * 
+   * @param random Random for payloads behavior
    * @param runAutomaton DFA describing how tokenization should happen (e.g. [a-zA-Z]+)
    * @param lowerCase true if the tokenizer should lowercase terms
    * @param filter DFA describing how terms should be filtered (set of stopwords, etc)
    * @param enablePositionIncrements true if position increments should reflect filtered terms.
-   * @param payload if payloads should be added containing the positions (for testing)
    */
-  public MockAnalyzer(CharacterRunAutomaton runAutomaton, boolean lowerCase, CharacterRunAutomaton filter, boolean enablePositionIncrements, boolean payload) {
+  public MockAnalyzer(Random random, CharacterRunAutomaton runAutomaton, boolean lowerCase, CharacterRunAutomaton filter, boolean enablePositionIncrements) {
+    this.random = random;
     this.runAutomaton = runAutomaton;
     this.lowerCase = lowerCase;
     this.filter = filter;
     this.enablePositionIncrements = enablePositionIncrements;
-    this.payload = payload;
   }
 
   /**
-   * Calls {@link #MockAnalyzer(CharacterRunAutomaton, boolean, CharacterRunAutomaton, boolean, boolean) 
-   * MockAnalyzer(runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false, true}).
+   * Calls {@link #MockAnalyzer(Random, CharacterRunAutomaton, boolean, CharacterRunAutomaton, boolean) 
+   * MockAnalyzer(random, runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false}).
    */
-  public MockAnalyzer(CharacterRunAutomaton runAutomaton, boolean lowerCase) {
-    this(runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false, true);
+  public MockAnalyzer(Random random, CharacterRunAutomaton runAutomaton, boolean lowerCase) {
+    this(random, runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false);
   }
 
-  /**
-   * Calls {@link #MockAnalyzer(CharacterRunAutomaton, boolean, CharacterRunAutomaton, boolean, boolean) 
-   * MockAnalyzer(runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false, payload}).
-   */
-  public MockAnalyzer(CharacterRunAutomaton runAutomaton, boolean lowerCase, boolean payload) {
-    this(runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false, payload);
-  }
-  
   /** 
    * Create a Whitespace-lowercasing analyzer with no stopwords removal.
    * <p>
-   * Calls {@link #MockAnalyzer(CharacterRunAutomaton, boolean, CharacterRunAutomaton, boolean, boolean) 
-   * MockAnalyzer(MockTokenizer.WHITESPACE, true, MockTokenFilter.EMPTY_STOPSET, false, true}).
+   * Calls {@link #MockAnalyzer(Random, CharacterRunAutomaton, boolean, CharacterRunAutomaton, boolean) 
+   * MockAnalyzer(random, MockTokenizer.WHITESPACE, true, MockTokenFilter.EMPTY_STOPSET, false}).
    */
-  public MockAnalyzer() {
-    this(MockTokenizer.WHITESPACE, true);
+  public MockAnalyzer(Random random) {
+    this(random, MockTokenizer.WHITESPACE, true);
   }
 
   @Override
   public TokenStream tokenStream(String fieldName, Reader reader) {
     MockTokenizer tokenizer = new MockTokenizer(reader, runAutomaton, lowerCase);
     TokenFilter filt = new MockTokenFilter(tokenizer, filter, enablePositionIncrements);
-    if (payload){
-      filt = new SimplePayloadFilter(filt, fieldName);
-    }
+    filt = maybePayload(filt, fieldName);
     return filt;
   }
 
@@ -105,15 +88,19 @@ public final class MockAnalyzer extends 
   @Override
   public TokenStream reusableTokenStream(String fieldName, Reader reader)
       throws IOException {
-    SavedStreams saved = (SavedStreams) getPreviousTokenStream();
+    @SuppressWarnings("unchecked") Map<String,SavedStreams> map = (Map) getPreviousTokenStream();
+    if (map == null) {
+      map = new HashMap<String,SavedStreams>();
+      setPreviousTokenStream(map);
+    }
+    
+    SavedStreams saved = map.get(fieldName);
     if (saved == null) {
       saved = new SavedStreams();
       saved.tokenizer = new MockTokenizer(reader, runAutomaton, lowerCase);
       saved.filter = new MockTokenFilter(saved.tokenizer, filter, enablePositionIncrements);
-      if (payload){
-        saved.filter = new SimplePayloadFilter(saved.filter, fieldName);
-      }
-      setPreviousTokenStream(saved);
+      saved.filter = maybePayload(saved.filter, fieldName);
+      map.put(fieldName, saved);
       return saved.filter;
     } else {
       saved.tokenizer.reset(reader);
@@ -122,6 +109,28 @@ public final class MockAnalyzer extends 
     }
   }
   
+  private synchronized TokenFilter maybePayload(TokenFilter stream, String fieldName) {
+    Integer val = previousMappings.get(fieldName);
+    if (val == null) {
+      switch(random.nextInt(3)) {
+        case 0: val = -1; // no payloads
+                break;
+        case 1: val = Integer.MAX_VALUE; // variable length payload
+                break;
+        case 2: val = random.nextInt(12); // fixed length payload
+                break;
+      }
+      previousMappings.put(fieldName, val); // save it so we are consistent for this field
+    }
+    
+    if (val == -1)
+      return stream;
+    else if (val == Integer.MAX_VALUE)
+      return new MockVariableLengthPayloadFilter(random, stream);
+    else
+      return new MockFixedLengthPayloadFilter(random, stream, val);
+  }
+  
   public void setPositionIncrementGap(int positionIncrementGap){
     this.positionIncrementGap = positionIncrementGap;
   }
@@ -131,35 +140,3 @@ public final class MockAnalyzer extends 
     return positionIncrementGap;
   }
 }
-
-final class SimplePayloadFilter extends TokenFilter {
-  String fieldName;
-  int pos;
-  final PayloadAttribute payloadAttr;
-  final CharTermAttribute termAttr;
-
-  public SimplePayloadFilter(TokenStream input, String fieldName) {
-    super(input);
-    this.fieldName = fieldName;
-    pos = 0;
-    payloadAttr = input.addAttribute(PayloadAttribute.class);
-    termAttr = input.addAttribute(CharTermAttribute.class);
-  }
-
-  @Override
-  public boolean incrementToken() throws IOException {
-    if (input.incrementToken()) {
-      payloadAttr.setPayload(new Payload(("pos: " + pos).getBytes()));
-      pos++;
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  public void reset() throws IOException {
-    super.reset();
-    pos = 0;
-  }
-}

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java Mon May  2 13:50:57 2011
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Random;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
@@ -30,6 +31,8 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.search.SimilarityProvider;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
 import static org.apache.lucene.util.LuceneTestCase.TEST_VERSION_CURRENT;
 
 class DocHelper {
@@ -218,9 +221,9 @@ class DocHelper {
    * @param doc
    * @throws IOException
    */ 
-  public static SegmentInfo writeDoc(Directory dir, Document doc) throws IOException
+  public static SegmentInfo writeDoc(Random random, Directory dir, Document doc) throws IOException
   {
-    return writeDoc(dir, new MockAnalyzer(MockTokenizer.WHITESPACE, false), null, doc);
+    return writeDoc(random, dir, new MockAnalyzer(random, MockTokenizer.WHITESPACE, false), null, doc);
   }
 
   /**
@@ -233,8 +236,8 @@ class DocHelper {
    * @param doc
    * @throws IOException
    */ 
-  public static SegmentInfo writeDoc(Directory dir, Analyzer analyzer, SimilarityProvider similarity, Document doc) throws IOException {
-    IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(
+  public static SegmentInfo writeDoc(Random random, Directory dir, Analyzer analyzer, SimilarityProvider similarity, Document doc) throws IOException {
+    IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig( /* LuceneTestCase.newIndexWriterConfig(random, */ 
         TEST_VERSION_CURRENT, analyzer).setSimilarityProvider(similarity));
     //writer.setUseCompoundFile(false);
     writer.addDocument(doc);

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java Mon May  2 13:50:57 2011
@@ -42,6 +42,7 @@ public class RandomIndexWriter implement
   private final Random r;
   int docCount;
   int flushAt;
+  private double flushAtFactor = 1.0;
   private boolean getReaderCalled;
 
   // Randomly calls Thread.yield so we mixup thread scheduling
@@ -67,7 +68,7 @@ public class RandomIndexWriter implement
 
   /** create a RandomIndexWriter with a random config: Uses TEST_VERSION_CURRENT and MockAnalyzer */
   public RandomIndexWriter(Random r, Directory dir) throws IOException {
-    this(r, dir, LuceneTestCase.newIndexWriterConfig(r, LuceneTestCase.TEST_VERSION_CURRENT, new MockAnalyzer()));
+    this(r, dir, LuceneTestCase.newIndexWriterConfig(r, LuceneTestCase.TEST_VERSION_CURRENT, new MockAnalyzer(r)));
   }
   
   /** create a RandomIndexWriter with a random config: Uses TEST_VERSION_CURRENT */
@@ -98,12 +99,20 @@ public class RandomIndexWriter implement
    */
   public void addDocument(Document doc) throws IOException {
     w.addDocument(doc);
+    maybeCommit();
+  }
+
+  private void maybeCommit() throws IOException {
     if (docCount++ == flushAt) {
       if (LuceneTestCase.VERBOSE) {
-        System.out.println("RIW.addDocument: now doing a commit");
+        System.out.println("RIW.add/updateDocument: now doing a commit at docCount=" + docCount);
       }
       w.commit();
-      flushAt += _TestUtil.nextInt(r, 10, 1000);
+      flushAt += _TestUtil.nextInt(r, (int) (flushAtFactor * 10), (int) (flushAtFactor * 1000));
+      if (flushAtFactor < 2e6) {
+        // gradually but exponentially increase time b/w flushes
+        flushAtFactor *= 1.05;
+      }
     }
   }
   
@@ -113,13 +122,7 @@ public class RandomIndexWriter implement
    */
   public void updateDocument(Term t, Document doc) throws IOException {
     w.updateDocument(t, doc);
-    if (docCount++ == flushAt) {
-      if (LuceneTestCase.VERBOSE) {
-        System.out.println("RIW.updateDocument: now doing a commit");
-      }
-      w.commit();
-      flushAt += _TestUtil.nextInt(r, 10, 1000);
-    }
+    maybeCommit();
   }
   
   public void addIndexes(Directory... dirs) throws CorruptIndexException, IOException {
@@ -181,7 +184,7 @@ public class RandomIndexWriter implement
         System.out.println("RIW.getReader: open new reader");
       }
       w.commit();
-      return IndexReader.open(w.getDirectory(), new KeepOnlyLastCommitDeletionPolicy(), r.nextBoolean(), _TestUtil.nextInt(r, 1, 10));
+      return IndexReader.open(w.getDirectory(), new KeepOnlyLastCommitDeletionPolicy(), r.nextBoolean(), _TestUtil.nextInt(r, 1, 10), w.getConfig().getCodecProvider());
     }
   }
 

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java Mon May  2 13:50:57 2011
@@ -120,7 +120,14 @@ public class MockRandomCodec extends Cod
 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-
+    // we pull this before the seed intentionally: because its not consumed at runtime
+    // (the skipInterval is written into postings header)
+    int skipInterval = _TestUtil.nextInt(seedRandom, 2, 10);
+    
+    if (LuceneTestCase.VERBOSE) {
+      System.out.println("MockRandomCodec: skipInterval=" + skipInterval);
+    }
+    
     final long seed = seedRandom.nextLong();
 
     if (LuceneTestCase.VERBOSE) {
@@ -136,12 +143,12 @@ public class MockRandomCodec extends Cod
     PostingsWriterBase postingsWriter;
 
     if (random.nextBoolean()) {
-      postingsWriter = new SepPostingsWriterImpl(state, new MockIntStreamFactory(random));
+      postingsWriter = new SepPostingsWriterImpl(state, new MockIntStreamFactory(random), skipInterval);
     } else {
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: writing Standard postings");
       }
-      postingsWriter = new StandardPostingsWriter(state);
+      postingsWriter = new StandardPostingsWriter(state, skipInterval);
     }
 
     if (random.nextBoolean()) {

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java Mon May  2 13:50:57 2011
@@ -2,13 +2,14 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 import java.util.Random;
+import java.lang.reflect.Method;
 
 import junit.framework.Assert;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.MultiReader;
@@ -166,7 +167,7 @@ public class QueryUtils {
     throws IOException {
     Directory d = new MockDirectoryWrapper(random, new RAMDirectory());
       IndexWriter w = new IndexWriter(d, new IndexWriterConfig(
-        TEST_VERSION_CURRENT, new MockAnalyzer()));
+        TEST_VERSION_CURRENT, new MockAnalyzer(random)));
       for (int i = 0; i < numDeletedDocs; i++) {
         w.addDocument(new Document());
       }
@@ -309,7 +310,7 @@ public class QueryUtils {
           // confirm that skipping beyond the last doc, on the
           // previous reader, hits NO_MORE_DOCS
           final IndexReader previousReader = lastReader[0];
-          IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
+          IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
           Weight w = q.weight(indexSearcher);
           Scorer scorer = w.scorer((AtomicReaderContext)previousReader.getTopReaderContext(), ScorerContext.def());
           if (scorer != null) {

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java Mon May  2 13:50:57 2011
@@ -32,7 +32,9 @@ import java.util.Random;
 import java.util.Set;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.ThrottledIndexOutput;
 import org.apache.lucene.util._TestUtil;
 
 /**
@@ -68,6 +70,7 @@ public class MockDirectoryWrapper extend
   private Set<String> createdFiles;
   Set<String> openFilesForWrite = new HashSet<String>();
   volatile boolean crashed;
+  private ThrottledIndexOutput throttledOutput;
 
   // use this for tracking files for crash.
   // additionally: provides debugging information in case you leave one open
@@ -113,6 +116,10 @@ public class MockDirectoryWrapper extend
   public void setPreventDoubleWrite(boolean value) {
     preventDoubleWrite = value;
   }
+  
+  public void setThrottledIndexOutput(ThrottledIndexOutput throttledOutput) {
+    this.throttledOutput = throttledOutput;
+  }
 
   @Override
   public synchronized void sync(Collection<String> names) throws IOException {
@@ -347,7 +354,7 @@ public class MockDirectoryWrapper extend
     IndexOutput io = new MockIndexOutputWrapper(this, delegate.createOutput(name), name);
     openFileHandles.put(io, new RuntimeException("unclosed IndexOutput"));
     openFilesForWrite.add(name);
-    return io;
+    return throttledOutput == null ? io : throttledOutput.newFromDelegate(io);
   }
 
   @Override
@@ -419,12 +426,30 @@ public class MockDirectoryWrapper extend
       throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open files: " + openFiles, cause);
     }
     open = false;
-    if (checkIndexOnClose && IndexReader.indexExists(this)) {
-      _TestUtil.checkIndex(this);
+    if (checkIndexOnClose) {
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("\nNOTE: MockDirectoryWrapper: now run CheckIndex");
+      } 
+      if (codecProvider != null) {
+        if (IndexReader.indexExists(this, codecProvider)) {
+          _TestUtil.checkIndex(this, codecProvider);
+        }
+      } else {
+        if (IndexReader.indexExists(this)) {
+          _TestUtil.checkIndex(this);
+        }
+      }
     }
     delegate.close();
   }
 
+  private CodecProvider codecProvider;
+
+  // We pass this CodecProvider to checkIndex when dir is closed...
+  public void setCodecProvider(CodecProvider cp) {
+    codecProvider = cp;
+  }
+
   boolean open = true;
   
   public synchronized boolean isOpen() {
@@ -559,4 +584,5 @@ public class MockDirectoryWrapper extend
     maybeYield();
     delegate.copy(to, src, dest);
   }
+  
 }

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/LineFileDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/LineFileDocs.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/LineFileDocs.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/LineFileDocs.java Mon May  2 13:50:57 2011
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.io.BufferedReader;
 import java.io.InputStreamReader;
 import java.io.InputStream;
-import java.io.BufferedInputStream;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.zip.GZIPInputStream;
 import java.util.Random;
@@ -79,8 +78,7 @@ public class LineFileDocs implements Clo
       size *= 2.8;
     }
 
-    final InputStream in = new BufferedInputStream(is, BUFFER_SIZE);
-    reader = new BufferedReader(new InputStreamReader(in, "UTF-8"), BUFFER_SIZE);
+    reader = new BufferedReader(new InputStreamReader(is, "UTF-8"), BUFFER_SIZE);
 
     // Override sizes for currently "known" line files:
     if (path.equals("europarl.lines.txt.gz")) {
@@ -128,7 +126,7 @@ public class LineFileDocs implements Clo
       body = new Field("body", "", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS);
       doc.add(body);
 
-      id = new Field("id", "", Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS);
+      id = new Field("docid", "", Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS);
       doc.add(id);
 
       date = new Field("date", "", Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS);

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java Mon May  2 13:50:57 2011
@@ -116,7 +116,7 @@ public abstract class LuceneTestCase ext
    * If this is set, it is the only method that should run.
    */
   static final String TEST_METHOD;
-  
+
   /** Create indexes in this directory, optimally use a subdir, named after the test */
   public static final File TEMP_DIR;
   static {
@@ -128,6 +128,9 @@ public abstract class LuceneTestCase ext
     TEMP_DIR = new File(s);
     TEMP_DIR.mkdirs();
   }
+  
+  /** set of directories we created, in afterclass we try to clean these up */
+  static final Set<String> tempDirs = Collections.synchronizedSet(new HashSet<String>());
 
   // by default we randomly pick a different codec for
   // each test case (non-J4 tests) and each test class (J4
@@ -142,6 +145,8 @@ public abstract class LuceneTestCase ext
   public static final String TEST_DIRECTORY = System.getProperty("tests.directory", "random");
   /** Get the number of times to run tests */
   public static final int TEST_ITER = Integer.parseInt(System.getProperty("tests.iter", "1"));
+  /** Get the minimum number of times to run tests until a failure happens */
+  public static final int TEST_ITER_MIN = Integer.parseInt(System.getProperty("tests.iter.min", Integer.toString(TEST_ITER)));
   /** Get the random seed for tests */
   public static final String TEST_SEED = System.getProperty("tests.seed", "random");
   /** whether or not nightly tests should run */
@@ -158,11 +163,11 @@ public abstract class LuceneTestCase ext
    * multiply it by the number of iterations
    */
   public static final int RANDOM_MULTIPLIER = Integer.parseInt(System.getProperty("tests.multiplier", "1"));
-  
+
   private int savedBoolMaxClauseCount;
 
   private volatile Thread.UncaughtExceptionHandler savedUncaughtExceptionHandler = null;
-  
+
   /** Used to track if setUp and tearDown are called correctly from subclasses */
   private boolean setup;
 
@@ -184,28 +189,28 @@ public abstract class LuceneTestCase ext
   private static class UncaughtExceptionEntry {
     public final Thread thread;
     public final Throwable exception;
-    
+
     public UncaughtExceptionEntry(Thread thread, Throwable exception) {
       this.thread = thread;
       this.exception = exception;
     }
   }
   private List<UncaughtExceptionEntry> uncaughtExceptions = Collections.synchronizedList(new ArrayList<UncaughtExceptionEntry>());
-  
+
   // saves default codec: we do this statically as many build indexes in @beforeClass
   private static String savedDefaultCodec;
   // default codec: not set when we use a per-field provider.
   private static Codec codec;
   // default codec provider
   private static CodecProvider savedCodecProvider;
-  
+
   private static Locale locale;
   private static Locale savedLocale;
   private static TimeZone timeZone;
   private static TimeZone savedTimeZone;
-  
+
   private static Map<MockDirectoryWrapper,StackTraceElement[]> stores;
-  
+
   private static final String[] TEST_CODECS = new String[] {"MockSep", "MockFixedIntBlock", "MockVariableIntBlock", "MockRandom"};
 
   private static void swapCodec(Codec c, CodecProvider cp) {
@@ -283,7 +288,7 @@ public abstract class LuceneTestCase ext
 
   // randomly picks from core and test codecs
   static String pickRandomCodec(Random rnd) {
-    int idx = rnd.nextInt(CodecProvider.CORE_CODECS.length + 
+    int idx = rnd.nextInt(CodecProvider.CORE_CODECS.length +
                           TEST_CODECS.length);
     if (idx < CodecProvider.CORE_CODECS.length) {
       return CodecProvider.CORE_CODECS[idx];
@@ -316,11 +321,12 @@ public abstract class LuceneTestCase ext
   /** @deprecated (4.0) until we fix no-fork problems in solr tests */
   @Deprecated
   private static List<String> testClassesRun = new ArrayList<String>();
-  
+
   @BeforeClass
   public static void beforeClassLuceneTestCaseJ4() {
     staticSeed = "random".equals(TEST_SEED) ? seedRand.nextLong() : TwoLongs.fromString(TEST_SEED).l1;
     random.setSeed(staticSeed);
+    tempDirs.clear();
     stores = Collections.synchronizedMap(new IdentityHashMap<MockDirectoryWrapper,StackTraceElement[]>());
     savedCodecProvider = CodecProvider.getDefault();
     if ("randomPerField".equals(TEST_CODEC)) {
@@ -341,7 +347,7 @@ public abstract class LuceneTestCase ext
     TimeZone.setDefault(timeZone);
     testsFailed = false;
   }
-  
+
   @AfterClass
   public static void afterClassLuceneTestCaseJ4() {
     if (! "false".equals(TEST_CLEAN_THREADS)) {
@@ -357,12 +363,12 @@ public abstract class LuceneTestCase ext
     if ("randomPerField".equals(TEST_CODEC)) {
       if (cp instanceof RandomCodecProvider)
         codecDescription = cp.toString();
-      else 
+      else
         codecDescription = "PreFlex";
     } else {
       codecDescription = codec.toString();
     }
-    
+
     if (CodecProvider.getDefault() == savedCodecProvider)
       removeTestCodecs(codec, CodecProvider.getDefault());
     CodecProvider.setDefault(savedCodecProvider);
@@ -392,14 +398,14 @@ public abstract class LuceneTestCase ext
     stores = null;
     // if verbose or tests failed, report some information back
     if (VERBOSE || testsFailed)
-      System.err.println("NOTE: test params are: codec=" + codecDescription + 
-        ", locale=" + locale + 
+      System.err.println("NOTE: test params are: codec=" + codecDescription +
+        ", locale=" + locale +
         ", timezone=" + (timeZone == null ? "(null)" : timeZone.getID()));
     if (testsFailed) {
       System.err.println("NOTE: all tests run in this JVM:");
       System.err.println(Arrays.toString(testClassesRun.toArray()));
-      System.err.println("NOTE: " + System.getProperty("os.name") + " " 
-          + System.getProperty("os.version") + " " 
+      System.err.println("NOTE: " + System.getProperty("os.name") + " "
+          + System.getProperty("os.version") + " "
           + System.getProperty("os.arch") + "/"
           + System.getProperty("java.vendor") + " "
           + System.getProperty("java.version") + " "
@@ -409,10 +415,20 @@ public abstract class LuceneTestCase ext
           + "free=" + Runtime.getRuntime().freeMemory() + ","
           + "total=" + Runtime.getRuntime().totalMemory());
     }
+    // clear out any temp directories if we can
+    if (!testsFailed) {
+      for (String path : tempDirs) {
+        try {
+          _TestUtil.rmDir(new File(path));
+        } catch (IOException e) {
+          e.printStackTrace();
+        }
+      }
+    }
   }
 
   private static boolean testsFailed; /* true if any tests failed */
-  
+
   // This is how we get control when errors occur.
   // Think of this as start/end/success/failed
   // events.
@@ -447,7 +463,7 @@ public abstract class LuceneTestCase ext
       LuceneTestCase.this.name = method.getName();
       super.starting(method);
     }
-    
+
   };
 
   @Before
@@ -465,7 +481,7 @@ public abstract class LuceneTestCase ext
           savedUncaughtExceptionHandler.uncaughtException(t, e);
       }
     });
-    
+
     savedBoolMaxClauseCount = BooleanQuery.getMaxClauseCount();
   }
 
@@ -497,7 +513,7 @@ public abstract class LuceneTestCase ext
     if ("perMethod".equals(TEST_CLEAN_THREADS)) {
       int rogueThreads = threadCleanup("test method: '" + getName() + "'");
       if (rogueThreads > 0) {
-        System.err.println("RESOURCE LEAK: test method: '" + getName() 
+        System.err.println("RESOURCE LEAK: test method: '" + getName()
             + "' left " + rogueThreads + " thread(s) running");
         // TODO: fail, but print seed for now.
         if (!testsFailed && uncaughtExceptions.isEmpty()) {
@@ -519,18 +535,18 @@ public abstract class LuceneTestCase ext
         fail("Some threads threw uncaught exceptions!");
       }
 
-      // calling assertSaneFieldCaches here isn't as useful as having test 
-      // classes call it directly from the scope where the index readers 
-      // are used, because they could be gc'ed just before this tearDown 
+      // calling assertSaneFieldCaches here isn't as useful as having test
+      // classes call it directly from the scope where the index readers
+      // are used, because they could be gc'ed just before this tearDown
       // method is called.
       //
       // But it's better then nothing.
       //
-      // If you are testing functionality that you know for a fact 
-      // "violates" FieldCache sanity, then you should either explicitly 
+      // If you are testing functionality that you know for a fact
+      // "violates" FieldCache sanity, then you should either explicitly
       // call purgeFieldCache at the end of your test method, or refactor
-      // your Test class so that the inconsistant FieldCache usages are 
-      // isolated in distinct test methods  
+      // your Test class so that the inconsistant FieldCache usages are
+      // isolated in distinct test methods
       assertSaneFieldCaches(getTestLabel());
 
     } finally {
@@ -541,14 +557,14 @@ public abstract class LuceneTestCase ext
   private final static int THREAD_STOP_GRACE_MSEC = 50;
   // jvm-wide list of 'rogue threads' we found, so they only get reported once.
   private final static IdentityHashMap<Thread,Boolean> rogueThreads = new IdentityHashMap<Thread,Boolean>();
-  
+
   static {
     // just a hack for things like eclipse test-runner threads
     for (Thread t : Thread.getAllStackTraces().keySet()) {
       rogueThreads.put(t, true);
     }
   }
-  
+
   /**
    * Looks for leftover running threads, trying to kill them off,
    * so they don't fail future tests.
@@ -559,20 +575,20 @@ public abstract class LuceneTestCase ext
     Thread[] stillRunning = new Thread[Thread.activeCount()+1];
     int threadCount = 0;
     int rogueCount = 0;
-    
+
     if ((threadCount = Thread.enumerate(stillRunning)) > 1) {
       while (threadCount == stillRunning.length) {
         // truncated response
         stillRunning = new Thread[stillRunning.length*2];
         threadCount = Thread.enumerate(stillRunning);
       }
-      
+
       for (int i = 0; i < threadCount; i++) {
         Thread t = stillRunning[i];
-          
-        if (t.isAlive() && 
-            !rogueThreads.containsKey(t) && 
-            t != Thread.currentThread() && 
+
+        if (t.isAlive() &&
+            !rogueThreads.containsKey(t) &&
+            t != Thread.currentThread() &&
             /* its ok to keep your searcher across test cases */
             (t.getName().startsWith("LuceneTestCase") && context.startsWith("test method")) == false) {
           System.err.println("WARNING: " + context  + " left thread running: " + t);
@@ -597,7 +613,7 @@ public abstract class LuceneTestCase ext
     }
     return rogueCount;
   }
-  
+
   /**
    * Asserts that FieldCacheSanityChecker does not detect any
    * problems with FieldCache.DEFAULT.
@@ -640,13 +656,13 @@ public abstract class LuceneTestCase ext
 
     }
   }
-  
+
   // @deprecated (4.0) These deprecated methods should be removed soon, when all tests using no Epsilon are fixed:
   @Deprecated
   static public void assertEquals(double expected, double actual) {
     assertEquals(null, expected, actual);
   }
-   
+
   @Deprecated
   static public void assertEquals(String message, double expected, double actual) {
     assertEquals(message, Double.valueOf(expected), Double.valueOf(actual));
@@ -661,18 +677,18 @@ public abstract class LuceneTestCase ext
   static public void assertEquals(String message, float expected, float actual) {
     assertEquals(message, Float.valueOf(expected), Float.valueOf(actual));
   }
-  
+
   // Replacement for Assume jUnit class, so we can add a message with explanation:
-  
+
   private static final class TestIgnoredException extends RuntimeException {
     TestIgnoredException(String msg) {
       super(msg);
     }
-    
+
     TestIgnoredException(String msg, Throwable t) {
       super(msg, t);
     }
-    
+
     @Override
     public String getMessage() {
       StringBuilder sb = new StringBuilder(super.getMessage());
@@ -680,7 +696,7 @@ public abstract class LuceneTestCase ext
         sb.append(" - ").append(getCause());
       return sb.toString();
     }
-    
+
     // only this one is called by our code, exception is not used outside this class:
     @Override
     public void printStackTrace(PrintStream s) {
@@ -692,19 +708,19 @@ public abstract class LuceneTestCase ext
       }
     }
   }
-  
+
   public static void assumeTrue(String msg, boolean b) {
     Assume.assumeNoException(b ? null : new TestIgnoredException(msg));
   }
- 
+
   public static void assumeFalse(String msg, boolean b) {
     assumeTrue(msg, !b);
   }
-  
+
   public static void assumeNoException(String msg, Exception e) {
     Assume.assumeNoException(e == null ? null : new TestIgnoredException(msg, e));
   }
- 
+
   public static <T> Set<T> asSet(T... args) {
     return new HashSet<T>(Arrays.asList(args));
   }
@@ -762,13 +778,15 @@ public abstract class LuceneTestCase ext
       c.setTermIndexInterval(_TestUtil.nextInt(r, 1, 1000));
     }
     if (r.nextBoolean()) {
-      c.setMaxThreadStates(_TestUtil.nextInt(r, 1, 20));
+      c.setIndexerThreadPool(new ThreadAffinityDocumentsWriterThreadPool(_TestUtil.nextInt(r, 1, 20)));
     }
 
     if (r.nextBoolean()) {
-      c.setMergePolicy(new MockRandomMergePolicy(r));
-    } else {
+      c.setMergePolicy(newTieredMergePolicy());
+    } else if (r.nextBoolean()) {
       c.setMergePolicy(newLogMergePolicy());
+    } else {
+      c.setMergePolicy(new MockRandomMergePolicy(r));
     }
 
     c.setReaderPooling(r.nextBoolean());
@@ -780,6 +798,10 @@ public abstract class LuceneTestCase ext
     return newLogMergePolicy(random);
   }
 
+  public static TieredMergePolicy newTieredMergePolicy() {
+    return newTieredMergePolicy(random);
+  }
+
   public static LogMergePolicy newLogMergePolicy(Random r) {
     LogMergePolicy logmp = r.nextBoolean() ? new LogDocMergePolicy() : new LogByteSizeMergePolicy();
     logmp.setUseCompoundFile(r.nextBoolean());
@@ -792,17 +814,22 @@ public abstract class LuceneTestCase ext
     return logmp;
   }
 
-  public static LogMergePolicy newInOrderLogMergePolicy() {
-    LogMergePolicy logmp = newLogMergePolicy();
-    logmp.setRequireContiguousMerge(true);
-    return logmp;
-  }
-
-  public static LogMergePolicy newInOrderLogMergePolicy(int mergeFactor) {
-    LogMergePolicy logmp = newLogMergePolicy();
-    logmp.setMergeFactor(mergeFactor);
-    logmp.setRequireContiguousMerge(true);
-    return logmp;
+  public static TieredMergePolicy newTieredMergePolicy(Random r) {
+    TieredMergePolicy tmp = new TieredMergePolicy();
+    if (r.nextInt(3) == 2) {
+      tmp.setMaxMergeAtOnce(2);
+      tmp.setMaxMergeAtOnceExplicit(2);
+    } else {
+      tmp.setMaxMergeAtOnce(_TestUtil.nextInt(r, 2, 20));
+      tmp.setMaxMergeAtOnceExplicit(_TestUtil.nextInt(r, 2, 30));
+    }
+    tmp.setMaxMergedSegmentMB(0.2 + r.nextDouble() * 2.0);
+    tmp.setFloorSegmentMB(0.2 + r.nextDouble() * 2.0);
+    tmp.setExpungeDeletesPctAllowed(0.0 + r.nextDouble() * 30.0);
+    tmp.setSegmentsPerTier(_TestUtil.nextInt(r, 2, 20));
+    tmp.setUseCompoundFile(r.nextBoolean());
+    tmp.setNoCFSRatio(0.1 + r.nextDouble()*0.8);
+    return tmp;
   }
 
   public static LogMergePolicy newLogMergePolicy(boolean useCFS) {
@@ -837,7 +864,7 @@ public abstract class LuceneTestCase ext
   public static MockDirectoryWrapper newDirectory() throws IOException {
     return newDirectory(random);
   }
-  
+
   /**
    * Returns a new Directory instance, using the specified random.
    * See {@link #newDirectory()} for more information.
@@ -848,7 +875,7 @@ public abstract class LuceneTestCase ext
     stores.put(dir, Thread.currentThread().getStackTrace());
     return dir;
   }
-  
+
   /**
    * Returns a new Directory instance, with contents copied from the
    * provided directory. See {@link #newDirectory()} for more
@@ -857,23 +884,23 @@ public abstract class LuceneTestCase ext
   public static MockDirectoryWrapper newDirectory(Directory d) throws IOException {
     return newDirectory(random, d);
   }
-  
+
   /** Returns a new FSDirectory instance over the given file, which must be a folder. */
   public static MockDirectoryWrapper newFSDirectory(File f) throws IOException {
     return newFSDirectory(f, null);
   }
-  
+
   /** Returns a new FSDirectory instance over the given file, which must be a folder. */
   public static MockDirectoryWrapper newFSDirectory(File f, LockFactory lf) throws IOException {
     String fsdirClass = TEST_DIRECTORY;
     if (fsdirClass.equals("random")) {
       fsdirClass = FS_DIRECTORIES[random.nextInt(FS_DIRECTORIES.length)];
     }
-    
+
     if (fsdirClass.indexOf(".") == -1) {// if not fully qualified, assume .store
       fsdirClass = "org.apache.lucene.store." + fsdirClass;
     }
-    
+
     Class<? extends FSDirectory> clazz;
     try {
       try {
@@ -881,11 +908,11 @@ public abstract class LuceneTestCase ext
       } catch (ClassCastException e) {
         // TEST_DIRECTORY is not a sub-class of FSDirectory, so draw one at random
         fsdirClass = FS_DIRECTORIES[random.nextInt(FS_DIRECTORIES.length)];
-        
+
         if (fsdirClass.indexOf(".") == -1) {// if not fully qualified, assume .store
           fsdirClass = "org.apache.lucene.store." + fsdirClass;
         }
-        
+
         clazz = Class.forName(fsdirClass).asSubclass(FSDirectory.class);
       }
       MockDirectoryWrapper dir = new MockDirectoryWrapper(random, newFSDirectoryImpl(clazz, f, lf));
@@ -895,7 +922,7 @@ public abstract class LuceneTestCase ext
       throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * Returns a new Directory instance, using the specified random
    * with contents copied from the provided directory. See 
@@ -953,44 +980,44 @@ public abstract class LuceneTestCase ext
   public static Field newField(Random random, String name, String value, Store store, Index index, TermVector tv) {
     if (!index.isIndexed())
       return new Field(name, value, store, index);
-    
+
     if (!store.isStored() && random.nextBoolean())
       store = Store.YES; // randomly store it
-    
+
     tv = randomTVSetting(random, tv);
-    
+
     return new Field(name, value, store, index, tv);
   }
-  
-  static final TermVector tvSettings[] = { 
-    TermVector.NO, TermVector.YES, TermVector.WITH_OFFSETS, 
-    TermVector.WITH_POSITIONS, TermVector.WITH_POSITIONS_OFFSETS 
+
+  static final TermVector tvSettings[] = {
+    TermVector.NO, TermVector.YES, TermVector.WITH_OFFSETS,
+    TermVector.WITH_POSITIONS, TermVector.WITH_POSITIONS_OFFSETS
   };
-  
+
   private static TermVector randomTVSetting(Random random, TermVector minimum) {
     switch(minimum) {
       case NO: return tvSettings[_TestUtil.nextInt(random, 0, tvSettings.length-1)];
       case YES: return tvSettings[_TestUtil.nextInt(random, 1, tvSettings.length-1)];
-      case WITH_OFFSETS: return random.nextBoolean() ? TermVector.WITH_OFFSETS 
+      case WITH_OFFSETS: return random.nextBoolean() ? TermVector.WITH_OFFSETS
           : TermVector.WITH_POSITIONS_OFFSETS;
-      case WITH_POSITIONS: return random.nextBoolean() ? TermVector.WITH_POSITIONS 
+      case WITH_POSITIONS: return random.nextBoolean() ? TermVector.WITH_POSITIONS
           : TermVector.WITH_POSITIONS_OFFSETS;
       default: return TermVector.WITH_POSITIONS_OFFSETS;
     }
   }
-  
+
   /** return a random Locale from the available locales on the system */
   public static Locale randomLocale(Random random) {
     Locale locales[] = Locale.getAvailableLocales();
     return locales[random.nextInt(locales.length)];
   }
-  
+
   /** return a random TimeZone from the available timezones on the system */
   public static TimeZone randomTimeZone(Random random) {
     String tzIds[] = TimeZone.getAvailableIDs();
     return TimeZone.getTimeZone(tzIds[random.nextInt(tzIds.length)]);
   }
-  
+
   /** return a Locale object equivalent to its programmatic name */
   public static Locale localeForName(String localeName) {
     String elements[] = localeName.split("\\_");
@@ -1012,7 +1039,7 @@ public abstract class LuceneTestCase ext
     "RAMDirectory",
     FS_DIRECTORIES[0], FS_DIRECTORIES[1], FS_DIRECTORIES[2]
   };
-  
+
   public static String randomDirectory(Random random) {
     if (random.nextInt(10) == 0) {
       return CORE_DIRECTORIES[random.nextInt(CORE_DIRECTORIES.length)];
@@ -1024,20 +1051,21 @@ public abstract class LuceneTestCase ext
   private static Directory newFSDirectoryImpl(
       Class<? extends FSDirectory> clazz, File file, LockFactory lockFactory)
       throws IOException {
+    FSDirectory d = null;
     try {
       // Assuming every FSDirectory has a ctor(File), but not all may take a
       // LockFactory too, so setting it afterwards.
       Constructor<? extends FSDirectory> ctor = clazz.getConstructor(File.class);
-      FSDirectory d = ctor.newInstance(file);
-      if (lockFactory != null) {
-        d.setLockFactory(lockFactory);
-      }
-      return d;
+      d = ctor.newInstance(file);
     } catch (Exception e) {
-      return FSDirectory.open(file);
+      d = FSDirectory.open(file);
     }
+    if (lockFactory != null) {
+      d.setLockFactory(lockFactory);
+    }
+    return d;
   }
-  
+
   static Directory newDirectoryImpl(Random random, String clazzName) {
     if (clazzName.equals("random"))
       clazzName = randomDirectory(random);
@@ -1050,6 +1078,7 @@ public abstract class LuceneTestCase ext
         final File tmpFile = File.createTempFile("test", "tmp", TEMP_DIR);
         tmpFile.delete();
         tmpFile.mkdir();
+        tempDirs.add(tmpFile.getAbsolutePath());
         return newFSDirectoryImpl(clazz.asSubclass(FSDirectory.class), tmpFile, null);
       }
 
@@ -1057,18 +1086,31 @@ public abstract class LuceneTestCase ext
       return clazz.newInstance();
     } catch (Exception e) {
       throw new RuntimeException(e);
-    } 
+    }
   }
-  
+
   /** create a new searcher over the reader.
    * This searcher might randomly use threads. */
   public static IndexSearcher newSearcher(IndexReader r) throws IOException {
+    return newSearcher(r, true);
+  }
+  
+  /** create a new searcher over the reader.
+   * This searcher might randomly use threads.
+   * if <code>maybeWrap</code> is true, this searcher might wrap the reader
+   * with one that returns null for getSequentialSubReaders.
+   */
+  public static IndexSearcher newSearcher(IndexReader r, boolean maybeWrap) throws IOException {
     if (random.nextBoolean()) {
-      return new IndexSearcher(r);
+      if (maybeWrap && random.nextBoolean()) {
+        return new IndexSearcher(new SlowMultiReaderWrapper(r));
+      } else {
+        return new IndexSearcher(r);
+      }
     } else {
       int threads = 0;
-      final ExecutorService ex = (random.nextBoolean()) ? null 
-          : Executors.newFixedThreadPool(threads = _TestUtil.nextInt(random, 1, 8), 
+      final ExecutorService ex = (random.nextBoolean()) ? null
+          : Executors.newFixedThreadPool(threads = _TestUtil.nextInt(random, 1, 8),
                       new NamedThreadFactory("LuceneTestCase"));
       if (ex != null && VERBOSE) {
         System.out.println("NOTE: newSearcher using ExecutorService with " + threads + " threads");
@@ -1093,12 +1135,12 @@ public abstract class LuceneTestCase ext
   public String getName() {
     return this.name;
   }
-  
+
   /** Gets a resource from the classpath as {@link File}. This method should only be used,
    * if a real file is needed. To get a stream, code should prefer
    * {@link Class#getResourceAsStream} using {@code this.getClass()}.
    */
-  
+
   protected File getDataFile(String name) throws IOException {
     try {
       return new File(this.getClass().getResource(name).toURI());
@@ -1109,11 +1151,11 @@ public abstract class LuceneTestCase ext
 
   // We get here from InterceptTestCaseEvents on the 'failed' event....
   public void reportAdditionalFailureInfo() {
-    System.err.println("NOTE: reproduce with: ant test -Dtestcase=" + getClass().getSimpleName() 
+    System.err.println("NOTE: reproduce with: ant test -Dtestcase=" + getClass().getSimpleName()
         + " -Dtestmethod=" + getName() + " -Dtests.seed=" + new TwoLongs(staticSeed, seed)
         + reproduceWithExtraParams());
   }
-  
+
   // extra params that were overridden needed to reproduce the command
   private String reproduceWithExtraParams() {
     StringBuilder sb = new StringBuilder();
@@ -1129,12 +1171,12 @@ public abstract class LuceneTestCase ext
   private static long staticSeed;
   // seed for individual test methods, changed in @before
   private long seed;
-  
+
   private static final Random seedRand = new Random();
   protected static final Random random = new Random(0);
 
   private String name = "<unknown>";
-  
+
   /**
    * Annotation for tests that should only be run during nightly builds.
    */
@@ -1142,7 +1184,7 @@ public abstract class LuceneTestCase ext
   @Inherited
   @Retention(RetentionPolicy.RUNTIME)
   public @interface Nightly {}
-  
+
   /** optionally filters the tests to be run by TEST_METHOD */
   public static class LuceneTestCaseRunner extends BlockJUnit4ClassRunner {
     private List<FrameworkMethod> testMethods;
@@ -1172,11 +1214,11 @@ public abstract class LuceneTestCase ext
           testMethods.add(new FrameworkMethod(m));
         }
       }
-      
+
       if (testMethods.isEmpty()) {
         throw new RuntimeException("No runnable methods!");
       }
-      
+
       if (TEST_NIGHTLY == false) {
         if (getTestClass().getJavaClass().isAnnotationPresent(Nightly.class)) {
           /* the test class is annotated with nightly, remove all methods */
@@ -1208,11 +1250,22 @@ public abstract class LuceneTestCase ext
       if (VERBOSE) {
         System.out.println("\nNOTE: running test " + arg0.getName());
       }
+      
+      // only print iteration info if the user requested more than one iterations
+      boolean verbose = VERBOSE && TEST_ITER > 1;
       for (int i = 0; i < TEST_ITER; i++) {
-        if (VERBOSE && TEST_ITER > 1) {
+        if (verbose) {
           System.out.println("\nNOTE: running iter=" + (1+i) + " of " + TEST_ITER);
         }
         super.runChild(arg0, arg1);
+        if (testsFailed) {
+          if (i >= TEST_ITER_MIN - 1) {
+            if (verbose) {
+              System.out.println("\nNOTE: iteration " + i + " failed !");
+            }
+            break;
+          }
+        }
       }
     }
 
@@ -1226,9 +1279,9 @@ public abstract class LuceneTestCase ext
         @Override
         public boolean shouldRun(Description d) {
           return TEST_METHOD == null || d.getMethodName().equals(TEST_METHOD);
-        }     
+        }
       };
-      
+
       try {
         f.apply(this);
       } catch (NoTestsRemainException e) {
@@ -1236,12 +1289,12 @@ public abstract class LuceneTestCase ext
       }
     }
   }
-  
+
   private static class RandomCodecProvider extends CodecProvider {
     private List<Codec> knownCodecs = new ArrayList<Codec>();
     private Map<String,Codec> previousMappings = new HashMap<String,Codec>();
     private final int perFieldSeed;
-    
+
     RandomCodecProvider(Random random) {
       this.perFieldSeed = random.nextInt();
       register(new StandardCodec());
@@ -1273,13 +1326,13 @@ public abstract class LuceneTestCase ext
       }
       return codec.name;
     }
-    
+
     @Override
     public synchronized String toString() {
       return "RandomCodecProvider: " + previousMappings.toString();
     }
   }
-  
+
   @Ignore("just a hack")
   public final void alwaysIgnoredTestMethod() {}
 }

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java Mon May  2 13:50:57 2011
@@ -34,8 +34,6 @@ import java.util.HashMap;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipFile;
 
-import org.junit.Assert;
-
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.index.CheckIndex;
@@ -43,17 +41,22 @@ import org.apache.lucene.index.Concurren
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.LogMergePolicy;
+import org.apache.lucene.index.MergePolicy;
 import org.apache.lucene.index.MergeScheduler;
+import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.store.Directory;
+import org.junit.Assert;
 
 public class _TestUtil {
 
   /** Returns temp dir, containing String arg in its name;
    *  does not create the directory. */
   public static File getTempDir(String desc) {
-    return new File(LuceneTestCase.TEMP_DIR, desc + "." + new Random().nextLong());
+    File f = new File(LuceneTestCase.TEMP_DIR, desc + "." + new Random().nextLong());
+    LuceneTestCase.tempDirs.add(f.getAbsolutePath());
+    return f;
   }
 
   /**
@@ -88,6 +91,7 @@ public class _TestUtil {
     rmDir(destDir);
     
     destDir.mkdir();
+    LuceneTestCase.tempDirs.add(destDir.getAbsolutePath());
     
     while (entries.hasMoreElements()) {
       ZipEntry entry = entries.nextElement();
@@ -157,6 +161,19 @@ public class _TestUtil {
     return start + r.nextInt(end-start+1);
   }
 
+  public static String randomSimpleString(Random r) {
+    final int end = r.nextInt(10);
+    if (end == 0) {
+      // allow 0 length
+      return "";
+    }
+    final char[] buffer = new char[end];
+    for (int i = 0; i < end; i++) {
+      buffer[i] = (char) _TestUtil.nextInt(r, 97, 102);
+    }
+    return new String(buffer, 0, end);
+  }
+
   /** Returns random string, including full unicode range. */
   public static String randomUnicodeString(Random r) {
     return randomUnicodeString(r, 20);
@@ -172,22 +189,35 @@ public class _TestUtil {
       return "";
     }
     final char[] buffer = new char[end];
-    for (int i = 0; i < end; i++) {
-      int t = r.nextInt(5);
+    randomFixedLengthUnicodeString(r, buffer, 0, buffer.length);
+    return new String(buffer, 0, end);
+  }
 
-      if (0 == t && i < end - 1) {
+  /**
+   * Fills provided char[] with valid random unicode code
+   * unit sequence.
+   */
+  public static void randomFixedLengthUnicodeString(Random random, char[] chars, int offset, int length) {
+    int i = offset;
+    final int end = offset + length;
+    while(i < end) {
+      final int t = random.nextInt(5);
+      if (0 == t && i < length - 1) {
         // Make a surrogate pair
         // High surrogate
-        buffer[i++] = (char) nextInt(r, 0xd800, 0xdbff);
+        chars[i++] = (char) nextInt(random, 0xd800, 0xdbff);
         // Low surrogate
-        buffer[i] = (char) nextInt(r, 0xdc00, 0xdfff);
+        chars[i++] = (char) nextInt(random, 0xdc00, 0xdfff);
+      } else if (t <= 1) {
+        chars[i++] = (char) random.nextInt(0x80);
+      } else if (2 == t) {
+        chars[i++] = (char) nextInt(random, 0x80, 0x800);
+      } else if (3 == t) {
+        chars[i++] = (char) nextInt(random, 0x800, 0xd7ff);
+      } else if (4 == t) {
+        chars[i++] = (char) nextInt(random, 0xe000, 0xffff);
       }
-      else if (t <= 1) buffer[i] = (char) r.nextInt(0x80);
-      else if (2 == t) buffer[i] = (char) nextInt(r, 0x80, 0x800);
-      else if (3 == t) buffer[i] = (char) nextInt(r, 0x800, 0xd7ff);
-      else if (4 == t) buffer[i] = (char) nextInt(r, 0xe000, 0xffff);
     }
-    return new String(buffer, 0, end);
   }
 
   private static final int[] blockStarts = {
@@ -325,9 +355,14 @@ public class _TestUtil {
    * count lowish */
   public static void reduceOpenFiles(IndexWriter w) {
     // keep number of open files lowish
-    LogMergePolicy lmp = (LogMergePolicy) w.getConfig().getMergePolicy();
-    lmp.setMergeFactor(Math.min(5, lmp.getMergeFactor()));
-
+    MergePolicy mp = w.getConfig().getMergePolicy();
+    if (mp instanceof LogMergePolicy) {
+      LogMergePolicy lmp = (LogMergePolicy) mp;
+      lmp.setMergeFactor(Math.min(5, lmp.getMergeFactor()));
+    } else if (mp instanceof TieredMergePolicy) {
+      TieredMergePolicy tmp = (TieredMergePolicy) mp;
+      tmp.setMaxMergeAtOnce(Math.min(5, tmp.getMaxMergeAtOnce()));
+    }
     MergeScheduler ms = w.getConfig().getMergeScheduler();
     if (ms instanceof ConcurrentMergeScheduler) {
       ((ConcurrentMergeScheduler) ms).setMaxThreadCount(2);

Modified: lucene/dev/branches/docvalues/lucene/src/test-framework/overview.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/test-framework/overview.html?rev=1098566&r1=1098565&r2=1098566&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/test-framework/overview.html (original)
+++ lucene/dev/branches/docvalues/lucene/src/test-framework/overview.html Mon May  2 13:50:57 2011
@@ -1,28 +1,28 @@
-<html>
-<!--
- 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.
--->
-<head>
-   <title>Apache Lucene Test Framework API</title>
-</head>
-<body>
-<p>
-  The Lucene Test Framework is used by Lucene as the basis for its tests.  
-  The framework can also be used for testing third-party code that uses
-  the Lucene API. 
-</p>
-</body>
-</html>
+<html>
+<!--
+ 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.
+-->
+<head>
+   <title>Apache Lucene Test Framework API</title>
+</head>
+<body>
+<p>
+  The Lucene Test Framework is used by Lucene as the basis for its tests.  
+  The framework can also be used for testing third-party code that uses
+  the Lucene API. 
+</p>
+</body>
+</html>