You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sa...@apache.org on 2011/05/22 23:45:45 UTC

svn commit: r1126234 [11/28] - in /lucene/dev/branches/solr2452: ./ dev-tools/eclipse/ dev-tools/idea/ dev-tools/idea/.idea/ dev-tools/idea/lucene/ dev-tools/idea/lucene/contrib/ant/ dev-tools/idea/lucene/contrib/db/bdb-je/ dev-tools/idea/lucene/contri...

Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java Sun May 22 21:45:19 2011
@@ -172,7 +172,7 @@ public class NIOFSDirectory extends FSDi
         final OutOfMemoryError outOfMemoryError = new OutOfMemoryError(
               "OutOfMemoryError likely caused by the Sun VM Bug described in "
               + "https://issues.apache.org/jira/browse/LUCENE-1566; try calling FSDirectory.setReadChunkSize "
-              + "with a a value smaller than the current chunk size (" + chunkSize + ")");
+              + "with a value smaller than the current chunk size (" + chunkSize + ")");
         outOfMemoryError.initCause(e);
         throw outOfMemoryError;
       }

Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/RAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/RAMDirectory.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/RAMDirectory.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/RAMDirectory.java Sun May 22 21:45:19 2011
@@ -27,8 +27,6 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 
-import org.apache.lucene.util.ThreadInterruptedException;
-
 /**
  * A memory-resident {@link Directory} implementation.  Locking
  * implementation is by default the {@link SingleInstanceLockFactory}
@@ -112,30 +110,6 @@ public class RAMDirectory extends Direct
     return file.getLastModified();
   }
 
-  /** Set the modified time of an existing file to now.
-   * @throws IOException if the file does not exist
-   */
-  @Override
-  public void touchFile(String name) throws IOException {
-    ensureOpen();
-    RAMFile file = fileMap.get(name);
-    if (file == null) {
-      throw new FileNotFoundException(name);
-    }
-    
-    long ts2, ts1 = System.currentTimeMillis();
-    do {
-      try {
-        Thread.sleep(0, 1);
-      } catch (InterruptedException ie) {
-        throw new ThreadInterruptedException(ie);
-      }
-      ts2 = System.currentTimeMillis();
-    } while(ts1 == ts2);
-    
-    file.setLastModified(ts2);
-  }
-
   /** Returns the length in bytes of a file in the directory.
    * @throws IOException if the file does not exist
    */

Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/RAMFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/RAMFile.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/RAMFile.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/RAMFile.java Sun May 22 21:45:19 2011
@@ -26,7 +26,6 @@ public class RAMFile {
   RAMDirectory directory;
   protected long sizeInBytes;
 
-  // This is publicly modifiable via Directory.touchFile(), so direct access not supported
   private long lastModified = System.currentTimeMillis();
 
   // File used as buffer, in no RAMDirectory

Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java Sun May 22 21:45:19 2011
@@ -125,7 +125,7 @@ public class SimpleFSDirectory extends F
           final OutOfMemoryError outOfMemoryError = new OutOfMemoryError(
               "OutOfMemoryError likely caused by the Sun VM Bug described in "
               + "https://issues.apache.org/jira/browse/LUCENE-1566; try calling FSDirectory.setReadChunkSize "
-              + "with a value smaller than the current chunks size (" + chunkSize + ")");
+              + "with a value smaller than the current chunk size (" + chunkSize + ")");
           outOfMemoryError.initCause(e);
           throw outOfMemoryError;
         }

Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/AttributeSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/AttributeSource.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/AttributeSource.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/AttributeSource.java Sun May 22 21:45:19 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/solr2452/lucene/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/BytesRef.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/BytesRef.java Sun May 22 21:45:19 2011
@@ -19,9 +19,6 @@ package org.apache.lucene.util;
 
 import java.util.Comparator;
 import java.io.UnsupportedEncodingException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.io.IOException;
 
 /** Represents byte[], as a slice (offset + length) into an
  *  existing byte[].
@@ -192,6 +189,9 @@ public final class BytesRef implements C
 
   @Override
   public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
     return this.bytesEquals((BytesRef) other);
   }
 

Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/CodecUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/CodecUtil.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/CodecUtil.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/CodecUtil.java Sun May 22 21:45:19 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/solr2452/lucene/src/java/org/apache/lucene/util/SorterTemplate.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/SorterTemplate.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/SorterTemplate.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/SorterTemplate.java Sun May 22 21:45:19 2011
@@ -62,13 +62,26 @@ public abstract class SorterTemplate {
 
   /** Sorts via in-place, but unstable, QuickSort algorithm.
    * For small collections falls back to {@link #insertionSort(int,int)}. */
-  public final void quickSort(int lo, int hi) {
+  public final void quickSort(final int lo, final int hi) {
+    if (hi <= lo) return;
+    // from Integer's Javadocs: ceil(log2(x)) = 32 - numberOfLeadingZeros(x - 1)
+    quickSort(lo, hi, (Integer.SIZE - Integer.numberOfLeadingZeros(hi - lo)) << 1);
+  }
+  
+  private void quickSort(int lo, int hi, int maxDepth) {
+    // fall back to insertion when array has short length
     final int diff = hi - lo;
     if (diff <= QUICKSORT_THRESHOLD) {
       insertionSort(lo, hi);
       return;
     }
     
+    // fall back to merge sort when recursion depth gets too big
+    if (--maxDepth == 0) {
+      mergeSort(lo, hi);
+      return;
+    }
+    
     final int mid = lo + (diff >>> 1);
     
     if (compare(lo, mid) > 0) {
@@ -101,8 +114,8 @@ public abstract class SorterTemplate {
       }
     }
 
-    quickSort(lo, left);
-    quickSort(left + 1, hi);
+    quickSort(lo, left, maxDepth);
+    quickSort(left + 1, hi, maxDepth);
   }
   
   /** Sorts via stable in-place MergeSort algorithm

Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/StringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/StringHelper.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/StringHelper.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/StringHelper.java Sun May 22 21:45:19 2011
@@ -1,5 +1,8 @@
 package org.apache.lucene.util;
 
+import java.util.Comparator;
+import java.util.StringTokenizer;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -54,4 +57,42 @@ public abstract class StringHelper {
 
   private StringHelper() {
   }
+  
+  /**
+   * @return a Comparator over versioned strings such as X.YY.Z
+   * @lucene.internal
+   */
+  public static Comparator<String> getVersionComparator() {
+    return versionComparator;
+  }
+  
+  private static Comparator<String> versionComparator = new Comparator<String>() {
+    public int compare(String a, String b) {
+      StringTokenizer aTokens = new StringTokenizer(a, ".");
+      StringTokenizer bTokens = new StringTokenizer(b, ".");
+      
+      while (aTokens.hasMoreTokens()) {
+        int aToken = Integer.parseInt(aTokens.nextToken());
+        if (bTokens.hasMoreTokens()) {
+          int bToken = Integer.parseInt(bTokens.nextToken());
+          if (aToken != bToken) {
+            return aToken - bToken;
+          }
+        } else {
+          // a has some extra trailing tokens. if these are all zeroes, thats ok.
+          if (aToken != 0) {
+            return 1; 
+          }
+        }
+      }
+      
+      // b has some extra trailing tokens. if these are all zeroes, thats ok.
+      while (bTokens.hasMoreTokens()) {
+        if (Integer.parseInt(bTokens.nextToken()) != 0)
+          return -1;
+      }
+      
+      return 0;
+    }
+  };
 }

Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/LevenshteinAutomata.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/LevenshteinAutomata.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/LevenshteinAutomata.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/LevenshteinAutomata.java Sun May 22 21:45:19 2011
@@ -143,13 +143,16 @@ public class LevenshteinAutomata {
       if (dest >= 0)
         for (int r = 0; r < numRanges; r++)
           states[k].addTransition(new Transition(rangeLower[r], rangeUpper[r], states[dest]));      
-      // reduce the state: this doesn't appear to help anything
-      //states[k].reduce();
     }
 
     Automaton a = new Automaton(states[0]);
     a.setDeterministic(true);
-    a.setNumberedStates(states);
+    // we create some useless unconnected states, and its a net-win overall to remove these,
+    // as well as to combine any adjacent transitions (it makes later algorithms more efficient).
+    // so, while we could set our numberedStates here, its actually best not to, and instead to
+    // force a traversal in reduce, pruning the unconnected states while we combine adjacent transitions.
+    //a.setNumberedStates(states);
+    a.reduce();
     // we need not trim transitions to dead states, as they are not created.
     //a.restoreInvariant();
     return a;

Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java Sun May 22 21:45:19 2011
@@ -30,6 +30,8 @@
 package org.apache.lucene.util.automaton;
 
 import java.util.BitSet;
+import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.LinkedList;
 
 /**
@@ -72,8 +74,12 @@ final public class MinimizationOperation
     final int[] sigma = a.getStartPoints();
     final State[] states = a.getNumberedStates();
     final int sigmaLen = sigma.length, statesLen = states.length;
-    final BitSet[][] reverse = new BitSet[statesLen][sigmaLen];
-    final BitSet[] splitblock = new BitSet[statesLen], partition = new BitSet[statesLen];
+    @SuppressWarnings("unchecked") final ArrayList<State>[][] reverse =
+      (ArrayList<State>[][]) new ArrayList[statesLen][sigmaLen];
+    @SuppressWarnings("unchecked") final HashSet<State>[] partition =
+      (HashSet<State>[]) new HashSet[statesLen];
+    @SuppressWarnings("unchecked") final ArrayList<State>[] splitblock =
+      (ArrayList<State>[]) new ArrayList[statesLen];
     final int[] block = new int[statesLen];
     final StateList[][] active = new StateList[statesLen][sigmaLen];
     final StateListNode[][] active2 = new StateListNode[statesLen][sigmaLen];
@@ -82,8 +88,8 @@ final public class MinimizationOperation
     final BitSet split = new BitSet(statesLen), 
       refine = new BitSet(statesLen), refine2 = new BitSet(statesLen);
     for (int q = 0; q < statesLen; q++) {
-      splitblock[q] = new BitSet(statesLen);
-      partition[q] = new BitSet(statesLen);
+      splitblock[q] = new ArrayList<State>();
+      partition[q] = new HashSet<State>();
       for (int x = 0; x < sigmaLen; x++) {
         active[q][x] = new StateList();
       }
@@ -92,23 +98,22 @@ final public class MinimizationOperation
     for (int q = 0; q < statesLen; q++) {
       final State qq = states[q];
       final int j = qq.accept ? 0 : 1;
-      partition[j].set(q);
+      partition[j].add(qq);
       block[q] = j;
       for (int x = 0; x < sigmaLen; x++) {
-        final BitSet[] r =
+        final ArrayList<State>[] r =
           reverse[qq.step(sigma[x]).number];
         if (r[x] == null)
-          r[x] = new BitSet();
-        r[x].set(q);
+          r[x] = new ArrayList<State>();
+        r[x].add(qq);
       }
     }
     // initialize active sets
     for (int j = 0; j <= 1; j++) {
-      final BitSet part = partition[j];
       for (int x = 0; x < sigmaLen; x++) {
-        for (int i = part.nextSetBit(0); i >= 0; i = part.nextSetBit(i+1)) {
-          if (reverse[i][x] != null)
-            active2[i][x] = active[j][x].add(states[i]);
+        for (final State qq : partition[j]) {
+          if (reverse[qq.number][x] != null)
+            active2[qq.number][x] = active[j][x].add(qq);
         }
       }
     }
@@ -121,18 +126,19 @@ final public class MinimizationOperation
     // process pending until fixed point
     int k = 2;
     while (!pending.isEmpty()) {
-      IntPair ip = pending.removeFirst();
+      final IntPair ip = pending.removeFirst();
       final int p = ip.n1;
       final int x = ip.n2;
       pending2.clear(x*statesLen + p);
       // find states that need to be split off their blocks
       for (StateListNode m = active[p][x].first; m != null; m = m.next) {
-        final BitSet r = reverse[m.q.number][x];
-        if (r != null) for (int i = r.nextSetBit(0); i >= 0; i = r.nextSetBit(i+1)) {
+        final ArrayList<State> r = reverse[m.q.number][x];
+        if (r != null) for (final State s : r) {
+          final int i = s.number;
           if (!split.get(i)) {
             split.set(i);
             final int j = block[i];
-            splitblock[j].set(i);
+            splitblock[j].add(s);
             if (!refine2.get(j)) {
               refine2.set(j);
               refine.set(j);
@@ -142,18 +148,19 @@ final public class MinimizationOperation
       }
       // refine blocks
       for (int j = refine.nextSetBit(0); j >= 0; j = refine.nextSetBit(j+1)) {
-        final BitSet sb = splitblock[j];
-        if (sb.cardinality() < partition[j].cardinality()) {
-          final BitSet b1 = partition[j], b2 = partition[k];
-          for (int i = sb.nextSetBit(0); i >= 0; i = sb.nextSetBit(i+1)) {
-            b1.clear(i);
-            b2.set(i);
-            block[i] = k;
+        final ArrayList<State> sb = splitblock[j];
+        if (sb.size() < partition[j].size()) {
+          final HashSet<State> b1 = partition[j];
+          final HashSet<State> b2 = partition[k];
+          for (final State s : sb) {
+            b1.remove(s);
+            b2.add(s);
+            block[s.number] = k;
             for (int c = 0; c < sigmaLen; c++) {
-              final StateListNode sn = active2[i][c];
+              final StateListNode sn = active2[s.number][c];
               if (sn != null && sn.sl == active[j][c]) {
                 sn.remove();
-                active2[i][c] = active[k][c].add(states[i]);
+                active2[s.number][c] = active[k][c].add(s);
               }
             }
           }
@@ -173,8 +180,8 @@ final public class MinimizationOperation
           k++;
         }
         refine2.clear(j);
-        for (int i = sb.nextSetBit(0); i >= 0; i = sb.nextSetBit(i+1))
-          split.clear(i);
+        for (final State s : sb)
+          split.clear(s.number);
         sb.clear();
       }
       refine.clear();
@@ -184,9 +191,7 @@ final public class MinimizationOperation
     for (int n = 0; n < newstates.length; n++) {
       final State s = new State();
       newstates[n] = s;
-      BitSet part = partition[n];
-      for (int i = part.nextSetBit(0); i >= 0; i = part.nextSetBit(i+1)) {
-        final State q = states[i];
+      for (State q : partition[n]) {
         if (q == a.initial) a.initial = s;
         s.accept = q.accept;
         s.number = q.number; // select representative

Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java Sun May 22 21:45:19 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;
@@ -255,9 +261,12 @@ public class Builder<T> {
     add(scratchIntsRef, output);
   }
 
+  /** It's OK to add the same input twice in a row with
+   *  different outputs, as long as outputs impls the merge
+   *  method. */
   public void add(IntsRef input, T output) throws IOException {
     //System.out.println("\nFST ADD: input=" + input + " output=" + fst.outputs.outputToString(output));
-    assert lastInput.length == 0 || input.compareTo(lastInput) > 0: "inputs are added out of order lastInput=" + lastInput + " vs input=" + input;
+    assert lastInput.length == 0 || input.compareTo(lastInput) >= 0: "inputs are added out of order lastInput=" + lastInput + " vs input=" + input;
     assert validOutput(output);
 
     //System.out.println("\nadd: " + input);
@@ -268,6 +277,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;
     }
@@ -340,8 +350,15 @@ public class Builder<T> {
       assert validOutput(output);
     }
 
-    // push remaining output:
-    frontier[prefixLenPlus1-1].setLastOutput(input.ints[input.offset + prefixLenPlus1-1], output);
+    if (lastInput.length == input.length && prefixLenPlus1 == 1+input.length) {
+      // same input more than 1 time in a row, mapping to
+      // multiple outputs
+      lastNode.output = fst.outputs.merge(lastNode.output, output);
+    } else {
+      // this new arc is private to this new input; set its
+      // arc output to the leftover output:
+      frontier[prefixLenPlus1-1].setLastOutput(input.ints[input.offset + prefixLenPlus1-1], output);
+    }
 
     // save last input
     lastInput.copy(input);
@@ -388,6 +405,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/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java Sun May 22 21:45:19 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");
     }
@@ -226,10 +231,11 @@ public class FST<T> {
   }
 
   void setEmptyOutput(T v) throws IOException {
-    if (emptyOutput != null && !emptyOutput.equals(v)) {
-      throw new IllegalStateException("empty output is already set: " + outputs.outputToString(emptyOutput) + " vs " + outputs.outputToString(v));
+    if (emptyOutput != null) {
+      emptyOutput = outputs.merge(emptyOutput, v);
+    } else {
+      emptyOutput = v;
     }
-    emptyOutput = v;
 
     // TODO: this is messy -- replace with sillyBytesWriter; maybe make
     // bytes private
@@ -250,11 +256,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);
@@ -439,25 +447,17 @@ public class FST<T> {
     // reverse bytes in-place; we do this so that the
     // "BIT_TARGET_NEXT" opto can work, ie, it reads the
     // node just before the current one
-    final int endAddress = writer.posWrite;
-    final int stopAt = (endAddress - startAddress)/2;
-    int upto = 0;
-    while (upto < stopAt) {
-      final byte b = bytes[startAddress+upto];
-      bytes[startAddress+upto] = bytes[endAddress-upto-1];
-      bytes[endAddress-upto-1] = b;
-      upto++;
-    }
+    final int endAddress = lastFrozenNode = writer.posWrite - 1;
 
-    lastFrozenNode = endAddress - 1;
-    /*
-    System.out.println("  return node addr=" + (endAddress-1));
-    for(int i=endAddress-1;i>=startAddress;i--) {
-      System.out.println("    bytes[" + i + "]=" + bytes[i]);
+    int left = startAddress;
+    int right = endAddress;
+    while (left < right) {
+      final byte b = bytes[left];
+      bytes[left++] = bytes[right];
+      bytes[right--] = b;
     }
-    */
 
-    return endAddress-1;
+    return endAddress;
   }
 
   /** Fills virtual 'start' arc, ie, an empty incoming arc to
@@ -468,7 +468,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 +587,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 +670,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/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java Sun May 22 21:45:19 2011
@@ -140,7 +140,7 @@ abstract class FSTEnum<T> {
         // Arcs are fixed array -- use binary search to find
         // the target.
 
-        final FST.BytesReader in = fst.getBytesReader(0);
+        final FST<T>.BytesReader in = fst.getBytesReader(0);
         int low = arc.arcIdx;
         int high = arc.numArcs-1;
         int mid = 0;
@@ -278,7 +278,7 @@ abstract class FSTEnum<T> {
         // Arcs are fixed array -- use binary search to find
         // the target.
 
-        final FST.BytesReader in = fst.getBytesReader(0);
+        final FST<T>.BytesReader in = fst.getBytesReader(0);
         int low = arc.arcIdx;
         int high = arc.numArcs-1;
         int mid = 0;

Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java Sun May 22 21:45:19 2011
@@ -40,7 +40,7 @@ final class NodeHash<T> {
       return false;
     }
     for(int arcUpto=0;arcUpto<node.numArcs;arcUpto++) {
-      final Builder.Arc arc = node.arcs[arcUpto];
+      final Builder.Arc<T> arc = node.arcs[arcUpto];
       if (arc.label != scratchArc.label ||
           !arc.output.equals(scratchArc.output) ||
           ((Builder.CompiledNode) arc.target).address != scratchArc.target ||

Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/Outputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/Outputs.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/Outputs.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/Outputs.java Sun May 22 21:45:19 2011
@@ -54,4 +54,8 @@ public abstract class Outputs<T> {
   public abstract T getNoOutput();
 
   public abstract String outputToString(T output);
+
+  public T merge(T first, T second) {
+    throw new UnsupportedOperationException();
+  }
 }

Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/PairOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/PairOutputs.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/PairOutputs.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/PairOutputs.java Sun May 22 21:45:19 2011
@@ -43,7 +43,7 @@ public class PairOutputs<A,B> extends Ou
       this.output2 = output2;
     }
 
-    @Override @SuppressWarnings("unchecked")
+    @Override @SuppressWarnings("rawtypes")
     public boolean equals(Object other) {
       if (other == this) {
         return true;

Modified: lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/PositiveIntOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/PositiveIntOutputs.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/PositiveIntOutputs.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/java/org/apache/lucene/util/automaton/fst/PositiveIntOutputs.java Sun May 22 21:45:19 2011
@@ -22,14 +22,11 @@ import java.io.IOException;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 
-// TODO: make a sharing and non-sharing variant; eg if you
-// output docFreq per term the FST will be smaller if you
-// don't share since they are not "well shared"
-
 /**
  * Output is a long, for each input term.  NOTE: the
  * resulting FST is not guaranteed to be minimal!  See
- * {@link Builder}.
+ * {@link Builder}.  You cannot store 0 output with this
+ * (that's reserved to mean "no output")!
  * @lucene.experimental
  */
 

Modified: lucene/dev/branches/solr2452/lucene/src/site/src/documentation/content/xdocs/fileformats.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/site/src/documentation/content/xdocs/fileformats.xml?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/site/src/documentation/content/xdocs/fileformats.xml (original)
+++ lucene/dev/branches/solr2452/lucene/src/site/src/documentation/content/xdocs/fileformats.xml Sun May 22 21:45:19 2011
@@ -90,6 +90,14 @@
         <p>
             In version 3.1, segments records the code version
             that created them. See LUCENE-2720 for details.
+            
+            Additionally segments track explicitly whether or
+            not they have term vectors. See LUCENE-2811 for details.
+           </p>
+        <p>
+            In version 3.2, numeric fields are written as natively
+            to stored fields file, previously they were stored in
+            text format only.
            </p>
         </section>
 
@@ -935,7 +943,7 @@
                     <b>3.1</b>
                     Segments --&gt; Format, Version, NameCounter, SegCount, &lt;SegVersion, SegName, SegSize, DelGen, DocStoreOffset, [DocStoreSegment, DocStoreIsCompoundFile], HasSingleNormFile, NumField,
                     NormGen<sup>NumField</sup>,
-                    IsCompoundFile, DeletionCount, HasProx, Diagnostics&gt;<sup>SegCount</sup>, CommitUserData, Checksum
+                    IsCompoundFile, DeletionCount, HasProx, Diagnostics, HasVectors&gt;<sup>SegCount</sup>, CommitUserData, Checksum
                 </p>
 
                 <p>
@@ -957,7 +965,7 @@
 
                 <p>
                     IsCompoundFile, HasSingleNormFile,
-                    DocStoreIsCompoundFile, HasProx --&gt; Int8
+                    DocStoreIsCompoundFile, HasProx, HasVectors --&gt; Int8
                 </p>
 
 		<p>
@@ -1083,6 +1091,10 @@
 		    Lucene version, OS, Java version, why the segment
 		    was created (merge, flush, addIndexes), etc.
                 </p>
+         
+        <p> HasVectors is 1 if this segment stores term vectors,
+            else it's 0.
+                </p>
 
             </section>
 
@@ -1293,10 +1305,18 @@
                                 <li>third bit is one for fields with compression option enabled
                                     (if compression is enabled, the algorithm used is ZLIB),
                                     only available for indexes until Lucene version 2.9.x</li>
+                                <li>4th to 6th bits (mask: 0x7&lt;&lt;3) define the type of a
+                                numeric field: <ul>
+                                  <li>all bits in mask are cleared if no numeric field at all</li>
+                                  <li>1&lt;&lt;3: Value is Int</li>
+                                  <li>2&lt;&lt;3: Value is Long</li>
+                                  <li>3&lt;&lt;3: Value is Int as Float (as of Integer.intBitsToFloat)</li>
+                                  <li>4&lt;&lt;3: Value is Long as Double (as of Double.longBitsToDouble)</li>
+                                </ul></li>
                             </ul>
                         </p>
                         <p>Value --&gt;
-                            String | BinaryValue (depending on Bits)
+                            String | BinaryValue | Int | Long (depending on Bits)
                         </p>
                         <p>BinaryValue --&gt;
                             ValueSize, &lt;Byte&gt;^ValueSize

Modified: lucene/dev/branches/solr2452/lucene/src/site/src/documentation/content/xdocs/gettingstarted.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/site/src/documentation/content/xdocs/gettingstarted.xml?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/site/src/documentation/content/xdocs/gettingstarted.xml (original)
+++ lucene/dev/branches/solr2452/lucene/src/site/src/documentation/content/xdocs/gettingstarted.xml Sun May 22 21:45:19 2011
@@ -28,11 +28,11 @@ may wish to skip sections.
 
 <ul>
 	<li><a href="demo.html">About the command-line Lucene demo and its usage</a>.  This section
-	is intended for anyone who wants to use the command-line Lucene demo.</li> <p/>
+	is intended for anyone who wants to use the command-line Lucene demo.</li>
 
 	<li><a href="demo2.html">About the sources and implementation for the command-line Lucene
 	demo</a>.  This section walks through the implementation details (sources) of the
-	command-line Lucene demo.  This section is intended for developers.</li> <p/>
+	command-line Lucene demo.  This section is intended for developers.</li>
 </ul>
 </section>
 

Modified: lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Sun May 22 21:45:19 2011
@@ -19,14 +19,26 @@ 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.  
+ * Base class for all Lucene unit tests that use TokenStreams. 
+ * <p>
+ * When writing unit tests for analysis components, its highly recommended
+ * to use the helper methods here (especially in conjunction with {@link MockAnalyzer} or
+ * {@link MockTokenizer}), as they contain many assertions and checks to 
+ * catch bugs.
+ * 
+ * @see MockAnalyzer
+ * @see MockTokenizer
  */
 public abstract class BaseTokenStreamTestCase extends LuceneTestCase {
   // some helpers to test Analyzers and TokenStreams:
@@ -117,11 +129,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 +241,40 @@ 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.end();
+      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/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java Sun May 22 21:45:19 2011
@@ -19,81 +19,79 @@ 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;
 
 /**
  * Analyzer for testing
+ * <p>
+ * This analyzer is a replacement for Whitespace/Simple/KeywordAnalyzers
+ * for unit tests. If you are testing a custom component such as a queryparser
+ * or analyzer-wrapper that consumes analysis streams, its a great idea to test
+ * it with this analyzer instead. MockAnalyzer has the following behavior:
+ * <ul>
+ *   <li>By default, the assertions in {@link MockTokenizer} are turned on for extra
+ *       checks that the consumer is consuming properly. These checks can be disabled
+ *       with {@link #setEnableChecks(boolean)}.
+ *   <li>Payload data is randomly injected into the stream for more thorough testing
+ *       of payloads.
+ * </ul>
+ * @see MockTokenizer
  */
 public final class MockAnalyzer extends Analyzer { 
   private final CharacterRunAutomaton runAutomaton;
   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>();
+  private boolean enableChecks = true;
 
   /**
    * 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);
+    tokenizer.setEnableChecks(enableChecks);
     TokenFilter filt = new MockTokenFilter(tokenizer, filter, enablePositionIncrements);
-    if (payload){
-      filt = new SimplePayloadFilter(filt, fieldName);
-    }
+    filt = maybePayload(filt, fieldName);
     return filt;
   }
 
@@ -105,23 +103,49 @@ 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.tokenizer.setEnableChecks(enableChecks);
       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);
-      saved.filter.reset();
       return saved.filter;
     }
   }
   
+  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;
   }
@@ -130,36 +154,12 @@ public final class MockAnalyzer extends 
   public int getPositionIncrementGap(String fieldName){
     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;
+  
+  /** 
+   * Toggle consumer workflow checking: if your test consumes tokenstreams normally you
+   * should leave this enabled.
+   */
+  public void setEnableChecks(boolean enableChecks) {
+    this.enableChecks = enableChecks;
   }
 }

Modified: lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/MockPayloadAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/MockPayloadAnalyzer.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/MockPayloadAnalyzer.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/MockPayloadAnalyzer.java Sun May 22 21:45:19 2011
@@ -86,6 +86,7 @@ final class MockPayloadFilter extends To
 
   @Override
   public void reset() throws IOException {
+    super.reset();
     i = 0;
     pos = 0;
   }

Modified: lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java Sun May 22 21:45:19 2011
@@ -20,14 +20,25 @@ package org.apache.lucene.analysis;
 import java.io.IOException;
 import java.io.Reader;
 
-import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.util.AttributeSource.AttributeFactory;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
 
 /**
- * Automaton-based tokenizer for testing. Optionally lowercases.
+ * Tokenizer for testing.
+ * <p>
+ * This tokenizer is a replacement for {@link #WHITESPACE}, {@link #SIMPLE}, and {@link #KEYWORD}
+ * tokenizers. If you are writing a component such as a TokenFilter, its a great idea to test
+ * it wrapping this tokenizer instead for extra checks. This tokenizer has the following behavior:
+ * <ul>
+ *   <li>An internal state-machine is used for checking consumer consistency. These checks can
+ *       be disabled with {@link #setEnableChecks(boolean)}.
+ *   <li>For convenience, optionally lowercases terms that it outputs.
+ * </ul>
  */
-public class MockTokenizer extends CharTokenizer {
+public class MockTokenizer extends Tokenizer {
   /** Acts Similar to WhitespaceTokenizer */
   public static final CharacterRunAutomaton WHITESPACE = 
     new CharacterRunAutomaton(new RegExp("[^ \t\r\n]+").toAutomaton());
@@ -43,23 +54,96 @@ public class MockTokenizer extends CharT
 
   private final CharacterRunAutomaton runAutomaton;
   private final boolean lowerCase;
+  private final int maxTokenLength;
+  public static final int DEFAULT_MAX_TOKEN_LENGTH = Integer.MAX_VALUE;
   private int state;
 
-  public MockTokenizer(AttributeFactory factory, Reader input, CharacterRunAutomaton runAutomaton, boolean lowerCase) {
-    super(LuceneTestCase.TEST_VERSION_CURRENT, factory, input);
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+  int off = 0;
+
+  // TODO: "register" with LuceneTestCase to ensure all streams are closed() ?
+  // currently, we can only check that the lifecycle is correct if someone is reusing,
+  // but not for "one-offs".
+  private static enum State { 
+    SETREADER,       // consumer set a reader input either via ctor or via reset(Reader)
+    RESET,           // consumer has called reset()
+    INCREMENT,       // consumer is consuming, has called incrementToken() == true
+    INCREMENT_FALSE, // consumer has called incrementToken() which returned false
+    END,             // consumer has called end() to perform end of stream operations
+    CLOSE            // consumer has called close() to release any resources
+  };
+  
+  private State streamState = State.CLOSE;
+  private boolean enableChecks = true;
+  
+  public MockTokenizer(AttributeFactory factory, Reader input, CharacterRunAutomaton runAutomaton, boolean lowerCase, int maxTokenLength) {
+    super(factory, input);
     this.runAutomaton = runAutomaton;
     this.lowerCase = lowerCase;
     this.state = runAutomaton.getInitialState();
+    this.streamState = State.SETREADER;
+    this.maxTokenLength = maxTokenLength;
+  }
+
+  public MockTokenizer(Reader input, CharacterRunAutomaton runAutomaton, boolean lowerCase, int maxTokenLength) {
+    this(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY, input, runAutomaton, lowerCase, maxTokenLength);
   }
 
   public MockTokenizer(Reader input, CharacterRunAutomaton runAutomaton, boolean lowerCase) {
-    super(LuceneTestCase.TEST_VERSION_CURRENT, input);
-    this.runAutomaton = runAutomaton;
-    this.lowerCase = lowerCase;
-    this.state = runAutomaton.getInitialState();
+    this(input, runAutomaton, lowerCase, DEFAULT_MAX_TOKEN_LENGTH);
   }
   
   @Override
+  public final boolean incrementToken() throws IOException {
+    assert !enableChecks || (streamState == State.RESET || streamState == State.INCREMENT) 
+                            : "incrementToken() called while in wrong state: " + streamState;
+    clearAttributes();
+    for (;;) {
+      int startOffset = off;
+      int cp = readCodePoint();
+      if (cp < 0) {
+        break;
+      } else if (isTokenChar(cp)) {
+        int endOffset;
+        do {
+          char chars[] = Character.toChars(normalize(cp));
+          for (int i = 0; i < chars.length; i++)
+            termAtt.append(chars[i]);
+          endOffset = off;
+          if (termAtt.length() >= maxTokenLength) {
+            break;
+          }
+          cp = readCodePoint();
+        } while (cp >= 0 && isTokenChar(cp));
+        offsetAtt.setOffset(correctOffset(startOffset), correctOffset(endOffset));
+        streamState = State.INCREMENT;
+        return true;
+      }
+    }
+    streamState = State.INCREMENT_FALSE;
+    return false;
+  }
+
+  protected int readCodePoint() throws IOException {
+    int ch = input.read();
+    if (ch < 0) {
+      return ch;
+    } else {
+      assert !Character.isLowSurrogate((char) ch);
+      off++;
+      if (Character.isHighSurrogate((char) ch)) {
+        int ch2 = input.read();
+        if (ch2 >= 0) {
+          off++;
+          assert Character.isLowSurrogate((char) ch2);
+          return Character.toCodePoint((char) ch, (char) ch2);
+        }
+      }
+      return ch;
+    }
+  }
+
   protected boolean isTokenChar(int c) {
     state = runAutomaton.step(state, c);
     if (state < 0) {
@@ -70,7 +154,6 @@ public class MockTokenizer extends CharT
     }
   }
   
-  @Override
   protected int normalize(int c) {
     return lowerCase ? Character.toLowerCase(c) : c;
   }
@@ -79,5 +162,43 @@ public class MockTokenizer extends CharT
   public void reset() throws IOException {
     super.reset();
     state = runAutomaton.getInitialState();
+    off = 0;
+    assert !enableChecks || streamState != State.RESET : "double reset()";
+    streamState = State.RESET;
+  }
+  
+  @Override
+  public void close() throws IOException {
+    super.close();
+    // in some exceptional cases (e.g. TestIndexWriterExceptions) a test can prematurely close()
+    // these tests should disable this check, by default we check the normal workflow.
+    // TODO: investigate the CachingTokenFilter "double-close"... for now we ignore this
+    assert !enableChecks || streamState == State.END || streamState == State.CLOSE : "close() called in wrong state: " + streamState;
+    streamState = State.CLOSE;
+  }
+
+  @Override
+  public void reset(Reader input) throws IOException {
+    super.reset(input);
+    assert !enableChecks || streamState == State.CLOSE : "setReader() called in wrong state: " + streamState;
+    streamState = State.SETREADER;
+  }
+
+  @Override
+  public void end() throws IOException {
+    int finalOffset = correctOffset(off);
+    offsetAtt.setOffset(finalOffset, finalOffset);
+    // some tokenizers, such as limiting tokenizers, call end() before incrementToken() returns false.
+    // these tests should disable this check (in general you should consume the entire stream)
+    assert !enableChecks || streamState == State.INCREMENT_FALSE : "end() called before incrementToken() returned false!";
+    streamState = State.END;
+  }
+
+  /** 
+   * Toggle consumer workflow checking: if your test consumes tokenstreams normally you
+   * should leave this enabled.
+   */
+  public void setEnableChecks(boolean enableChecks) {
+    this.enableChecks = enableChecks;
   }
 }

Modified: lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java Sun May 22 21:45:19 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/solr2452/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java Sun May 22 21:45:19 2011
@@ -18,7 +18,9 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
+import java.util.List;
 import java.util.Random;
 import java.util.Set;
 
@@ -40,14 +42,13 @@ public class MockRandomMergePolicy exten
 
     if (segmentInfos.size() > 1 && random.nextInt(5) == 3) {
       
-      SegmentInfos segmentInfos2 = new SegmentInfos();
-      segmentInfos2.addAll(segmentInfos);
-      Collections.shuffle(segmentInfos2, random);
+      List<SegmentInfo> segments = new ArrayList<SegmentInfo>(segmentInfos.asList());
+      Collections.shuffle(segments, random);
 
       // TODO: sometimes make more than 1 merge?
       mergeSpec = new MergeSpecification();
       final int segsToMerge = _TestUtil.nextInt(random, 1, segmentInfos.size());
-      mergeSpec.add(new OneMerge(segmentInfos2.range(0, segsToMerge)));
+      mergeSpec.add(new OneMerge(segments.subList(0, segsToMerge)));
     }
 
     return mergeSpec;
@@ -58,21 +59,36 @@ public class MockRandomMergePolicy exten
       SegmentInfos segmentInfos, int maxSegmentCount, Set<SegmentInfo> segmentsToOptimize)
     throws CorruptIndexException, IOException {
 
-    //System.out.println("MRMP: findMergesForOptimize sis=" + segmentInfos);
+    final List<SegmentInfo> eligibleSegments = new ArrayList<SegmentInfo>();
+    for(SegmentInfo info : segmentInfos) {
+      if (segmentsToOptimize.contains(info)) {
+        eligibleSegments.add(info);
+      }
+    }
+
+    //System.out.println("MRMP: findMergesForOptimize sis=" + segmentInfos + " eligible=" + eligibleSegments);
     MergeSpecification mergeSpec = null;
-    if (segmentInfos.size() > 1 || (segmentInfos.size() == 1 && segmentInfos.info(0).hasDeletions())) {
+    if (eligibleSegments.size() > 1 || (eligibleSegments.size() == 1 && eligibleSegments.get(0).hasDeletions())) {
       mergeSpec = new MergeSpecification();
-      SegmentInfos segmentInfos2 = new SegmentInfos();
-      segmentInfos2.addAll(segmentInfos);
-      Collections.shuffle(segmentInfos2, random);
+      // Already shuffled having come out of a set but
+      // shuffle again for good measure:
+      Collections.shuffle(eligibleSegments, random);
       int upto = 0;
-      while(upto < segmentInfos.size()) {
-        int max = Math.min(10, segmentInfos.size()-upto);
+      while(upto < eligibleSegments.size()) {
+        int max = Math.min(10, eligibleSegments.size()-upto);
         int inc = max <= 2 ? max : _TestUtil.nextInt(random, 2, max);
-        mergeSpec.add(new OneMerge(segmentInfos2.range(upto, upto+inc)));
+        mergeSpec.add(new OneMerge(eligibleSegments.subList(upto, upto+inc)));
         upto += inc;
       }
     }
+
+    if (mergeSpec != null) {
+      for(OneMerge merge : mergeSpec.merges) {
+        for(SegmentInfo info : merge.segments) {
+          assert segmentsToOptimize.contains(info);
+        }
+      }
+    }
     return mergeSpec;
   }
 

Modified: lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java Sun May 22 21:45:19 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/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java Sun May 22 21:45:19 2011
@@ -186,7 +186,7 @@ public class MockFixedIntBlockCodec exte
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) {
+  public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) throws IOException {
     SepPostingsReaderImpl.files(segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);

Modified: lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java Sun May 22 21:45:19 2011
@@ -211,7 +211,7 @@ public class MockVariableIntBlockCodec e
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) {
+  public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) throws IOException {
     SepPostingsReaderImpl.files(segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);

Modified: lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java Sun May 22 21:45:19 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) {
@@ -133,15 +140,18 @@ public class MockRandomCodec extends Cod
     out.close();
 
     final Random random = new Random(seed);
+    
+    random.nextInt(); // consume a random for buffersize
+    
     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()) {
@@ -186,7 +196,7 @@ public class MockRandomCodec extends Cod
 
               @Override
               public boolean isIndexTerm(BytesRef term, TermStats stats) {
-                return rand.nextInt(gap) == 17;
+                return rand.nextInt(gap) == gap/2;
               }
 
               @Override
@@ -231,16 +241,22 @@ public class MockRandomCodec extends Cod
     in.close();
 
     final Random random = new Random(seed);
+    
+    int readBufferSize = _TestUtil.nextInt(random, 1, 4096);
+    if (LuceneTestCase.VERBOSE) {
+      System.out.println("MockRandomCodec: readBufferSize=" + readBufferSize);
+    }
+
     PostingsReaderBase postingsReader;
 
     if (random.nextBoolean()) {
       postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo,
-                                                 state.readBufferSize, new MockIntStreamFactory(random), state.codecId);
+                                                 readBufferSize, new MockIntStreamFactory(random), state.codecId);
     } else {
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: reading Standard postings");
       }
-      postingsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize, state.codecId);
+      postingsReader = new StandardPostingsReader(state.dir, state.segmentInfo, readBufferSize, state.codecId);
     }
 
     if (random.nextBoolean()) {
@@ -305,7 +321,7 @@ public class MockRandomCodec extends Cod
                                                 state.fieldInfos,
                                                 state.segmentInfo.name,
                                                 postingsReader,
-                                                state.readBufferSize,
+                                                readBufferSize,
                                                 termsCacheSize,
                                                 state.codecId);
       success = true;

Modified: lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java?rev=1126234&r1=1126233&r2=1126234&view=diff
==============================================================================
--- lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java (original)
+++ lucene/dev/branches/solr2452/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java Sun May 22 21:45:19 2011
@@ -130,7 +130,7 @@ public class MockSepCodec extends Codec 
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) {
+  public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) throws IOException {
     SepPostingsReaderImpl.files(segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);