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

svn commit: r1081952 [8/17] - in /lucene/dev/branches/bulkpostings: ./ dev-tools/ 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/demo/ dev-tools/idea/luce...

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/function/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/function/package.html?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/function/package.html (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/function/package.html Tue Mar 15 21:35:17 2011
@@ -130,14 +130,14 @@
     Using field (byte) values to as scores:
     <p>
     Indexing:
-    <pre>
+    <pre class="prettyprint">
       f = new Field("score", "7", Field.Store.NO, Field.Index.UN_TOKENIZED);
       f.setOmitNorms(true);
       d1.add(f);
     </pre>
     <p>
     Search:
-    <pre>
+    <pre class="prettyprint">
       Query q = new FieldScoreQuery("score", FieldScoreQuery.Type.BYTE);
     </pre>
     Document d1 above would get a score of 7.
@@ -148,7 +148,7 @@
     <p>
     Dividing the original score of each document by a square root of its docid
     (just to demonstrate what it takes to manipulate scores this way)
-    <pre>
+    <pre class="prettyprint">
       Query q = queryParser.parse("my query text");
       CustomScoreQuery customQ = new CustomScoreQuery(q) {
         public float customScore(int doc, float subQueryScore, float valSrcScore) {
@@ -158,7 +158,7 @@
     </pre>
         <p>
         For more informative debug info on the custom query, also override the name() method:
-        <pre>
+        <pre class="prettyprint">
       CustomScoreQuery customQ = new CustomScoreQuery(q) {
         public float customScore(int doc, float subQueryScore, float valSrcScore) {
           return subQueryScore / Math.sqrt(docid);
@@ -171,7 +171,7 @@
         <p>
         Taking the square root of the original score and multiplying it by a "short field driven score", ie, the
         short value that was indexed for the scored doc in a certain field:
-        <pre>
+        <pre class="prettyprint">
       Query q = queryParser.parse("my query text");
       FieldScoreQuery qf = new FieldScoreQuery("shortScore", FieldScoreQuery.Type.SHORT);
       CustomScoreQuery customQ = new CustomScoreQuery(q,qf) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java Tue Mar 15 21:35:17 2011
@@ -53,7 +53,7 @@ public class NearSpansUnordered extends 
 
   private class CellQueue extends PriorityQueue<SpansCell> {
     public CellQueue(int size) {
-      initialize(size);
+      super(size);
     }
     
     @Override

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java Tue Mar 15 21:35:17 2011
@@ -145,7 +145,7 @@ public class SpanOrQuery extends SpanQue
 
   private class SpanQueue extends PriorityQueue<Spans> {
     public SpanQueue(int size) {
-      initialize(size);
+      super(size);
     }
 
     @Override

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/package.html?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/package.html (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/package.html Tue Mar 15 21:35:17 2011
@@ -59,7 +59,7 @@ two starts and ends at the greater of th
 <p>For example, a span query which matches "John Kerry" within ten
 words of "George Bush" within the first 100 words of the document
 could be constructed with:
-<pre>
+<pre class="prettyprint">
 SpanQuery john   = new SpanTermQuery(new Term("content", "john"));
 SpanQuery kerry  = new SpanTermQuery(new Term("content", "kerry"));
 SpanQuery george = new SpanTermQuery(new Term("content", "george"));
@@ -82,7 +82,7 @@ SpanQuery johnKerryNearGeorgeBushAtStart
 So, for example, the above query can be restricted to documents which
 also use the word "iraq" with:
 
-<pre>
+<pre class="prettyprint">
 Query query = new BooleanQuery();
 query.add(johnKerryNearGeorgeBushAtStart, true, false);
 query.add(new TermQuery("content", "iraq"), true, false);

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/DataOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/DataOutput.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/DataOutput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/DataOutput.java Tue Mar 15 21:35:17 2011
@@ -82,7 +82,7 @@ public abstract class DataOutput {
     writeInt((int) i);
   }
 
-  /** Writes an long in a variable-length format.  Writes between one and five
+  /** Writes an long in a variable-length format.  Writes between one and nine
    * bytes.  Smaller values take fewer bytes.  Negative numbers are not
    * supported.
    * @see DataInput#readVLong()

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/AttributeSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/AttributeSource.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/AttributeSource.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/AttributeSource.java Tue Mar 15 21:35:17 2011
@@ -131,14 +131,14 @@ public class AttributeSource {
   /**
    * returns the used AttributeFactory.
    */
-  public AttributeFactory getAttributeFactory() {
+  public final AttributeFactory getAttributeFactory() {
     return this.factory;
   }
   
   /** Returns a new iterator that iterates the attribute classes
    * in the same order they were added in.
    */
-  public Iterator<Class<? extends Attribute>> getAttributeClassesIterator() {
+  public final Iterator<Class<? extends Attribute>> getAttributeClassesIterator() {
     return Collections.unmodifiableSet(attributes.keySet()).iterator();
   }
   
@@ -146,7 +146,7 @@ public class AttributeSource {
    * This iterator may contain less entries that {@link #getAttributeClassesIterator},
    * if one instance implements more than one Attribute interface.
    */
-  public Iterator<AttributeImpl> getAttributeImplsIterator() {
+  public final Iterator<AttributeImpl> getAttributeImplsIterator() {
     if (hasAttributes()) {
       if (currentState == null) {
         computeCurrentState();

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/BytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/BytesRefHash.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/BytesRefHash.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/BytesRefHash.java Tue Mar 15 21:35:17 2011
@@ -117,7 +117,7 @@ public final class BytesRefHash {
    */
   public BytesRef get(int ord, BytesRef ref) {
     assert bytesStart != null : "bytesStart is null - not initialized";
-    assert ord < bytesStart.length: "ord exceeeds byteStart len: " + bytesStart.length;
+    assert ord < bytesStart.length: "ord exceeds byteStart len: " + bytesStart.length;
     return pool.setBytesRef(ref, bytesStart[ord]);
   }
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java Tue Mar 15 21:35:17 2011
@@ -19,6 +19,7 @@ package org.apache.lucene.util;
 
 import java.nio.CharBuffer;
 import java.nio.ByteBuffer;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute; // javadoc
 
 /**
  * Provides support for converting byte sequences to Strings and back again.
@@ -39,7 +40,10 @@ import java.nio.ByteBuffer;
  * <p/>
  *
  * @lucene.experimental
+ * @deprecated Implement {@link TermToBytesRefAttribute} and store bytes directly
+ * instead. This class will be removed in Lucene 5.0
  */
+@Deprecated
 public final class IndexableBinaryStringTools {
 
   private static final CodingCase[] CODING_CASES = {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/PriorityQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/PriorityQueue.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/PriorityQueue.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/PriorityQueue.java Tue Mar 15 21:35:17 2011
@@ -28,11 +28,57 @@ package org.apache.lucene.util;
 */
 public abstract class PriorityQueue<T> {
   private int size;
-  private int maxSize;
-  protected T[] heap;
+  private final int maxSize;
+  private final T[] heap;
+
+  public PriorityQueue(int maxSize) {
+    this(maxSize, true);
+  }
+
+  @SuppressWarnings("unchecked")
+  public PriorityQueue(int maxSize, boolean prepopulate) {
+    size = 0;
+    int heapSize;
+    if (0 == maxSize)
+      // We allocate 1 extra to avoid if statement in top()
+      heapSize = 2;
+    else {
+      if (maxSize == Integer.MAX_VALUE) {
+        // Don't wrap heapSize to -1, in this case, which
+        // causes a confusing NegativeArraySizeException.
+        // Note that very likely this will simply then hit
+        // an OOME, but at least that's more indicative to
+        // caller that this values is too big.  We don't +1
+        // in this case, but it's very unlikely in practice
+        // one will actually insert this many objects into
+        // the PQ:
+        heapSize = Integer.MAX_VALUE;
+      } else {
+        // NOTE: we add +1 because all access to heap is
+        // 1-based not 0-based.  heap[0] is unused.
+        heapSize = maxSize + 1;
+      }
+    }
+    heap = (T[]) new Object[heapSize]; // T is unbounded type, so this unchecked cast works always
+    this.maxSize = maxSize;
+    
+    if (prepopulate) {
+      // If sentinel objects are supported, populate the queue with them
+      T sentinel = getSentinelObject();
+      if (sentinel != null) {
+        heap[1] = sentinel;
+        for (int i = 2; i < heap.length; i++) {
+          heap[i] = getSentinelObject();
+        }
+        size = maxSize;
+      }
+    }
+  }
 
   /** Determines the ordering of objects in this priority queue.  Subclasses
-    must define this one method. */
+   *  must define this one method.
+   *  @return <code>true</code> iff parameter <tt>a</tt> is less than parameter <tt>b</tt>.
+   */
   protected abstract boolean lessThan(T a, T b);
 
   /**
@@ -78,45 +124,6 @@ public abstract class PriorityQueue<T> {
     return null;
   }
 
-  /** Subclass constructors must call this. */
-  @SuppressWarnings("unchecked")
-  protected final void initialize(int maxSize) {
-    size = 0;
-    int heapSize;
-    if (0 == maxSize)
-      // We allocate 1 extra to avoid if statement in top()
-      heapSize = 2;
-    else {
-      if (maxSize == Integer.MAX_VALUE) {
-        // Don't wrap heapSize to -1, in this case, which
-        // causes a confusing NegativeArraySizeException.
-        // Note that very likely this will simply then hit
-        // an OOME, but at least that's more indicative to
-        // caller that this values is too big.  We don't +1
-        // in this case, but it's very unlikely in practice
-        // one will actually insert this many objects into
-        // the PQ:
-        heapSize = Integer.MAX_VALUE;
-      } else {
-        // NOTE: we add +1 because all access to heap is
-        // 1-based not 0-based.  heap[0] is unused.
-        heapSize = maxSize + 1;
-      }
-    }
-    heap = (T[]) new Object[heapSize]; // T is unbounded type, so this unchecked cast works always
-    this.maxSize = maxSize;
-    
-    // If sentinel objects are supported, populate the queue with them
-    T sentinel = getSentinelObject();
-    if (sentinel != null) {
-      heap[1] = sentinel;
-      for (int i = 2; i < heap.length; i++) {
-        heap[i] = getSentinelObject();
-      }
-      size = maxSize;
-    }
-  }
-
   /**
    * Adds an Object to a PriorityQueue in log(size) time. If one tries to add
    * more objects than maxSize from initialize an
@@ -245,4 +252,11 @@ public abstract class PriorityQueue<T> {
     }
     heap[i] = node;				  // install saved node
   }
+  
+  /** This method returns the internal heap array as Object[].
+   * @lucene.internal
+   */
+  protected final Object[] getHeapArray() {
+    return (Object[]) heap;
+  }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/SmallFloat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/SmallFloat.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/SmallFloat.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/SmallFloat.java Tue Mar 15 21:35:17 2011
@@ -39,7 +39,7 @@ public class SmallFloat {
     int fzero = (63-zeroExp)<<numMantissaBits;
     int bits = Float.floatToRawIntBits(f);
     int smallfloat = bits >> (24-numMantissaBits);
-    if (smallfloat < fzero) {
+    if (smallfloat <= fzero) {
       return (bits<=0) ?
         (byte)0   // negative numbers and zero both map to 0 byte
        :(byte)1;  // underflow is mapped to smallest non-zero number.
@@ -75,7 +75,7 @@ public class SmallFloat {
   public static byte floatToByte315(float f) {
     int bits = Float.floatToRawIntBits(f);
     int smallfloat = bits >> (24-3);
-    if (smallfloat < (63-15)<<3) {
+    if (smallfloat <= ((63-15)<<3)) {
       return (bits<=0) ? (byte)0 : (byte)1;
     }
     if (smallfloat >= ((63-15)<<3) + 0x100) {
@@ -103,7 +103,7 @@ public class SmallFloat {
   public static byte floatToByte52(float f) {
     int bits = Float.floatToRawIntBits(f);
     int smallfloat = bits >> (24-5);
-    if (smallfloat < (63-2)<<5) {
+    if (smallfloat <= (63-2)<<5) {
       return (bits<=0) ? (byte)0 : (byte)1;
     }
     if (smallfloat >= ((63-2)<<5) + 0x100) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/Version.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/Version.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/Version.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/Version.java Tue Mar 15 21:35:17 2011
@@ -41,6 +41,13 @@ public enum Version {
    */
   @Deprecated
   LUCENE_31,
+  
+  /**
+   * Match settings and bugs in Lucene's 3.2 release.
+   * @deprecated (4.0) Use latest
+   */
+  @Deprecated
+  LUCENE_32,
 
   /** Match settings and bugs in Lucene's 4.0 release. 
    *  <p>

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java Tue Mar 15 21:35:17 2011
@@ -83,7 +83,7 @@ public class Builder<T> {
     @SuppressWarnings("unchecked") final UnCompiledNode<T>[] f = (UnCompiledNode<T>[]) new UnCompiledNode[10];
     frontier = f;
     for(int idx=0;idx<frontier.length;idx++) {
-      frontier[idx] = new UnCompiledNode<T>(this);
+      frontier[idx] = new UnCompiledNode<T>(this, idx);
     }
   }
 
@@ -91,7 +91,7 @@ public class Builder<T> {
     return fst.nodeCount;
   }
 
-  public int getTermCount() {
+  public long getTermCount() {
     return frontier[0].inputCount;
   }
 
@@ -201,7 +201,7 @@ public class Builder<T> {
           // undecided on whether to prune it.  later, it
           // will be either compiled or pruned, so we must
           // allocate a new node:
-          frontier[idx] = new UnCompiledNode<T>(this);
+          frontier[idx] = new UnCompiledNode<T>(this, idx);
         }
       }
     }
@@ -292,7 +292,7 @@ public class Builder<T> {
         new UnCompiledNode[ArrayUtil.oversize(input.length+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       System.arraycopy(frontier, 0, next, 0, frontier.length);
       for(int idx=frontier.length;idx<next.length;idx++) {
-        next[idx] = new UnCompiledNode<T>(this);
+        next[idx] = new UnCompiledNode<T>(this, idx);
       }
       frontier = next;
     }
@@ -422,14 +422,24 @@ public class Builder<T> {
     Arc<T>[] arcs;
     T output;
     boolean isFinal;
-    int inputCount;
+    long inputCount;
 
+    /** This node's depth, starting from the automaton root. */
+    final int depth;
+
+    /**
+     * @param depth
+     *          The node's depth starting from the automaton root. Needed for
+     *          LUCENE-2934 (node expansion based on conditions other than the
+     *          fanout size).
+     */
     @SuppressWarnings("unchecked")
-    public UnCompiledNode(Builder<T> owner) {
+    public UnCompiledNode(Builder<T> owner, int depth) {
       this.owner = owner;
       arcs = (Arc<T>[]) new Arc[1];
       arcs[0] = new Arc<T>();
       output = owner.NO_OUTPUT;
+      this.depth = depth;
     }
 
     public boolean isCompiled() {
@@ -441,6 +451,9 @@ public class Builder<T> {
       isFinal = false;
       output = owner.NO_OUTPUT;
       inputCount = 0;
+
+      // We don't clear the depth here because it never changes 
+      // for nodes on the frontier (even when reused).
     }
 
     public T getLastOutput(int labelToMatch) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java Tue Mar 15 21:35:17 2011
@@ -25,6 +25,7 @@ import org.apache.lucene.store.IndexInpu
 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;
 
 /** Represents an FST using a compact byte[] format.
  *  <p> The format is similar to what's used by Morfologik
@@ -47,11 +48,21 @@ public class FST<T> {
   // this when number of arcs is > NUM_ARCS_ARRAY:
   private final static int BIT_ARCS_AS_FIXED_ARRAY = 1 << 6;
 
-  // If the node has >= this number of arcs, the arcs are
-  // stored as a fixed array.  Fixed array consumes more RAM
-  // but enables binary search on the arcs (instead of
-  // linear scan) on lookup by arc label:
-  private final static int NUM_ARCS_FIXED_ARRAY = 10;
+  /**
+   * @see #shouldExpand(UnCompiledNode)
+   */
+  final static int FIXED_ARRAY_SHALLOW_DISTANCE = 3; // 0 => only root node.
+
+  /**
+   * @see #shouldExpand(UnCompiledNode)
+   */
+  final static int FIXED_ARRAY_NUM_ARCS_SHALLOW = 5;
+
+  /**
+   * @see #shouldExpand(UnCompiledNode)
+   */
+  final static int FIXED_ARRAY_NUM_ARCS_DEEP = 10;
+
   private int[] bytesPerArc = new int[0];
 
   // Increment version to change it
@@ -88,7 +99,7 @@ public class FST<T> {
   public int arcWithOutputCount;
 
   // If arc has this label then that arc is final/accepted
-  public static int END_LABEL = -1;
+  public static final int END_LABEL = -1;
 
   public final static class Arc<T> {
     public int label;
@@ -286,9 +297,7 @@ public class FST<T> {
     final int v;
     if (inputType == INPUT_TYPE.BYTE1) {
       v = in.readByte()&0xFF;
-    } else if (inputType == INPUT_TYPE.BYTE2) {
-      v = in.readVInt();
-    } else {
+    } else { 
       v = in.readVInt();
     }
     return v;
@@ -315,7 +324,7 @@ public class FST<T> {
     int startAddress = writer.posWrite;
     //System.out.println("  startAddr=" + startAddress);
 
-    final boolean doFixedArray = node.numArcs >= NUM_ARCS_FIXED_ARRAY;
+    final boolean doFixedArray = shouldExpand(node);
     final int fixedArrayStart;
     if (doFixedArray) {
       if (bytesPerArc.length < node.numArcs) {
@@ -467,9 +476,66 @@ public class FST<T> {
     return arc;
   }
 
-  /** Follow the follow arc and read the first arc of its
-   *  target; this changes the provide arc (2nd arg) in-place
-   *  and returns it. */
+  /** Follows the <code>follow</code> arc and reads the last
+   *  arc of its target; this changes the provided
+   *  <code>arc</code> (2nd arg) in-place and returns it.
+   * 
+   * @return Returns the second argument
+   * (<code>arc</code>). */
+  public Arc<T> readLastTargetArc(Arc<T> follow, Arc<T> arc) throws IOException {
+    //System.out.println("readLast");
+    if (!targetHasArcs(follow)) {
+      //System.out.println("  end node");
+      assert follow.isFinal();
+      arc.label = -1;
+      arc.output = follow.nextFinalOutput;
+      arc.flags = BIT_LAST_ARC;
+      return arc;
+    } else {
+      final BytesReader in = getBytesReader(follow.target);
+      arc.flags = in.readByte();
+      if (arc.flag(BIT_ARCS_AS_FIXED_ARRAY)) {
+        // array: jump straight to end
+        arc.numArcs = in.readVInt();
+        arc.bytesPerArc = in.readByte() & 0xFF;
+        //System.out.println("  array numArcs=" + arc.numArcs + " bpa=" + arc.bytesPerArc);
+        arc.posArcsStart = in.pos;
+        arc.arcIdx = arc.numArcs - 2;
+      } else {
+        // non-array: linear scan
+        arc.bytesPerArc = 0;
+        //System.out.println("  scan");
+        while(!arc.isLast()) {
+          // skip this arc:
+          readLabel(in);
+          if (arc.flag(BIT_ARC_HAS_OUTPUT)) {
+            outputs.read(in);
+          }
+          if (arc.flag(BIT_ARC_HAS_FINAL_OUTPUT)) {
+            outputs.read(in);
+          }
+          if (arc.flag(BIT_STOP_NODE)) {
+          } else if (arc.flag(BIT_TARGET_NEXT)) {
+          } else {
+            in.pos -= 4;
+          }
+          arc.flags = in.readByte();
+        }
+        arc.nextArc = in.pos+1;
+      }
+      readNextRealArc(arc);
+      assert arc.isLast();
+      return arc;
+    }
+  }
+
+  /**
+   * Follow the <code>follow</code> arc and read the first arc of its target;
+   * this changes the provided <code>arc</code> (2nd arg) in-place and returns
+   * it.
+   * 
+   * @return Returns the second argument (<code>arc</code>).
+   */
   public Arc<T> readFirstTargetArc(Arc<T> follow, Arc<T> arc) throws IOException {
     //int pos = address;
     //System.out.println("    readFirstTarget follow.target=" + follow.target + " isFinal=" + follow.isFinal());
@@ -503,15 +569,30 @@ public class FST<T> {
       arc.numArcs = in.readVInt();
       arc.bytesPerArc = in.readByte() & 0xFF;
       arc.arcIdx = -1;
-      arc.posArcsStart = in.pos;
+      arc.nextArc = arc.posArcsStart = in.pos;
       //System.out.println("  bytesPer=" + arc.bytesPerArc + " numArcs=" + arc.numArcs + " arcsStart=" + pos);
     } else {
-      in.pos++;
+      arc.nextArc = address;
       arc.bytesPerArc = 0;
     }
-    arc.nextArc = in.pos;
-    arc.label = 0;
-    return readNextArc(arc);
+    return readNextRealArc(arc);
+  }
+
+  /**
+   * Checks if <code>arc</code>'s target state is in expanded (or vector) format. 
+   * 
+   * @return Returns <code>true</code> if <code>arc</code> points to a state in an
+   * expanded array format.
+   */
+  boolean isExpandedTarget(Arc<T> follow) throws IOException {
+    if (follow.isFinal()) {
+      return false;
+    } else {
+      final BytesReader in = getBytesReader(follow.target);
+      final byte b = in.readByte();
+      
+      return (b & BIT_ARCS_AS_FIXED_ARRAY) != 0;
+    }
   }
 
   /** In-place read; returns the arc. */
@@ -566,6 +647,7 @@ public class FST<T> {
     if (arc.bytesPerArc != 0) {
       // arcs are at fixed entries
       arc.arcIdx++;
+      assert arc.arcIdx < arc.numArcs;
       in = getBytesReader(arc.posArcsStart - arc.arcIdx*arc.bytesPerArc);
     } else {
       // arcs are packed
@@ -708,6 +790,26 @@ public class FST<T> {
   public int getArcWithOutputCount() {
     return arcWithOutputCount;
   }
+  
+  /**
+   * Nodes will be expanded if their depth (distance from the root node) is
+   * &lt;= this value and their number of arcs is &gt;=
+   * {@link #FIXED_ARRAY_NUM_ARCS_SHALLOW}.
+   * 
+   * <p>
+   * Fixed array consumes more RAM but enables binary search on the arcs
+   * (instead of a linear scan) on lookup by arc label.
+   * 
+   * @return <code>true</code> if <code>node</code> should be stored in an
+   *         expanded (array) form.
+   * 
+   * @see #FIXED_ARRAY_NUM_ARCS_DEEP
+   * @see Builder.UnCompiledNode#depth
+   */
+  private boolean shouldExpand(UnCompiledNode<T> node) {
+    return (node.depth <= FIXED_ARRAY_SHALLOW_DISTANCE && node.numArcs >= FIXED_ARRAY_NUM_ARCS_SHALLOW) || 
+            node.numArcs >= FIXED_ARRAY_NUM_ARCS_DEEP;
+  }
 
   // Non-static: writes to FST's byte[]
   class BytesWriter extends DataOutput {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java Tue Mar 15 21:35:17 2011
@@ -274,7 +274,7 @@ abstract class FSTEnum<T> {
     while(true) {
       //System.out.println("  cycle upto=" + upto + " arc.label=" + arc.label + " (" + (char) arc.label + ") targetLabel=" + targetLabel + " isLast?=" + arc.isLast());
 
-      if (arc.bytesPerArc != 0 && arc.label != -1) {
+      if (arc.bytesPerArc != 0 && arc.label != FST.END_LABEL) {
         // Arcs are fixed array -- use binary search to find
         // the target.
 
@@ -465,12 +465,7 @@ abstract class FSTEnum<T> {
       }
       incr();
 
-      final FST.Arc<T> nextArc = getArc(upto);
-      fst.readFirstTargetArc(arc, nextArc);
-      arc = nextArc;
-      while(!arc.isLast()) {
-        fst.readNextArc(arc);
-      }
+      arc = fst.readLastTargetArc(arc, getArc(upto));
     }
   }
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/Util.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/Util.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/Util.java Tue Mar 15 21:35:17 2011
@@ -17,12 +17,8 @@ package org.apache.lucene.util.automaton
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Set;
-import java.util.HashSet;
+import java.io.*;
+import java.util.*;
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
@@ -160,90 +156,171 @@ public final class Util {
       return output;
     }
   }
+  
+  /**
+   * Dumps an {@link FST} to a GraphViz's <code>dot</code> language description
+   * for visualization. Example of use:
+   * 
+   * <pre>
+   * PrintStream ps = new PrintStream(&quot;out.dot&quot;);
+   * fst.toDot(ps);
+   * ps.close();
+   * </pre>
+   * 
+   * and then, from command line:
+   * 
+   * <pre>
+   * dot -Tpng -o out.png out.dot
+   * </pre>
+   * 
+   * <p>
+   * Note: larger FSTs (a few thousand nodes) won't even render, don't bother.
+   * 
+   * @param sameRank
+   *          If <code>true</code>, the resulting <code>dot</code> file will try
+   *          to order states in layers of breadth-first traversal. This may
+   *          mess up arcs, but makes the output FST's structure a bit clearer.
+   * 
+   * @param labelStates
+   *          If <code>true</code> states will have labels equal to their offsets in their
+   *          binary format. Expands the graph considerably. 
+   * 
+   * @see "http://www.graphviz.org/"
+   */
+  public static <T> void toDot(FST<T> fst, Writer out, boolean sameRank, boolean labelStates) 
+    throws IOException {    
+    final String expandedNodeColor = "blue";
 
+    // This is the start arc in the automaton (from the epsilon state to the first state 
+    // with outgoing transitions.
+    final FST.Arc<T> startArc = fst.getFirstArc(new FST.Arc<T>());
 
-  // NOTE: this consumes alot of RAM!
-  // arcs w/ NEXT opto are in blue
-  /*
-    eg:
-      PrintStream ps = new PrintStream("out.dot");
-      fst.toDot(ps);
-      ps.close();
-      System.out.println("SAVED out.dot");
-      
-    then dot -Tpng out.dot > /x/tmp/out.png
-  */
+    // A queue of transitions to consider for the next level.
+    final List<FST.Arc<T>> thisLevelQueue = new ArrayList<FST.Arc<T>>();
 
-  public static<T> void toDot(FST<T> fst, PrintStream out) throws IOException {
+    // A queue of transitions to consider when processing the next level.
+    final List<FST.Arc<T>> nextLevelQueue = new ArrayList<FST.Arc<T>>();
+    nextLevelQueue.add(startArc);
     
-    final FST.Arc<T> startArc = fst.getFirstArc(new FST.Arc<T>());
+    // A list of states on the same level (for ranking).
+    final List<Integer> sameLevelStates = new ArrayList<Integer>();
 
-    final List<FST.Arc<T>> queue = new ArrayList<FST.Arc<T>>();
-    queue.add(startArc);
+    // A bitset of already seen states (target offset).
+    final BitSet seen = new BitSet();
+    seen.set(startArc.target);
+
+    // Shape for states.
+    final String stateShape = "circle";
+
+    // Emit DOT prologue.
+    out.write("digraph FST {\n");
+    out.write("  rankdir = LR; splines=true; concentrate=true; ordering=out; ranksep=2.5; \n");
 
-    final Set<Integer> seen = new HashSet<Integer>();
-    seen.add(startArc.target);
-    
-    out.println("digraph FST {");
-    out.println("  rankdir = LR;");
-    //out.println("  " + startNode + " [shape=circle label=" + startNode + "];");
-    out.println("  " + startArc.target + " [label=\"\" shape=circle];");
-    out.println("  initial [shape=point color=white label=\"\"];");
-    out.println("  initial -> " + startArc.target);
+    if (!labelStates) {
+      out.write("  node [shape=circle, width=.2, height=.2, style=filled]\n");      
+    }
+
+    emitDotState(out, "initial", "point", "white", "");
+    emitDotState(out, Integer.toString(startArc.target), stateShape, 
+        fst.isExpandedTarget(startArc) ? expandedNodeColor : null, 
+        "");
+    out.write("  initial -> " + startArc.target + "\n");
 
     final T NO_OUTPUT = fst.outputs.getNoOutput();
+    int level = 0;
+
+    while (!nextLevelQueue.isEmpty()) {
+      // we could double buffer here, but it doesn't matter probably.
+      thisLevelQueue.addAll(nextLevelQueue);
+      nextLevelQueue.clear();
+
+      level++;
+      out.write("\n  // Transitions and states at level: " + level + "\n");
+      while (!thisLevelQueue.isEmpty()) {
+        final FST.Arc<T> arc = thisLevelQueue.remove(thisLevelQueue.size() - 1);
+        
+        if (fst.targetHasArcs(arc)) {
+          // scan all arcs
+          final int node = arc.target;
+          fst.readFirstTargetArc(arc, arc);
+          
+          while (true) {
+            // Emit the unseen state and add it to the queue for the next level.
+            if (arc.target >= 0 && !seen.get(arc.target)) {
+              final boolean isExpanded = fst.isExpandedTarget(arc);
+              emitDotState(out, Integer.toString(arc.target), stateShape, 
+                  isExpanded ?  expandedNodeColor : null, 
+                  labelStates ? Integer.toString(arc.target) : ""); 
+              seen.set(arc.target);
+              nextLevelQueue.add(new FST.Arc<T>().copyFrom(arc));
+              sameLevelStates.add(arc.target);
+            }
 
-    while(queue.size() != 0) {
-      FST.Arc<T> arc = queue.get(queue.size()-1);
-      queue.remove(queue.size()-1);
-      //System.out.println("dot cycle target=" + arc.target);
-
-      if (fst.targetHasArcs(arc)) {
-
-        // scan all arcs
-        final int node = arc.target;
-        fst.readFirstTargetArc(arc, arc);
-        while(true) {
-
-          //System.out.println("  cycle label=" + arc.label + " (" + (char) arc.label + ") target=" + arc.target);
-          if (!seen.contains(arc.target)) {
-            final String shape;
-            if (arc.target == -1) {
-              shape = "doublecircle";
+            String outs;
+            if (arc.output != NO_OUTPUT) {
+              outs = "/" + fst.outputs.outputToString(arc.output);
             } else {
-              shape = "circle";
+              outs = "";
+            }
+
+            final String cl;
+            if (arc.label == FST.END_LABEL) {
+              cl = "~";
+            } else {
+              cl = printableLabel(arc.label);
+            }
+
+            out.write("  " + node + " -> " + arc.target + " [label=\"" + cl + outs + "\"]\n");
+            
+            // Break the loop if we're on the last arc of this state.
+            if (arc.isLast()) {
+              break;
             }
-            out.println("  " + arc.target + " [shape=" + shape + "];");
-            seen.add(arc.target);
-            queue.add(new FST.Arc<T>().copyFrom(arc));
-            //System.out.println("    new!");
-          }
-          String outs;
-          if (arc.output != NO_OUTPUT) {
-            outs = "/" + fst.outputs.outputToString(arc.output);
-          } else {
-            outs = "";
-          }
-          final char cl;
-          if (arc.label == FST.END_LABEL) {
-            cl = '~';
-          } else {
-            cl = (char) arc.label;
-          }
-          out.println("  " + node + " -> " + arc.target + " [label=\"" + cl + outs + "\"]");
-          //if (arc.flag(FST.BIT_TARGET_NEXT)) {
-          //out.print(" color=blue");
-          //}
-          //out.println("];");
-        
-          if (arc.isLast()) {
-            break;
-          } else {
             fst.readNextArc(arc);
           }
         }
       }
+
+      // Emit state ranking information.
+      if (sameRank && sameLevelStates.size() > 1) {
+        out.write("  {rank=same; ");
+        for (int state : sameLevelStates) {
+          out.write(state + "; ");
+        }
+        out.write(" }\n");
+      }
+      sameLevelStates.clear();                
+    }
+
+    // Emit terminating state (always there anyway).
+    out.write("  -1 [style=filled, color=black, shape=circle, label=\"\"]\n\n");
+    out.write("  {rank=sink; -1 }\n");
+    
+    out.write("}\n");
+    out.flush();
+  }
+
+  /**
+   * Emit a single state in the <code>dot</code> language. 
+   */
+  private static void emitDotState(Writer out, String name, String shape,
+      String color, String label) throws IOException {
+    out.write("  " + name 
+        + " [" 
+        + (shape != null ? "shape=" + shape : "") + " "
+        + (color != null ? "color=" + color : "") + " "
+        + (label != null ? "label=\"" + label + "\"" : "label=\"\"") + " "
+        + "]\n");
+  }
+
+  /**
+   * Ensures an arc's label is indeed printable (dot uses US-ASCII). 
+   */
+  private static String printableLabel(int label) {
+    if (label >= 0x20 && label <= 0x7d) {
+      return Character.toString((char) label);
+    } else {
+      return "0x" + Integer.toHexString(label);
     }
-    out.println("}");
   }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/overview.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/overview.html?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/overview.html (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/overview.html Tue Mar 15 21:35:17 2011
@@ -159,27 +159,19 @@ method.</li>
 Some simple examples of code which does this are:
 <ul>
 <li>
-&nbsp;<a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/src/demo/org/apache/lucene/demo/FileDocument.java">FileDocument.java</a> contains
-code to create a Document for a file.</li>
-
-<li>
-&nbsp;<a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/src/demo/org/apache/lucene/demo/IndexFiles.java">IndexFiles.java</a> creates an
+&nbsp;<a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/contrib/demo/src/java/org/apache/lucene/demo/IndexFiles.java">IndexFiles.java</a> creates an
 index for all the files contained in a directory.</li>
 
 <li>
-&nbsp;<a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/src/demo/org/apache/lucene/demo/DeleteFiles.java">DeleteFiles.java</a> deletes some
-of these files from the index.</li>
-
-<li>
-&nbsp;<a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/src/demo/org/apache/lucene/demo/SearchFiles.java">SearchFiles.java</a> prompts for
+&nbsp;<a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java">SearchFiles.java</a> prompts for
 queries and searches an index.</li>
 </ul>
 To demonstrate these, try something like:
-<blockquote><tt>> <b>java -cp lucene.jar:lucene-demo.jar org.apache.lucene.demo.IndexFiles rec.food.recipes/soups</b></tt>
+<blockquote><tt>> <b>java -cp lucene.jar:lucene-demo.jar:lucene-analyzers-common.jar org.apache.lucene.demo.IndexFiles rec.food.recipes/soups</b></tt>
 <br><tt>adding rec.food.recipes/soups/abalone-chowder</tt>
 <br><tt>&nbsp; </tt>[ ... ]
 
-<p><tt>> <b>java -cp lucene.jar:lucene-demo.jar org.apache.lucene.demo.SearchFiles</b></tt>
+<p><tt>> <b>java -cp lucene.jar:lucene-demo.jar:lucene-analyzers-common.jar org.apache.lucene.demo.SearchFiles</b></tt>
 <br><tt>Query: <b>chowder</b></tt>
 <br><tt>Searching for: chowder</tt>
 <br><tt>34 total matching documents</tt>
@@ -195,16 +187,5 @@ and the word "manhattan" ... ]
 <br>&nbsp;&nbsp;&nbsp; [ Note: "+" and "-" are canonical, but "AND", "OR"
 and "NOT" may be used. ]</blockquote>
 
-The <a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene/src/demo/org/apache/lucene/demo/IndexHTML.java">IndexHTML</a> demo is more sophisticated.&nbsp;
-It incrementally maintains an index of HTML files, adding new files as
-they appear, deleting old files as they disappear and re-indexing files
-as they change.
-<blockquote><tt>> <b>java -cp lucene.jar:lucene-demo.jar org.apache.lucene.demo.IndexHTML -create java/jdk1.1.6/docs/relnotes</b></tt>
-<br><tt>adding java/jdk1.1.6/docs/relnotes/SMICopyright.html</tt>
-<br><tt>&nbsp; </tt>[ ... create an index containing all the relnotes ]
-<p><tt>> <b>rm java/jdk1.1.6/docs/relnotes/smicopyright.html</b></tt>
-<p><tt>> <b>java -cp lucene.jar:lucene-demo.jar org.apache.lucene.demo.IndexHTML java/jdk1.1.6/docs/relnotes</b></tt>
-<br><tt>deleting java/jdk1.1.6/docs/relnotes/SMICopyright.html</tt></blockquote>
-
 </body>
 </html>

Modified: lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/demo.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/demo.xml?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/demo.xml (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/demo.xml Tue Mar 15 21:35:17 2011
@@ -18,26 +18,26 @@ It walks you through some basic installa
 </section>
 
 
-<section id="About the Demos"><title>About the Demos</title>
+<section id="About the Demo"><title>About the Demo</title>
 <p>
-The Lucene command-line demo code consists of two applications that demonstrate various
-functionalities of Lucene and how one should go about adding Lucene to their applications.
+The Lucene command-line demo code consists of an application that demonstrates various
+functionalities of Lucene and how you can add Lucene to your applications.
 </p>
 </section>
 
 <section id="Setting your CLASSPATH"><title>Setting your CLASSPATH</title>
 <p>
 First, you should <a href="http://www.apache.org/dyn/closer.cgi/lucene/java/">download</a> the
-latest Lucene distribution and then extract it to a working directory.  Alternatively, you can <a
-href="http://wiki.apache.org/lucene-java/SourceRepository">check out the sources from
-Subversion</a>, and then run <code>ant war-demo</code> to generate the JARs and WARs.
+latest Lucene distribution and then extract it to a working directory.
 </p>
 <p>
-You should see the Lucene JAR file in the directory you created when you extracted the archive.  It
-should be named something like <code>lucene-core-{version}.jar</code>.  You should also see a file
-called <code>lucene-demos-{version}.jar</code>.  If you checked out the sources from Subversion then
-the JARs are located under the <code>build</code> subdirectory (after running <code>ant</code>
-successfully).  Put both of these files in your Java CLASSPATH.
+You need three JARs: the Lucene JAR, the common analysis JAR, and the Lucene demo JAR.  You should
+see the Lucene JAR file in the directory you created when you extracted the archive -- it
+should be named something like <code>lucene-core-{version}.jar</code>.  You should also see files
+called <code>lucene-analyzers-common-{version}.jar</code> and <code>lucene-demo-{version}.jar</code>.
+</p>
+<p>
+Put all three of these files in your Java CLASSPATH.
 </p>
 </section>
 
@@ -47,7 +47,7 @@ Once you've gotten this far you're proba
 you've set your CLASSPATH correctly, just type:
 
 <pre>
-    java org.apache.lucene.demo.IndexFiles {full-path-to-lucene}/src
+    java org.apache.lucene.demo.IndexFiles -docs {path-to-lucene}/src
 </pre>
 
 This will produce a subdirectory called <code>index</code> which will contain an index of all of the

Modified: lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/demo2.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/demo2.xml?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/demo2.xml (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/demo2.xml Tue Mar 15 21:35:17 2011
@@ -22,10 +22,15 @@ how to use Lucene in their applications.
 <section id="Location of the source"><title>Location of the source</title>
 
 <p>
-Relative to the directory created when you extracted Lucene or retrieved it from Subversion, you
-should see a directory called <code>src</code> which in turn contains a directory called
-<code>demo</code>.  This is the root for all of the Lucene demos.  Under this directory is
-<code>org/apache/lucene/demo</code>.  This is where all the Java sources for the demos live.
+NOTE: to examine the sources, you need to download and extract a source checkout of 
+Lucene: (lucene-{version}-src.zip).
+</p>
+
+<p>
+Relative to the directory created when you extracted Lucene, you
+should see a directory called <code>lucene/contrib/demo/</code>.  This is the root for the Lucene
+demo.  Under this directory is <code>src/java/org/apache/lucene/demo/</code>.  This is where all
+the Java sources for the demo live.
 </p>
 
 <p>
@@ -39,62 +44,80 @@ Bring it up in <code>vi</code> or your e
 
 <p>
 As we discussed in the previous walk-through, the <a
-href="api/demo/org/apache/lucene/demo/IndexFiles.html">IndexFiles</a> class creates a Lucene
+href="api/contrib-demo/org/apache/lucene/demo/IndexFiles.html">IndexFiles</a> class creates a Lucene
 Index. Let's take a look at how it does this.
 </p>
 
 <p>
-The first substantial thing the <code>main</code> function does is instantiate <a
-href="api/core/org/apache/lucene/index/IndexWriter.html">IndexWriter</a>.  It passes the string
-"<code>index</code>" and a new instance of a class called <a
-href="api/core/org/apache/lucene/analysis/standard/StandardAnalyzer.html">StandardAnalyzer</a>. 
-The "<code>index</code>" string is the name of the filesystem directory where all index information
-should be stored.  Because we're not passing a full path, this will be created as a subdirectory of
-the current working directory (if it does not already exist). On some platforms, it may be created
-in other directories (such as the user's home directory).
+The <code>main()</code> method parses the command-line parameters, then in preparation for
+instantiating <a href="api/core/org/apache/lucene/index/IndexWriter.html">IndexWriter</a>, opens a 
+<a href="api/core/org/apache/lucene/store/Directory.html">Directory</a> and instantiates
+<a href="api/module-analysis-common/org/apache/lucene/analysis/standard/StandardAnalyzer.html"
+>StandardAnalyzer</a> and
+<a href="api/core/org/apache/lucene/index/IndexWriterConfig.html">IndexWriterConfig</a>.
 </p>
 
 <p>
-The <a href="api/core/org/apache/lucene/index/IndexWriter.html">IndexWriter</a> is the main
-class responsible for creating indices.  To use it you must instantiate it with a path that it can
-write the index into.  If this path does not exist it will first create it.  Otherwise it will
-refresh the index at that path.  You can also create an index using one of the subclasses of <a
-href="api/core/org/apache/lucene/store/Directory.html">Directory</a>.  In any case, you must also pass an
-instance of <a
-href="api/core/org/apache/lucene/analysis/Analyzer.html">org.apache.lucene.analysis.Analyzer</a>.
+The value of the <code>-index</code> command-line parameter is the name of the filesystem directory
+where all index information should be stored.  If <code>IndexFiles</code> is invoked with a 
+relative path given in the <code>-index</code> command-line parameter, or if the <code>-index</code>
+command-line parameter is not given, causing the default relative index path "<code>index</code>"
+to be used, the index path will be created as a subdirectory of the current working directory
+(if it does not already exist).  On some platforms, the index path may be created in a different
+directory (such as the user's home directory).
 </p>
 
 <p>
-The particular <a href="api/core/org/apache/lucene/analysis/Analyzer.html">Analyzer</a> we
-are using, <a
-href="api/core/org/apache/lucene/analysis/standard/StandardAnalyzer.html">StandardAnalyzer</a>, is
-little more than a standard Java Tokenizer, converting all strings to lowercase and filtering out
-stop words and characters from the index.  By stop words and characters I mean common language
-words such as articles (a, an, the, etc.) and other strings that may have less value for searching
-(e.g. <b>'s</b>) .  It should be noted that there are different rules for every language, and you
-should use the proper analyzer for each.  Lucene currently provides Analyzers for a number of
-different languages (see the <code>*Analyzer.java</code> sources under <a
-href="http://svn.apache.org/repos/asf/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/">modules/analysis/common/src/java/org/apache/lucene/analysis</a>).
+The <code>-docs</code> command-line parameter value is the location of the directory containing
+files to be indexed.
+</p>
+  
+<p>
+The <code>-update</code> command-line parameter tells <code>IndexFiles</code> not to delete the
+index if it already exists.  When <code>-update</code> is not given, <code>IndexFiles</code> will
+first wipe the slate clean before indexing any documents.
 </p>
 
 <p>
-Looking further down in the file, you should see the <code>indexDocs()</code> code.  This recursive
-function simply crawls the directories and uses <a
-href="api/demo/org/apache/lucene/demo/FileDocument.html">FileDocument</a> to create <a
-href="api/core/org/apache/lucene/document/Document.html">Document</a> objects.  The <a
-href="api/core/org/apache/lucene/document/Document.html">Document</a> is simply a data object to
-represent the content in the file as well as its creation time and location.  These instances are
-added to the <code>indexWriter</code>.  Take a look inside <a
-href="api/demo/org/apache/lucene/demo/FileDocument.html">FileDocument</a>.  It's not particularly
-complicated.  It just adds fields to the <a
-href="api/core/org/apache/lucene/document/Document.html">Document</a>.
+Lucene <a href="api/core/org/apache/lucene/store/Directory.html">Directory</a>s are used by the
+<code>IndexWriter</code> to store information in the index.  In addition to the 
+<a href="api/core/org/apache/lucene/store/FSDirectory.html">FSDirectory</a> implementation we are using,
+there are several other <code>Directory</code> subclasses that can write to RAM, to databases, etc.
+</p>
+  
+<p>
+Lucene <a href="api/core/org/apache/lucene/analysis/Analyzer.html">Analyzer</a>s are processing pipelines
+that break up text into indexed tokens, a.k.a. terms, and optionally perform other operations on these
+tokens, e.g. downcasing, synonym insertion, filtering out unwanted tokens, etc.  The <code>Analyzer</code>
+we are using is <code>StandardAnalyzer</code>, which creates tokens using the Word Break rules from the
+Unicode Text Segmentation algorithm specified in <a href="http://unicode.org/reports/tr29/">Unicode
+Standard Annex #29</a>; converts tokens to lowercase; and then filters out stopwords.  Stopwords are
+common language words such as articles (a, an, the, etc.) and other tokens that may have less value for
+searching.  It should be noted that there are different rules for every language, and you should use the
+proper analyzer for each.  Lucene currently provides Analyzers for a number of different languages (see
+the javadocs under 
+<a href="api/module-analysis-common/org/apache/lucene/analysis/package-summary.html"
+>modules/analysis/common/src/java/org/apache/lucene/analysis</a>).
 </p>
 
 <p>
-As you can see there isn't much to creating an index.  The devil is in the details.  You may also
-wish to examine the other samples in this directory, particularly the <a
-href="api/demo/org/apache/lucene/demo/IndexHTML.html">IndexHTML</a> class.  It is a bit more
-complex but builds upon this example.
+The <code>IndexWriterConfig</code> instance holds all configuration for <code>IndexWriter</code>.  For
+example, we set the <code>OpenMode</code> to use here based on the value of the <code>-update</code>
+command-line parameter.
+</p>
+  
+<p>
+Looking further down in the file, after <code>IndexWriter</code> is instantiated, you should see the
+<code>indexDocs()</code> code.  This recursive function crawls the directories and creates
+<a href="api/core/org/apache/lucene/document/Document.html">Document</a> objects.  The 
+<code>Document</code> is simply a data object to represent the text content from the file as well as
+its creation time and location.  These instances are added to the <code>IndexWriter</code>.  If
+the <code>-update</code> command-line parameter is given, the <code>IndexWriter</code> 
+<code>OpenMode</code> will be set to <code>OpenMode.CREATE_OR_APPEND</code>, and rather than
+adding documents to the index, the <code>IndexWriter</code> will <strong>update</strong> them
+in the index by attempting to find an already-indexed document with the same identifier (in our
+case, the file path serves as the identifier); deleting it from the index if it exists; and then
+adding the new document to the index.
 </p>
 
 </section>
@@ -102,43 +125,30 @@ complex but builds upon this example.
 <section id="Searching Files"><title>Searching Files</title>
 
 <p>
-The <a href="api/demo/org/apache/lucene/demo/SearchFiles.html">SearchFiles</a> class is
-quite simple.  It primarily collaborates with an <a
-href="api/core/org/apache/lucene/search/IndexSearcher.html">IndexSearcher</a>, <a
-href="api/core/org/apache/lucene/analysis/standard/StandardAnalyzer.html">StandardAnalyzer</a>
-(which is used in the <a
-href="api/core/org/apache/lucene/demo/IndexFiles.html">IndexFiles</a> class as well) and a
-<a href="api/core/org/apache/lucene/queryParser/QueryParser.html">QueryParser</a>.  The
+The <a href="api/contrib-demo/org/apache/lucene/demo/SearchFiles.html">SearchFiles</a> class is
+quite simple.  It primarily collaborates with an 
+<a href="api/core/org/apache/lucene/search/IndexSearcher.html">IndexSearcher</a>, 
+<a href="api/modules-analysis-common/org/apache/lucene/analysis/standard/StandardAnalyzer.html"
+>StandardAnalyzer</a> (which is used in the
+<a href="api/contrib-demo/org/apache/lucene/demo/IndexFiles.html">IndexFiles</a> class as well)
+and a <a href="api/core/org/apache/lucene/queryParser/QueryParser.html">QueryParser</a>.  The
 query parser is constructed with an analyzer used to interpret your query text in the same way the
-documents are interpreted: finding the end of words and removing useless words like 'a', 'an' and
-'the'.  The <a href="api/core/org/apache/lucene/search/Query.html">Query</a> object contains
-the results from the <a
-href="api/core/org/apache/lucene/queryParser/QueryParser.html">QueryParser</a> which is passed to
-the searcher.  Note that it's also possible to programmatically construct a rich <a
-href="api/core/org/apache/lucene/search/Query.html">Query</a> object without using the query
+documents are interpreted: finding word boundaries, downcasing, and removing useless words like
+'a', 'an' and 'the'.  The <a href="api/core/org/apache/lucene/search/Query.html">Query</a>
+object contains the results from the
+<a href="api/core/org/apache/lucene/queryParser/QueryParser.html">QueryParser</a> which is passed
+to the searcher.  Note that it's also possible to programmatically construct a rich 
+<a href="api/core/org/apache/lucene/search/Query.html">Query</a> object without using the query
 parser.  The query parser just enables decoding the <a href="queryparsersyntax.html">Lucene query
-syntax</a> into the corresponding <a
-href="api/core/org/apache/lucene/search/Query.html">Query</a> object. Search can be executed in 
-two different ways: 
-<ul>
-<li>Streaming: A <a href="api/core/org/apache/lucene/search/Collector.html">Collector</a> subclass
-simply prints out the document ID and score for each matching document.</li>
-<li>Paging: Using a <a href="api/core/org/apache/lucene/search/TopScoreDocCollector.html">TopScoreDocCollector</a>
- the search results are printed in pages, sorted by score (i. e. relevance).</li>
-</ul>  
+syntax</a> into the corresponding <a href="api/core/org/apache/lucene/search/Query.html">Query</a>
+object.
 </p>
 
-</section>
-
-<section id="The Web example..."><title>The Web example...</title>
-
 <p>
-<a href="demo3.html">read on&gt;&gt;&gt;</a>
+<code>SearchFiles</code> uses the <code>IndexSearcher.search(query,n)</code> method that returns
+<a href="api/core/org/apache/lucene/search/TopDocs.html">TopDocs</a> with max <code>n</code> hits.
+The results are printed in pages, sorted by score (i.e. relevance).
 </p>
-
 </section>
-
 </body>
 </document>
-
-

Modified: lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/fileformats.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/fileformats.xml?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/fileformats.xml (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/fileformats.xml Tue Mar 15 21:35:17 2011
@@ -87,6 +87,10 @@
 	        uncompressed). See issue LUCENE-1960 for details.
             </p>
 
+        <p>
+            In version 3.1, segments records the code version
+            that created them. See LUCENE-2720 for details.
+           </p>
         </section>
 
         <section id="Definitions"><title>Definitions</title>
@@ -928,8 +932,8 @@
                     written twice.
                 </p>
                 <p>
-                    <b>2.9</b>
-                    Segments --&gt; Format, Version, NameCounter, SegCount, &lt;SegName, SegSize, DelGen, DocStoreOffset, [DocStoreSegment, DocStoreIsCompoundFile], HasSingleNormFile, NumField,
+                    <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
                 </p>
@@ -944,7 +948,7 @@
                 </p>
 
                 <p>
-                   SegName, DocStoreSegment --&gt; String
+                   SegVersion, SegName, DocStoreSegment --&gt; String
                 </p>
 
 		<p>
@@ -974,6 +978,10 @@
                 </p>
 
                 <p>
+                    SegVersion is the code version that created the segment.
+                </p>
+
+                <p>
                     SegName is the name of the segment, and is used as the file name prefix
                     for all of the files that compose the segment's index.
                 </p>

Modified: lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/gettingstarted.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/gettingstarted.xml?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/gettingstarted.xml (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/gettingstarted.xml Tue Mar 15 21:35:17 2011
@@ -14,7 +14,7 @@
     <title>Getting Started</title>
 <p>
 This document is intended as a "getting started" guide.  It has three audiences: first-time users
-looking to install Apache Lucene in their application or web server; developers looking to modify or base
+looking to install Apache Lucene in their application; developers looking to modify or base
 the applications they develop on Lucene; and developers looking to become involved in and contribute
 to the development of Lucene.  This document is written in tutorial and walk-through format.  The
 goal is to help you "get started".  It does not go into great depth on some of the conceptual or
@@ -33,20 +33,6 @@ may wish to skip sections.
 	<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/>
-
-	<li><a href="demo3.html">About installing and configuring the demo template web
-	application</a>.  While this walk-through assumes Tomcat as your container of choice,
-	there is no reason you can't (provided you have the requisite knowledge) adapt the
-	instructions to your container.  This section is intended for those responsible for the
-	development or deployment of Lucene-based web applications.</li> <p/>
-
-	<li><a href="demo4.html">About the sources used to construct the demo template web
-	application</a>.  Please note the template application is designed to highlight features of
-	Lucene and is <b>not</b> an example of best practices.  (One would hopefully use MVC
-	architecture such as provided by Jakarta Struts and taglibs, but showing you how to do that
-	would be WAY beyond the scope of this guide.)  This section is intended for developers and
-	those wishing to customize the demo template web application to their needs.  </li>
-
 </ul>
 </section>
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/lucene-contrib/index.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/lucene-contrib/index.xml?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/lucene-contrib/index.xml (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/lucene-contrib/index.xml Tue Mar 15 21:35:17 2011
@@ -59,6 +59,12 @@
                 <p>The benchmark contribution contains tools for benchmarking Lucene using standard, freely available corpora.</p>
                 <p>See <a href="../api/contrib-benchmark/index.html">benchmark javadoc</a></p>
             </section>            
+        
+            <section id="demo"><title>demo</title>
+                <p>The demo contrib contains the Lucene demo: IndexFiles and SearchFiles, described under
+                   <a href="../gettingstarted.html">Getting Started</a>.</p>
+                <p>See <a href="../api/contrib-demo/index.html">demo javadoc</a></p>
+            </section>
             
             <section id="db"><title>db</title>
                 <p>Provides integration with Berkley DB.</p>

Modified: lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/site.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/site.xml?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/site.xml (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/site/src/documentation/content/xdocs/site.xml Tue Mar 15 21:35:17 2011
@@ -50,12 +50,13 @@ See http://forrest.apache.org/docs/linki
       <javadoc label="Javadocs">
          <javadoc-all label="All" href="ext:javadocs-all"/>
 		 <javadoc-core label="Core" href="ext:javadocs-core"/>
-		 <javadoc-demo label="Demo" href="ext:javadocs-demo"/>
+		 <javadoc-test-framework label="Test Framework" href="ext:javadocs-test-framework"/>
 		 <javadoc-contrib label="Contrib">
 		    <javadoc-contrib-ant label="Ant" href="ext:javadocs-contrib-ant"/>
 		    <javadoc-contrib-bdb label="Bdb" href="ext:javadocs-contrib-bdb"/>
 		    <javadoc-contrib-bdb-je label="Bdb-je" href="ext:javadocs-contrib-bdb-je"/>
 		    <javadoc-contrib-benchmark label="Benchmark" href="ext:javadocs-contrib-benchmark"/>
+        <javadoc-contrib-demo label="Demo" href="ext:javadocs-contrib-demo"/>
 		    <javadoc-contrib-highlighter label="Highlighter" href="ext:javadocs-contrib-highlighter"/>
 		    <javadoc-contrib-instantiated label="Instantiated" href="ext:javadocs-contrib-instantiated"/>
 		    <javadoc-contrib-lucli label="Lucli" href="ext:javadocs-contrib-lucli"/>
@@ -98,11 +99,12 @@ See http://forrest.apache.org/docs/linki
     <changes-contrib href="changes/Contrib-Changes.html" />
     <javadocs-all href="api/all/index.html"/>
 	<javadocs-core href="api/core/index.html"/>
-	<javadocs-demo href="api/demo/index.html"/>
+	<javadocs-test-framework href="api/test-framework/index.html"/>
 	<javadocs-contrib-ant href="api/contrib-ant/index.html"/>
 	<javadocs-contrib-bdb href="api/contrib-bdb/index.html"/>
 	<javadocs-contrib-bdb-je href="api/contrib-bdb-je/index.html"/>
 	<javadocs-contrib-benchmark href="api/contrib-benchmark/index.html"/>
+  <javadocs-contrib-demo href="api/contrib-demo/index.html"/>
 	<javadocs-contrib-highlighter href="api/contrib-highlighter/index.html"/>
 	<javadocs-contrib-instantiated href="api/contrib-instantiated/index.html"/>
 	<javadocs-contrib-lucli href="api/contrib-lucli/index.html"/>

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java Tue Mar 15 21:35:17 2011
@@ -36,6 +36,10 @@ public final class MockAnalyzer extends 
   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);    
   }
@@ -47,7 +51,7 @@ public final class MockAnalyzer extends 
    * @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
+   * @param payload if payloads should be added containing the positions (for testing)
    */
   public MockAnalyzer(CharacterRunAutomaton runAutomaton, boolean lowerCase, CharacterRunAutomaton filter, boolean enablePositionIncrements, boolean payload) {
     this.runAutomaton = runAutomaton;
@@ -58,21 +62,26 @@ public final class MockAnalyzer extends 
   }
 
   /**
-   * Creates a new MockAnalyzer, with no filtering.
-   * 
-   * @param runAutomaton DFA describing how tokenization should happen (e.g. [a-zA-Z]+)
-   * @param lowerCase true if the tokenizer should lowercase terms
+   * Calls {@link #MockAnalyzer(CharacterRunAutomaton, boolean, CharacterRunAutomaton, boolean, boolean) 
+   * MockAnalyzer(runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false, true}).
    */
   public MockAnalyzer(CharacterRunAutomaton runAutomaton, boolean lowerCase) {
     this(runAutomaton, lowerCase, MockTokenFilter.EMPTY_STOPSET, false, true);
   }
 
+  /**
+   * 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 
+   * 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}).
    */
   public MockAnalyzer() {
     this(MockTokenizer.WHITESPACE, true);

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenFilter.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenFilter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenFilter.java Tue Mar 15 21:35:17 2011
@@ -60,6 +60,13 @@ public final class MockTokenFilter exten
   private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
   private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
   
+  /**
+   * Create a new MockTokenFilter.
+   * 
+   * @param input TokenStream to filter
+   * @param filter DFA representing the terms that should be removed.
+   * @param enablePositionIncrements true if the removal should accumulate position increments.
+   */
   public MockTokenFilter(TokenStream input, CharacterRunAutomaton filter, boolean enablePositionIncrements) {
     super(input);
     this.filter = filter;

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java Tue Mar 15 21:35:17 2011
@@ -20,7 +20,7 @@ package org.apache.lucene.analysis;
 import java.io.IOException;
 import java.io.Reader;
 
-import org.apache.lucene.util.Version;
+import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
 
@@ -46,14 +46,14 @@ public class MockTokenizer extends CharT
   private int state;
 
   public MockTokenizer(AttributeFactory factory, Reader input, CharacterRunAutomaton runAutomaton, boolean lowerCase) {
-    super(Version.LUCENE_CURRENT, factory, input);
+    super(LuceneTestCase.TEST_VERSION_CURRENT, factory, input);
     this.runAutomaton = runAutomaton;
     this.lowerCase = lowerCase;
     this.state = runAutomaton.getInitialState();
   }
 
   public MockTokenizer(Reader input, CharacterRunAutomaton runAutomaton, boolean lowerCase) {
-    super(Version.LUCENE_CURRENT, input);
+    super(LuceneTestCase.TEST_VERSION_CURRENT, input);
     this.runAutomaton = runAutomaton;
     this.lowerCase = lowerCase;
     this.state = runAutomaton.getInitialState();

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java Tue Mar 15 21:35:17 2011
@@ -24,6 +24,7 @@ import java.util.Random;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexWriter; // javadoc
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.Version;
@@ -91,6 +92,10 @@ public class RandomIndexWriter implement
     }
   } 
 
+  /**
+   * Adds a Document.
+   * @see IndexWriter#addDocument(Document)
+   */
   public void addDocument(Document doc) throws IOException {
     w.addDocument(doc);
     if (docCount++ == flushAt) {
@@ -102,6 +107,21 @@ public class RandomIndexWriter implement
     }
   }
   
+  /**
+   * Updates a document.
+   * @see IndexWriter#updateDocument(Term, Document)
+   */
+  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);
+    }
+  }
+  
   public void addIndexes(Directory... dirs) throws CorruptIndexException, IOException {
     w.addIndexes(dirs);
   }
@@ -127,17 +147,21 @@ public class RandomIndexWriter implement
   }
 
   public IndexReader getReader() throws IOException {
+    return getReader(true);
+  }
+
+  public IndexReader getReader(boolean applyDeletions) throws IOException {
     getReaderCalled = true;
     if (r.nextInt(4) == 2)
       w.optimize();
     // If we are writing with PreFlexRW, force a full
     // IndexReader.open so terms are sorted in codepoint
     // order during searching:
-    if (!w.codecs.getDefaultFieldCodec().equals("PreFlex") && r.nextBoolean()) {
+    if (!applyDeletions || !w.codecs.getDefaultFieldCodec().equals("PreFlex") && r.nextBoolean()) {
       if (LuceneTestCase.VERBOSE) {
         System.out.println("RIW.getReader: use NRT reader");
       }
-      return w.getReader();
+      return w.getReader(applyDeletions);
     } else {
       if (LuceneTestCase.VERBOSE) {
         System.out.println("RIW.getReader: open new reader");
@@ -147,6 +171,10 @@ public class RandomIndexWriter implement
     }
   }
 
+  /**
+   * Close this writer.
+   * @see IndexWriter#close()
+   */
   public void close() throws IOException {
     // if someone isn't using getReader() API, we want to be sure to
     // maybeOptimize since presumably they might open a reader on the dir.
@@ -156,6 +184,13 @@ public class RandomIndexWriter implement
     w.close();
   }
 
+  /**
+   * Forces an optimize.
+   * <p>
+   * NOTE: this should be avoided in tests unless absolutely necessary,
+   * as it will result in less test coverage.
+   * @see IndexWriter#optimize()
+   */
   public void optimize() throws IOException {
     w.optimize();
   }

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java Tue Mar 15 21:35:17 2011
@@ -130,7 +130,7 @@ public class MockFixedIntBlockCodec exte
 
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
       success = true;
       return ret;
     } finally {
@@ -174,7 +174,6 @@ public class MockFixedIntBlockCodec exte
                                                 state.segmentInfo.name,
                                                 postingsReader,
                                                 state.readBufferSize,
-                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
                                                 StandardCodec.TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java Tue Mar 15 21:35:17 2011
@@ -150,7 +150,7 @@ public class MockVariableIntBlockCodec e
 
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
       success = true;
       return ret;
     } finally {
@@ -195,7 +195,6 @@ public class MockVariableIntBlockCodec e
                                                 state.segmentInfo.name,
                                                 postingsReader,
                                                 state.readBufferSize,
-                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
                                                 StandardCodec.TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java Tue Mar 15 21:35:17 2011
@@ -205,7 +205,7 @@ public class MockRandomCodec extends Cod
 
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
       success = true;
       return ret;
     } finally {
@@ -306,7 +306,6 @@ public class MockRandomCodec extends Cod
                                                 state.segmentInfo.name,
                                                 postingsReader,
                                                 state.readBufferSize,
-                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
                                                 termsCacheSize,
                                                 state.codecId);
       success = true;

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java Tue Mar 15 21:35:17 2011
@@ -70,7 +70,7 @@ public class MockSepCodec extends Codec 
 
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
       success = true;
       return ret;
     } finally {
@@ -114,7 +114,6 @@ public class MockSepCodec extends Codec 
                                                 state.segmentInfo.name,
                                                 postingsReader,
                                                 state.readBufferSize,
-                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
                                                 StandardCodec.TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java Tue Mar 15 21:35:17 2011
@@ -27,7 +27,7 @@ import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.index.BulkPostingsEnum;
 
 /** Reads IndexInputs written with {@link
- *  SingleIntIndexOutput}.  NOTE: this class is just for
+ *  MockSingleIntIndexOutput}.  NOTE: this class is just for
  *  demonstration puprposes (it is a very slow way to read a
  *  block of ints).
  *

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/search/CheckHits.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/search/CheckHits.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/search/CheckHits.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/search/CheckHits.java Tue Mar 15 21:35:17 2011
@@ -39,8 +39,8 @@ public class CheckHits {
     
   /**
    * Tests that all documents up to maxDoc which are *not* in the
-   * expected result set, have an explanation which indicates no match
-   * (ie: Explanation value of 0.0f)
+   * expected result set, have an explanation which indicates that 
+   * the document does not match
    */
   public static void checkNoMatchExplanations(Query q, String defaultFieldName,
                                               IndexSearcher searcher, int[] results)
@@ -59,9 +59,9 @@ public class CheckHits {
       Explanation exp = searcher.explain(q, doc);
       Assert.assertNotNull("Explanation of [["+d+"]] for #"+doc+" is null",
                              exp);
-      Assert.assertEquals("Explanation of [["+d+"]] for #"+doc+
-                            " doesn't indicate non-match: " + exp.toString(),
-                            0.0f, exp.getValue(), 0.0f);
+      Assert.assertFalse("Explanation of [["+d+"]] for #"+doc+
+                         " doesn't indicate non-match: " + exp.toString(),
+                         exp.isMatch());
     }
     
   }
@@ -268,7 +268,7 @@ public class CheckHits {
    * query corresponds with the true score. 
    *
    * @see ExplanationAsserter
-   * @see #checkExplanations(Query, String, Searcher, boolean) for a
+   * @see #checkExplanations(Query, String, IndexSearcher, boolean) for a
    * "deep" testing of the explanation details.
    *   
    * @param query the query to test
@@ -484,6 +484,9 @@ public class CheckHits {
       
       Assert.assertNotNull("Explanation of [["+d+"]] for #"+doc+" is null", exp);
       verifyExplanation(d,doc,scorer.score(),deep,exp);
+      Assert.assertTrue("Explanation of [["+d+"]] for #"+ doc + 
+                        " does not indicate match: " + exp.toString(), 
+                        exp.isMatch());
     }
     @Override
     public void setNextReader(AtomicReaderContext context) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java Tue Mar 15 21:35:17 2011
@@ -2,14 +2,13 @@ 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;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.MultiReader;
@@ -19,6 +18,7 @@ import org.apache.lucene.store.MockDirec
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.ReaderUtil;
+import org.apache.lucene.util._TestUtil;
 
 import static org.apache.lucene.util.LuceneTestCase.TEST_VERSION_CURRENT;
 
@@ -172,16 +172,7 @@ public class QueryUtils {
       }
       w.commit();
       w.deleteDocuments( new MatchAllDocsQuery() );
-      try {
-        // Carefully invoke what is a package-private (test
-        // only, internal) method on IndexWriter:
-        Method m = IndexWriter.class.getDeclaredMethod("keepFullyDeletedSegments");
-        m.setAccessible(true);
-        m.invoke(w);
-      } catch (Exception e) {
-        // Should not happen?
-        throw new RuntimeException(e);
-      }
+      _TestUtil.keepFullyDeletedSegments(w);
       w.commit();
 
       if (0 < numDeletedDocs)