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

svn commit: r1101062 [8/21] - in /lucene/dev/branches/bulkpostings: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/ant/ dev-tools/idea/lucene/contrib/db/bdb-je/ dev-tools/idea/lucene/contrib/db/bdb/ dev-tools/idea/...

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java Mon May  9 15:24:04 2011
@@ -18,12 +18,12 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Map;
 
+import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.util.BytesRef;
+
 /** This class implements {@link InvertedDocConsumer}, which
  *  is passed each token produced by the analyzer on each
  *  field.  It stores these tokens in a hash table, and
@@ -36,78 +36,118 @@ final class TermsHash extends InvertedDo
 
   final TermsHashConsumer consumer;
   final TermsHash nextTermsHash;
-  final DocumentsWriter docWriter;
+  final DocumentsWriterPerThread docWriter;
+
+  final IntBlockPool intPool;
+  final ByteBlockPool bytePool;
+  ByteBlockPool termBytePool;
+
+  final boolean primary;
+  final DocumentsWriterPerThread.DocState docState;
+
+  // Used when comparing postings via termRefComp, in TermsHashPerField
+  final BytesRef tr1 = new BytesRef();
+  final BytesRef tr2 = new BytesRef();
 
-  boolean trackAllocations;
+  // Used by perField to obtain terms from the analysis chain
+  final BytesRef termBytesRef = new BytesRef(10);
 
-  public TermsHash(final DocumentsWriter docWriter, boolean trackAllocations, final TermsHashConsumer consumer, final TermsHash nextTermsHash) {
+  final boolean trackAllocations;
+
+
+  public TermsHash(final DocumentsWriterPerThread docWriter, final TermsHashConsumer consumer, boolean trackAllocations, final TermsHash nextTermsHash) {
+    this.docState = docWriter.docState;
     this.docWriter = docWriter;
     this.consumer = consumer;
+    this.trackAllocations = trackAllocations; 
     this.nextTermsHash = nextTermsHash;
-    this.trackAllocations = trackAllocations;
+    intPool = new IntBlockPool(docWriter);
+    bytePool = new ByteBlockPool(docWriter.byteBlockAllocator);
+
+    if (nextTermsHash != null) {
+      // We are primary
+      primary = true;
+      termBytePool = bytePool;
+      nextTermsHash.termBytePool = bytePool;
+    } else {
+      primary = false;
+    }
   }
 
   @Override
-  InvertedDocConsumerPerThread addThread(DocInverterPerThread docInverterPerThread) {
-    return new TermsHashPerThread(docInverterPerThread, this, nextTermsHash, null);
+  public void abort() {
+    reset();
+    try {
+      consumer.abort();
+    } finally {
+      if (nextTermsHash != null) {
+        nextTermsHash.abort();
+      }
+    }
   }
 
-  TermsHashPerThread addThread(DocInverterPerThread docInverterPerThread, TermsHashPerThread primaryPerThread) {
-    return new TermsHashPerThread(docInverterPerThread, this, nextTermsHash, primaryPerThread);
-  }
+  // Clear all state
+  void reset() {
+    intPool.reset();
+    bytePool.reset();
 
-  @Override
-  public void abort() {
-    consumer.abort();
-    if (nextTermsHash != null)
-      nextTermsHash.abort();
+    if (primary) {
+      bytePool.reset();
+    }
   }
 
   @Override
-  synchronized void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
-    Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> childThreadsAndFields = new HashMap<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>>();
-    Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> nextThreadsAndFields;
-
-    if (nextTermsHash != null)
-      nextThreadsAndFields = new HashMap<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>>();
-    else
-      nextThreadsAndFields = null;
+  void flush(Map<FieldInfo,InvertedDocConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
+    Map<FieldInfo,TermsHashConsumerPerField> childFields = new HashMap<FieldInfo,TermsHashConsumerPerField>();
+    Map<FieldInfo,InvertedDocConsumerPerField> nextChildFields;
+
+    if (nextTermsHash != null) {
+      nextChildFields = new HashMap<FieldInfo,InvertedDocConsumerPerField>();
+    } else {
+      nextChildFields = null;
+    }
 
-    for (final Map.Entry<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> entry : threadsAndFields.entrySet()) {
+    for (final Map.Entry<FieldInfo,InvertedDocConsumerPerField> entry : fieldsToFlush.entrySet()) {
+        TermsHashPerField perField = (TermsHashPerField) entry.getValue();
+        childFields.put(entry.getKey(), perField.consumer);
+        if (nextTermsHash != null) {
+          nextChildFields.put(entry.getKey(), perField.nextPerField);
+        }
+    }
 
-      TermsHashPerThread perThread = (TermsHashPerThread) entry.getKey();
+    consumer.flush(childFields, state);
 
-      Collection<InvertedDocConsumerPerField> fields = entry.getValue();
+    if (nextTermsHash != null) {
+      nextTermsHash.flush(nextChildFields, state);
+    }
+  }
 
-      Iterator<InvertedDocConsumerPerField> fieldsIt = fields.iterator();
-      Collection<TermsHashConsumerPerField> childFields = new HashSet<TermsHashConsumerPerField>();
-      Collection<InvertedDocConsumerPerField> nextChildFields;
+  @Override
+  InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
+    return new TermsHashPerField(docInverterPerField, this, nextTermsHash, fieldInfo);
+  }
 
-      if (nextTermsHash != null)
-        nextChildFields = new HashSet<InvertedDocConsumerPerField>();
-      else
-        nextChildFields = null;
+  @Override
+  public boolean freeRAM() {
+    return false;
+  }
 
-      while(fieldsIt.hasNext()) {
-        TermsHashPerField perField = (TermsHashPerField) fieldsIt.next();
-        childFields.add(perField.consumer);
-        if (nextTermsHash != null)
-          nextChildFields.add(perField.nextPerField);
+  @Override
+  void finishDocument() throws IOException {
+    try {
+      consumer.finishDocument(this);
+    } finally {
+      if (nextTermsHash != null) {
+        nextTermsHash.consumer.finishDocument(nextTermsHash);
       }
-
-      childThreadsAndFields.put(perThread.consumer, childFields);
-      if (nextTermsHash != null)
-        nextThreadsAndFields.put(perThread.nextPerThread, nextChildFields);
     }
-    
-    consumer.flush(childThreadsAndFields, state);
-
-    if (nextTermsHash != null)
-      nextTermsHash.flush(nextThreadsAndFields, state);
   }
 
   @Override
-  synchronized public boolean freeRAM() {
-    return false;
+  void startDocument() throws IOException {
+    consumer.startDocument();
+    if (nextTermsHash != null) {
+      nextTermsHash.consumer.startDocument();
+    }
   }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java Mon May  9 15:24:04 2011
@@ -18,11 +18,12 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.Collection;
 import java.util.Map;
 
 abstract class TermsHashConsumer {
-  abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread);
-  abstract void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException;
+  abstract void flush(Map<FieldInfo, TermsHashConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException;
   abstract void abort();
-  }
+  abstract void startDocument() throws IOException;
+  abstract void finishDocument(TermsHash termsHash) throws IOException;
+  abstract public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo);
+}

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java Mon May  9 15:24:04 2011
@@ -34,9 +34,10 @@ final class TermsHashPerField extends In
 
   final TermsHashConsumerPerField consumer;
 
+  final TermsHash termsHash;
+
   final TermsHashPerField nextPerField;
-  final TermsHashPerThread perThread;
-  final DocumentsWriter.DocState docState;
+  final DocumentsWriterPerThread.DocState docState;
   final FieldInvertState fieldState;
   TermToBytesRefAttribute termAtt;
   BytesRef termBytesRef;
@@ -52,27 +53,27 @@ final class TermsHashPerField extends In
   final FieldInfo fieldInfo;
 
   final BytesRefHash bytesHash;
- 
+
   ParallelPostingsArray postingsArray;
   private final AtomicLong bytesUsed;
 
-  public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHashPerThread perThread, final TermsHashPerThread nextPerThread, final FieldInfo fieldInfo) {
-    this.perThread = perThread;
-    intPool = perThread.intPool;
-    bytePool = perThread.bytePool;
-    termBytePool = perThread.termBytePool;
-    docState = perThread.docState;
-    bytesUsed =  perThread.termsHash.trackAllocations?perThread.termsHash.docWriter.bytesUsed:new AtomicLong();
-
+  public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHash termsHash, final TermsHash nextTermsHash, final FieldInfo fieldInfo) {
+    intPool = termsHash.intPool;
+    bytePool = termsHash.bytePool;
+    termBytePool = termsHash.termBytePool;
+    docState = termsHash.docState;
+    this.termsHash = termsHash;
+    bytesUsed = termsHash.trackAllocations ? termsHash.docWriter.bytesUsed
+        : new AtomicLong();
     fieldState = docInverterPerField.fieldState;
-    this.consumer = perThread.consumer.addField(this, fieldInfo);
+    this.consumer = termsHash.consumer.addField(this, fieldInfo);
     PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
-    bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts); 
+    bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts);
     streamCount = consumer.getStreamCount();
     numPostingInt = 2*streamCount;
     this.fieldInfo = fieldInfo;
-    if (nextPerThread != null)
-      nextPerField = (TermsHashPerField) nextPerThread.addField(docInverterPerField, fieldInfo);
+    if (nextTermsHash != null)
+      nextPerField = (TermsHashPerField) nextTermsHash.addField(docInverterPerField, fieldInfo);
     else
       nextPerField = null;
   }
@@ -80,7 +81,7 @@ final class TermsHashPerField extends In
   void shrinkHash(int targetSize) {
     // Fully free the bytesHash on each flush but keep the pool untouched
     // bytesHash.clear will clear the ByteStartArray and in turn the ParallelPostingsArray too
-    bytesHash.clear(false); 
+    bytesHash.clear(false);
   }
 
   public void reset() {
@@ -90,7 +91,7 @@ final class TermsHashPerField extends In
   }
 
   @Override
-  synchronized public void abort() {
+  public void abort() {
     reset();
     if (nextPerField != null)
       nextPerField.abort();
@@ -99,14 +100,13 @@ final class TermsHashPerField extends In
   public void initReader(ByteSliceReader reader, int termID, int stream) {
     assert stream < streamCount;
     int intStart = postingsArray.intStarts[termID];
-    final int[] ints = intPool.buffers[intStart >> DocumentsWriter.INT_BLOCK_SHIFT];
-    final int upto = intStart & DocumentsWriter.INT_BLOCK_MASK;
+    final int[] ints = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT];
+    final int upto = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK;
     reader.init(bytePool,
                 postingsArray.byteStarts[termID]+stream*ByteBlockPool.FIRST_LEVEL_SIZE,
                 ints[upto+stream]);
   }
 
-
   /** Collapse the hash table & sort in-place. */
   public int[] sortPostings(Comparator<BytesRef> termComp) {
    return bytesHash.sort(termComp);
@@ -124,7 +124,7 @@ final class TermsHashPerField extends In
       nextPerField.start(f);
     }
   }
-  
+
   @Override
   boolean start(Fieldable[] fields, int count) throws IOException {
     doCall = consumer.start(fields, count);
@@ -143,11 +143,12 @@ final class TermsHashPerField extends In
       // First time we are seeing this token since we last
       // flushed the hash.
       // Init stream slices
-      if (numPostingInt + intPool.intUpto > DocumentsWriter.INT_BLOCK_SIZE)
+      if (numPostingInt + intPool.intUpto > DocumentsWriterPerThread.INT_BLOCK_SIZE)
         intPool.nextBuffer();
 
-      if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE)
+      if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE) {
         bytePool.nextBuffer();
+      }
 
       intUptos = intPool.buffer;
       intUptoStart = intPool.intUpto;
@@ -166,8 +167,8 @@ final class TermsHashPerField extends In
     } else {
       termID = (-termID)-1;
       int intStart = postingsArray.intStarts[termID];
-      intUptos = intPool.buffers[intStart >> DocumentsWriter.INT_BLOCK_SHIFT];
-      intUptoStart = intStart & DocumentsWriter.INT_BLOCK_MASK;
+      intUptos = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT];
+      intUptoStart = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK;
       consumer.addTerm(termID);
     }
   }
@@ -192,7 +193,7 @@ final class TermsHashPerField extends In
       if (docState.maxTermPrefix == null) {
         final int saved = termBytesRef.length;
         try {
-          termBytesRef.length = Math.min(30, DocumentsWriter.MAX_TERM_LENGTH_UTF8);
+          termBytesRef.length = Math.min(30, DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8);
           docState.maxTermPrefix = termBytesRef.toString();
         } finally {
           termBytesRef.length = saved;
@@ -204,7 +205,7 @@ final class TermsHashPerField extends In
     if (termID >= 0) {// New posting
       bytesHash.byteStart(termID);
       // Init stream slices
-      if (numPostingInt + intPool.intUpto > DocumentsWriter.INT_BLOCK_SIZE) {
+      if (numPostingInt + intPool.intUpto > DocumentsWriterPerThread.INT_BLOCK_SIZE) {
         intPool.nextBuffer();
       }
 
@@ -229,8 +230,8 @@ final class TermsHashPerField extends In
     } else {
       termID = (-termID)-1;
       final int intStart = postingsArray.intStarts[termID];
-      intUptos = intPool.buffers[intStart >> DocumentsWriter.INT_BLOCK_SHIFT];
-      intUptoStart = intStart & DocumentsWriter.INT_BLOCK_MASK;
+      intUptos = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT];
+      intUptoStart = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK;
       consumer.addTerm(termID);
     }
 
@@ -278,7 +279,7 @@ final class TermsHashPerField extends In
     if (nextPerField != null)
       nextPerField.finish();
   }
-  
+
   private static final class PostingsBytesStartArray extends BytesStartArray {
 
     private final TermsHashPerField perField;
@@ -289,10 +290,10 @@ final class TermsHashPerField extends In
       this.perField = perField;
       this.bytesUsed = bytesUsed;
     }
-    
+
     @Override
     public int[] init() {
-      if(perField.postingsArray == null) { 
+      if(perField.postingsArray == null) {
         perField.postingsArray = perField.consumer.createPostingsArray(2);
         bytesUsed.addAndGet(perField.postingsArray.size * perField.postingsArray.bytesPerPosting());
       }
@@ -312,7 +313,7 @@ final class TermsHashPerField extends In
     @Override
     public int[] clear() {
       if(perField.postingsArray != null) {
-        bytesUsed.addAndGet(-perField.postingsArray.size * perField.postingsArray.bytesPerPosting());
+        bytesUsed.addAndGet(-(perField.postingsArray.size * perField.postingsArray.bytesPerPosting()));
         perField.postingsArray = null;
       }
       return null;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java Mon May  9 15:24:04 2011
@@ -152,10 +152,10 @@ public class BlockTermsReader extends Fi
   }
 
   protected void readHeader(IndexInput input) throws IOException {
-    CodecUtil.checkHeader(in, BlockTermsWriter.CODEC_NAME,
+    CodecUtil.checkHeader(input, BlockTermsWriter.CODEC_NAME,
                           BlockTermsWriter.VERSION_START,
                           BlockTermsWriter.VERSION_CURRENT);
-    dirOffset = in.readLong();    
+    dirOffset = input.readLong();
   }
   
   protected void seekDir(IndexInput input, long dirOffset)
@@ -850,6 +850,11 @@ public class BlockTermsReader extends Fi
       private void decodeMetaData() throws IOException {
         //System.out.println("BTR.decodeMetadata mdUpto=" + metaDataUpto + " vs termCount=" + state.termCount + " state=" + state);
         if (!seekPending) {
+          // TODO: cutover to random-access API
+          // here.... really stupid that we have to decode N
+          // wasted term metadata just to get to the N+1th
+          // that we really need...
+
           // lazily catch up on metadata decode:
           final int limit = state.termCount;
           // We must set/incr state.termCount because

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java Mon May  9 15:24:04 2011
@@ -70,6 +70,11 @@ public class CodecProvider {
       }
     }
   }
+  
+  /** @lucene.internal */
+  public synchronized Set<String> listAll() {
+    return codecs.keySet();
+  }
 
   public Collection<String> getAllExtensions() {
     return knownExtensions;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListReader.java Mon May  9 15:24:04 2011
@@ -184,9 +184,21 @@ public abstract class MultiLevelSkipList
     }
   }
   
+  /** returns x == 0 ? 0 : Math.floor(Math.log(x) / Math.log(base)) */
+  static int log(int x, int base) {
+    assert base >= 2;
+    int ret = 0;
+    long n = base; // needs to be a long to avoid overflow
+    while (x >= n) {
+      n *= base;
+      ret++;
+    }
+    return ret;
+  }
+  
   /** Loads the skip levels  */
   private void loadSkipLevels() throws IOException {
-    numberOfSkipLevels = docCount == 0 ? 0 : (int) Math.floor(Math.log(docCount) / Math.log(skipInterval[0]));
+    numberOfSkipLevels = log(docCount, skipInterval[0]);
     if (numberOfSkipLevels > maxNumberOfSkipLevels) {
       numberOfSkipLevels = maxNumberOfSkipLevels;
     }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListWriter.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/MultiLevelSkipListWriter.java Mon May  9 15:24:04 2011
@@ -61,7 +61,7 @@ public abstract class MultiLevelSkipList
     this.skipInterval = skipInterval;
     
     // calculate the maximum number of skip levels for this document frequency
-    numberOfSkipLevels = df == 0 ? 0 : (int) Math.floor(Math.log(df) / Math.log(skipInterval));
+    numberOfSkipLevels = MultiLevelSkipListReader.log(df, skipInterval);
     
     // make sure it does not exceed maxSkipLevels
     if (numberOfSkipLevels > maxSkipLevels) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java Mon May  9 15:24:04 2011
@@ -18,6 +18,9 @@ package org.apache.lucene.index.codecs;
  */
 
 import java.io.IOException;
+import java.io.FileOutputStream;   // for toDot
+import java.io.OutputStreamWriter; // for toDot
+import java.io.Writer;             // for toDot
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -34,6 +37,7 @@ import org.apache.lucene.util.automaton.
 import org.apache.lucene.util.automaton.fst.BytesRefFSTEnum;
 import org.apache.lucene.util.automaton.fst.FST;
 import org.apache.lucene.util.automaton.fst.PositiveIntOutputs;
+import org.apache.lucene.util.automaton.fst.Util; // for toDot
 
 /** See {@link VariableGapTermsIndexWriter}
  * 
@@ -52,11 +56,13 @@ public class VariableGapTermsIndexReader
   // start of the field info data
   protected long dirOffset;
 
+  final String segment;
+
   public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, String codecId)
     throws IOException {
 
     in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
-    
+    this.segment = segment;
     boolean success = false;
 
     try {
@@ -176,6 +182,14 @@ public class VariableGapTermsIndexReader
         fst = new FST<Long>(clone, fstOutputs);
         clone.close();
 
+        /*
+        final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
+        Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
+        Util.toDot(fst, w, false, false);
+        System.out.println("FST INDEX: SAVED to " + dotFileName);
+        w.close();
+        */
+
         if (indexDivisor > 1) {
           // subsample
           final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java Mon May  9 15:24:04 2011
@@ -310,7 +310,7 @@ public final class TermInfosReader {
         }
       } else {
         assert sameTermInfo(ti, tiOrd, enumerator);
-        assert (int) enumerator.position == tiOrd.termOrd;
+        assert enumerator.position == tiOrd.termOrd;
       }
     } else {
       ti = null;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java Mon May  9 15:24:04 2011
@@ -69,15 +69,8 @@ public class PulsingPostingsReaderImpl e
 
     @Override
     public Object clone() {
-      PulsingTermState clone;
-      clone = (PulsingTermState) super.clone();
-      if (postingsSize != -1) {
-        clone.postings = new byte[postingsSize];
-        System.arraycopy(postings, 0, clone.postings, 0, postingsSize);
-      } else {
-        assert wrappedTermState != null;
-        clone.wrappedTermState = (BlockTermState) wrappedTermState.clone();
-      }
+      PulsingTermState clone = new PulsingTermState();
+      clone.copyFrom(this);
       return clone;
     }
 
@@ -91,8 +84,10 @@ public class PulsingPostingsReaderImpl e
           postings = new byte[ArrayUtil.oversize(other.postingsSize, 1)];
         }
         System.arraycopy(other.postings, 0, postings, 0, other.postingsSize);
-      } else {
+      } else if (wrappedTermState != null) {
         wrappedTermState.copyFrom(other.wrappedTermState);
+      } else {
+        wrappedTermState = (BlockTermState) other.wrappedTermState.clone();
       }
 
       // NOTE: we do not copy the
@@ -259,6 +254,7 @@ public class PulsingPostingsReaderImpl e
     private Bits skipDocs;
     private int docID;
     private int freq;
+    private int payloadLength;
 
     public PulsingDocsEnum(FieldInfo fieldInfo) {
       omitTF = fieldInfo.omitTermFreqAndPositions;
@@ -272,6 +268,7 @@ public class PulsingPostingsReaderImpl e
       System.arraycopy(termState.postings, 0, bytes, 0, termState.postingsSize);
       postings.reset(bytes);
       docID = 0;
+      payloadLength = 0;
       freq = 1;
       this.skipDocs = skipDocs;
       return this;
@@ -303,7 +300,6 @@ public class PulsingPostingsReaderImpl e
 
           // Skip positions
           if (storePayloads) {
-            int payloadLength = -1;
             for(int pos=0;pos<freq;pos++) {
               final int posCode = postings.readVInt();
               if ((posCode & 1) != 0) {
@@ -583,6 +579,7 @@ public class PulsingPostingsReaderImpl e
       postings.reset(bytes);
       this.skipDocs = skipDocs;
       payloadLength = 0;
+      posPending = 0;
       docID = 0;
       //System.out.println("PR d&p reset storesPayloads=" + storePayloads + " bytes=" + bytes.length + " this=" + this);
       return this;
@@ -590,7 +587,7 @@ public class PulsingPostingsReaderImpl e
 
     @Override
     public int nextDoc() throws IOException {
-      //System.out.println("PR d&p nextDoc this=" + this);
+      //System.out.println("PR.nextDoc this=" + this);
 
       while(true) {
         //System.out.println("  cycle skip posPending=" + posPending);
@@ -598,15 +595,16 @@ public class PulsingPostingsReaderImpl e
         skipPositions();
 
         if (postings.eof()) {
-          //System.out.println("PR   END");
+          //System.out.println("  END");
           return docID = NO_MORE_DOCS;
         }
-
+        //System.out.println("  read doc code");
         final int code = postings.readVInt();
         docID += code >>> 1;            // shift off low bit
         if ((code & 1) != 0) {          // if low bit is set
           freq = 1;                     // freq is one
         } else {
+          //System.out.println("  read freq");
           freq = postings.readVInt();     // else read freq
         }
         posPending = freq;
@@ -631,10 +629,12 @@ public class PulsingPostingsReaderImpl e
 
     @Override
     public int advance(int target) throws IOException {
+      //System.out.println("PR.advance target=" + target);
       int doc;
       while((doc=nextDoc()) != NO_MORE_DOCS) {
+        //System.out.println("  nextDoc got doc=" + doc);
         if (doc >= target) {
-          return doc;
+          return docID = doc;
         }
       }
       return docID = NO_MORE_DOCS;
@@ -642,7 +642,7 @@ public class PulsingPostingsReaderImpl e
 
     @Override
     public int nextPosition() throws IOException {
-      //System.out.println("PR d&p nextPosition posPending=" + posPending + " vs freq=" + freq);
+      //System.out.println("PR.nextPosition posPending=" + posPending + " vs freq=" + freq);
       
       assert posPending > 0;
       posPending--;
@@ -652,6 +652,7 @@ public class PulsingPostingsReaderImpl e
           //System.out.println("PR     skip payload=" + payloadLength);
           postings.skipBytes(payloadLength);
         }
+        //System.out.println("  read pos code");
         final int code = postings.readVInt();
         //System.out.println("PR     code=" + code);
         if ((code & 1) != 0) {
@@ -664,16 +665,17 @@ public class PulsingPostingsReaderImpl e
         position += postings.readVInt();
       }
 
-      //System.out.println("PR d&p nextPos return pos=" + position + " this=" + this);
+      //System.out.println("  return pos=" + position + " hasPayload=" + !payloadRetrieved + " posPending=" + posPending + " this=" + this);
       return position;
     }
 
     private void skipPositions() throws IOException {
+      //System.out.println("PR.skipPositions: posPending=" + posPending);
       while(posPending != 0) {
         nextPosition();
       }
       if (storePayloads && !payloadRetrieved) {
-        //System.out.println("  skip payload len=" + payloadLength);
+        //System.out.println("  skip last payload len=" + payloadLength);
         postings.skipBytes(payloadLength);
         payloadRetrieved = true;
       }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java Mon May  9 15:24:04 2011
@@ -201,6 +201,7 @@ public final class PulsingPostingsWriter
       if (!omitTF) {
         int lastDocID = 0;
         int pendingIDX = 0;
+        int lastPayloadLength = -1;
         while(pendingIDX < pendingCount) {
           final Position doc = pending[pendingIDX];
 
@@ -217,7 +218,6 @@ public final class PulsingPostingsWriter
           }
 
           int lastPos = 0;
-          int lastPayloadLength = -1;
           for(int posIDX=0;posIDX<doc.termFreq;posIDX++) {
             final Position pos = pending[pendingIDX++];
             assert pos.docID == doc.docID;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java Mon May  9 15:24:04 2011
@@ -152,14 +152,8 @@ public class SepPostingsReaderImpl exten
 
     @Override
     public Object clone() {
-      SepTermState other = (SepTermState) super.clone();
-      other.docIndex = (IntIndexInput.Index) docIndex.clone();
-      if (freqIndex != null) {
-        other.freqIndex = (IntIndexInput.Index) freqIndex.clone();
-      }
-      if (posIndex != null) {
-        other.posIndex = (IntIndexInput.Index) posIndex.clone();
-      }
+      SepTermState other = new SepTermState();
+      other.copyFrom(this);
       return other;
     }
 
@@ -167,12 +161,28 @@ public class SepPostingsReaderImpl exten
     public void copyFrom(TermState _other) {
       super.copyFrom(_other);
       SepTermState other = (SepTermState) _other;
-      docIndex.set(other.docIndex);
-      if (freqIndex != null && other.freqIndex != null) {
-        freqIndex.set(other.freqIndex);
+      if (docIndex == null) {
+        docIndex = (IntIndexInput.Index) other.docIndex.clone();
+      } else {
+        docIndex.set(other.docIndex);
+      }
+      if (other.freqIndex != null) {
+        if (freqIndex == null) {
+          freqIndex = (IntIndexInput.Index) other.freqIndex.clone();
+        } else {
+          freqIndex.set(other.freqIndex);
+        }
+      } else {
+        freqIndex = null;
       }
-      if (posIndex != null && other.posIndex != null) {
-        posIndex.set(other.posIndex);
+      if (other.posIndex != null) {
+        if (posIndex == null) {
+          posIndex = (IntIndexInput.Index) other.posIndex.clone();
+        } else {
+          posIndex.set(other.posIndex);
+        }
+      } else {
+        posIndex = null;
       }
       payloadFP = other.payloadFP;
       skipFP = other.skipFP;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java Mon May  9 15:24:04 2011
@@ -69,12 +69,13 @@ public final class SepPostingsWriterImpl
    * smaller indexes, greater acceleration, but fewer accelerable cases, while
    * smaller values result in bigger indexes, less acceleration and more
    * accelerable cases. More detailed experiments would be useful here. */
-  final int skipInterval = 16;
+  final int skipInterval;
+  static final int DEFAULT_SKIP_INTERVAL = 16;
   
   /**
    * Expert: minimum docFreq to write any skip data at all
    */
-  final int skipMinimum = skipInterval;
+  final int skipMinimum;
 
   /** Expert: The maximum number of skip levels. Smaller values result in 
    * slightly smaller indexes, but slower skipping in big posting lists.
@@ -102,8 +103,13 @@ public final class SepPostingsWriterImpl
   private final RAMOutputStream indexBytesWriter = new RAMOutputStream();
 
   public SepPostingsWriterImpl(SegmentWriteState state, IntStreamFactory factory) throws IOException {
-    super();
+    this(state, factory, DEFAULT_SKIP_INTERVAL);
+  }
 
+  public SepPostingsWriterImpl(SegmentWriteState state, IntStreamFactory factory, int skipInterval) throws IOException {
+    super();
+    this.skipInterval = skipInterval;
+    this.skipMinimum = skipInterval; /* set to the same for now */
     final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, DOC_EXTENSION);
     docOut = factory.createOutput(state.directory, docFileName);
     docIndex = docOut.index();

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java Mon May  9 15:24:04 2011
@@ -177,6 +177,7 @@ class SepSkipListReader extends MultiLev
   @Override
   protected int readSkipData(int level, IndexInput skipStream) throws IOException {
     int delta;
+    assert !omitTF || !currentFieldStoresPayloads;
     if (currentFieldStoresPayloads) {
       // the current field stores payloads.
       // if the doc delta is odd then we have

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java Mon May  9 15:24:04 2011
@@ -50,12 +50,13 @@ public final class StandardPostingsWrite
    * smaller indexes, greater acceleration, but fewer accelerable cases, while
    * smaller values result in bigger indexes, less acceleration and more
    * accelerable cases. More detailed experiments would be useful here. */
-  final int skipInterval = 16;
+  static final int DEFAULT_SKIP_INTERVAL = 16;
+  final int skipInterval;
   
   /**
    * Expert: minimum docFreq to write any skip data at all
    */
-  final int skipMinimum = skipInterval;
+  final int skipMinimum;
 
   /** Expert: The maximum number of skip levels. Smaller values result in 
    * slightly smaller indexes, but slower skipping in big posting lists.
@@ -82,7 +83,12 @@ public final class StandardPostingsWrite
   private RAMOutputStream bytesWriter = new RAMOutputStream();
 
   public StandardPostingsWriter(SegmentWriteState state) throws IOException {
+    this(state, DEFAULT_SKIP_INTERVAL);
+  }
+  public StandardPostingsWriter(SegmentWriteState state, int skipInterval) throws IOException {
     super();
+    this.skipInterval = skipInterval;
+    this.skipMinimum = skipInterval; /* set to the same for now */
     //this.segment = state.segmentName;
     String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.FREQ_EXTENSION);
     freqOut = state.directory.createOutput(fileName);

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/BooleanQuery.java Mon May  9 15:24:04 2011
@@ -367,8 +367,12 @@ public class BooleanQuery extends Query 
         Query query = c.getQuery().rewrite(reader);    // rewrite first
 
         if (getBoost() != 1.0f) {                 // incorporate boost
-          if (query == c.getQuery())                   // if rewrite was no-op
+          if (query == c.getQuery()) {                   // if rewrite was no-op
             query = (Query)query.clone();         // then clone before boost
+          }
+          // Since the BooleanQuery only has 1 clause, the BooleanQuery will be
+          // written out. Therefore the rewritten Query's boost must incorporate both
+          // the clause's boost, and the boost of the BooleanQuery itself
           query.setBoost(getBoost() * query.getBoost());
         }
 
@@ -381,8 +385,12 @@ public class BooleanQuery extends Query 
       BooleanClause c = clauses.get(i);
       Query query = c.getQuery().rewrite(reader);
       if (query != c.getQuery()) {                     // clause rewrote: must clone
-        if (clone == null)
+        if (clone == null) {
+          // The BooleanQuery clone is lazily initialized so only initialize
+          // it if a rewritten clause differs from the original clause (and hasn't been
+          // initialized already).  If nothing differs, the clone isn't needlessly created
           clone = (BooleanQuery)this.clone();
+        }
         clone.clauses.set(i, new BooleanClause(query, c.getOccur()));
       }
     }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/HitQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/HitQueue.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/HitQueue.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/HitQueue.java Mon May  9 15:24:04 2011
@@ -21,8 +21,6 @@ import org.apache.lucene.util.PriorityQu
 
 final class HitQueue extends PriorityQueue<ScoreDoc> {
 
-  private boolean prePopulate;
-
   /**
    * Creates a new instance with <code>size</code> elements. If
    * <code>prePopulate</code> is set to true, the queue will pre-populate itself

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Mon May  9 15:24:04 2011
@@ -46,8 +46,18 @@ import org.apache.lucene.util.ThreadInte
  *
  * <p>Applications usually need only call the inherited
  * {@link #search(Query,int)}
- * or {@link #search(Query,Filter,int)} methods. For performance reasons it is 
- * recommended to open only one IndexSearcher and use it for all of your searches.
+ * or {@link #search(Query,Filter,int)} methods. For
+ * performance reasons, if your index is unchanging, you
+ * should share a single IndexSearcher instance across
+ * multiple searches instead of creating a new one
+ * per-search.  If your index has changed and you wish to
+ * see the changes reflected in searching, you should
+ * use {@link IndexReader#reopen} to obtain a new reader and
+ * then create a new IndexSearcher from that.  Also, for
+ * low-latency turnaround it's best to use a near-real-time
+ * reader ({@link IndexReader#open(IndexWriter,boolean)}).
+ * Once you have a new {@link IndexReader}, it's relatively
+ * cheap to create a new IndexSearcher from it.
  * 
  * <a name="thread-safety"></a><p><b>NOTE</b>: <code>{@link
  * IndexSearcher}</code> instances are completely
@@ -857,4 +867,9 @@ public class IndexSearcher {
       this.leaves = leaves;
     }
   }
+
+  @Override
+  public String toString() {
+    return "IndexSearcher(" + reader + ")";
+  }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java Mon May  9 15:24:04 2011
@@ -214,12 +214,12 @@ public class MultiPhraseQuery extends Qu
           docFreq = reader.docFreq(term.field(), term.bytes());
         }
 
-        postingsFreqs[pos] = new PhraseQuery.PostingsAndFreq(postingsEnum, docFreq, positions.get(pos).intValue());
+        postingsFreqs[pos] = new PhraseQuery.PostingsAndFreq(postingsEnum, docFreq, positions.get(pos).intValue(), terms[0]);
       }
 
       // sort by increasing docFreq order
       if (slop == 0) {
-        ArrayUtil.quickSort(postingsFreqs);
+        ArrayUtil.mergeSort(postingsFreqs);
       }
 
       if (slop == 0) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhrasePositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhrasePositions.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhrasePositions.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhrasePositions.java Mon May  9 15:24:04 2011
@@ -28,13 +28,15 @@ final class PhrasePositions {
   int position;					  // position in doc
   int count;					  // remaining pos in this doc
   int offset;					  // position in phrase
+  final int ord;                                  // unique across all PhrasePositions instances
   final DocsAndPositionsEnum postings;  	  // stream of docs & positions
   PhrasePositions next;	                          // used to make lists
   boolean repeats;       // there's other pp for same term (e.g. query="1st word 2nd word"~1) 
 
-  PhrasePositions(DocsAndPositionsEnum postings, int o) {
+  PhrasePositions(DocsAndPositionsEnum postings, int o, int ord) {
     this.postings = postings;
     offset = o;
+    this.ord = ord;
   }
 
   final boolean next() throws IOException {	  // increments to next doc

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQuery.java Mon May  9 15:24:04 2011
@@ -124,16 +124,48 @@ public class PhraseQuery extends Query {
     final DocsAndPositionsEnum postings;
     final int docFreq;
     final int position;
+    final Term term;
 
-    public PostingsAndFreq(DocsAndPositionsEnum postings, int docFreq, int position) {
+    public PostingsAndFreq(DocsAndPositionsEnum postings, int docFreq, int position, Term term) {
       this.postings = postings;
       this.docFreq = docFreq;
       this.position = position;
+      this.term = term;
     }
 
     public int compareTo(PostingsAndFreq other) {
+      if (docFreq == other.docFreq) {
+        if (position == other.position) {
+          return term.compareTo(other.term);
+        }
+        return position - other.position;
+      }
       return docFreq - other.docFreq;
     }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + docFreq;
+      result = prime * result + position;
+      result = prime * result + ((term == null) ? 0 : term.hashCode());
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) return true;
+      if (obj == null) return false;
+      if (getClass() != obj.getClass()) return false;
+      PostingsAndFreq other = (PostingsAndFreq) obj;
+      if (docFreq != other.docFreq) return false;
+      if (position != other.position) return false;
+      if (term == null) {
+        if (other.term != null) return false;
+      } else if (!term.equals(other.term)) return false;
+      return true;
+    }
   }
 
   private class PhraseWeight extends Weight {
@@ -197,12 +229,12 @@ public class PhraseQuery extends Query {
             return null;
           }
         }
-        postingsFreqs[i] = new PostingsAndFreq(postingsEnum, reader.docFreq(t.field(), t.bytes()), positions.get(i).intValue());
+        postingsFreqs[i] = new PostingsAndFreq(postingsEnum, reader.docFreq(t.field(), t.bytes()), positions.get(i).intValue(), t);
       }
 
       // sort by increasing docFreq order
       if (slop == 0) {
-        ArrayUtil.quickSort(postingsFreqs);
+        ArrayUtil.mergeSort(postingsFreqs);
       }
 
       if (slop == 0) {				  // optimize exact case

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQueue.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQueue.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQueue.java Mon May  9 15:24:04 2011
@@ -30,10 +30,16 @@ final class PhraseQueue extends Priority
       if (pp1.position == pp2.position)
         // same doc and pp.position, so decide by actual term positions. 
         // rely on: pp.position == tp.position - offset. 
-        return pp1.offset < pp2.offset;
-      else
+        if (pp1.offset == pp2.offset) {
+          return pp1.ord < pp2.ord;
+        } else {
+          return pp1.offset < pp2.offset;
+        }
+      else {
         return pp1.position < pp2.position;
-    else
+      }
+    else {
       return pp1.doc < pp2.doc;
+    }
   }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseScorer.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseScorer.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseScorer.java Mon May  9 15:24:04 2011
@@ -55,7 +55,7 @@ abstract class PhraseScorer extends Scor
     // this allows to easily identify a matching (exact) phrase 
     // when all PhrasePositions have exactly the same position.
     for (int i = 0; i < postings.length; i++) {
-      PhrasePositions pp = new PhrasePositions(postings[i].postings, postings[i].position);
+      PhrasePositions pp = new PhrasePositions(postings[i].postings, postings[i].position, i);
       if (last != null) {			  // add next to end of list
         last.next = pp;
       } else {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java Mon May  9 15:24:04 2011
@@ -134,7 +134,7 @@ public abstract class TopTermsRewrite<Q 
     final Term placeholderTerm = new Term(query.field);
     final Q q = getTopLevelQuery();
     final ScoreTerm[] scoreTerms = stQueue.toArray(new ScoreTerm[stQueue.size()]);
-    ArrayUtil.quickSort(scoreTerms, scoreTermSortByTermComp);
+    ArrayUtil.mergeSort(scoreTerms, scoreTermSortByTermComp);
     for (final ScoreTerm st : scoreTerms) {
       final Term term = placeholderTerm.createTerm(st.bytes);
       assert reader.docFreq(term) == st.termState.docFreq() : "reader DF is " + reader.docFreq(term) + " vs " + st.termState.docFreq();

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java?rev=1101062&r1=1101061&r2=1101062&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java Mon May  9 15:24:04 2011
@@ -190,7 +190,7 @@ public class NearSpansOrdered extends Sp
 
   /** Advance the subSpans to the same document */
   private boolean toSameDoc() throws IOException {
-    ArrayUtil.quickSort(subSpansByDoc, spanDocComparator);
+    ArrayUtil.mergeSort(subSpansByDoc, spanDocComparator);
     int firstIndex = 0;
     int maxDoc = subSpansByDoc[subSpansByDoc.length - 1].doc();
     while (subSpansByDoc[firstIndex].doc() != maxDoc) {

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

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

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