You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2011/08/20 21:20:48 UTC

svn commit: r1159905 [4/6] - in /lucene/dev/trunk: lucene/contrib/sandbox/src/java/org/apache/lucene/sandbox/queries/ lucene/src/java/org/apache/lucene/index/ lucene/src/java/org/apache/lucene/index/codecs/ lucene/src/java/org/apache/lucene/index/codec...

Copied: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java (from r1159679, lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java?p2=lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java&p1=lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java&r1=1159679&r2=1159905&rev=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java Sat Aug 20 19:20:45 2011
@@ -18,6 +18,8 @@ package org.apache.lucene.index.codecs.s
  */
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.index.CorruptIndexException;
@@ -38,8 +40,8 @@ import org.apache.lucene.util.IOUtils;
  *  to .pyl, skip data to .skp
  *
  * @lucene.experimental */
-public final class SepPostingsWriterImpl extends PostingsWriterBase {
-  final static String CODEC = "SepDocFreqSkip";
+public final class SepPostingsWriter extends PostingsWriterBase {
+  final static String CODEC = "SepPostingsWriter";
 
   final static String DOC_EXTENSION = "doc";
   final static String SKIP_EXTENSION = "skp";
@@ -89,26 +91,22 @@ public final class SepPostingsWriterImpl
   boolean storePayloads;
   IndexOptions indexOptions;
 
-  long lastSkipFP;
-
   FieldInfo fieldInfo;
 
   int lastPayloadLength;
   int lastPosition;
   long payloadStart;
-  long lastPayloadStart;
   int lastDocID;
   int df;
-  private int pendingTermCount;
 
   // Holds pending byte[] blob for the current terms block
   private final RAMOutputStream indexBytesWriter = new RAMOutputStream();
 
-  public SepPostingsWriterImpl(SegmentWriteState state, IntStreamFactory factory) throws IOException {
+  public SepPostingsWriter(SegmentWriteState state, IntStreamFactory factory) throws IOException {
     this(state, factory, DEFAULT_SKIP_INTERVAL);
   }
 
-  public SepPostingsWriterImpl(SegmentWriteState state, IntStreamFactory factory, int skipInterval) throws IOException {
+  public SepPostingsWriter(SegmentWriteState state, IntStreamFactory factory, int skipInterval) throws IOException {
     freqOut = null;
     freqIndex = null;
     posOut = null;
@@ -171,6 +169,7 @@ public final class SepPostingsWriterImpl
   @Override
   public void startTerm() throws IOException {
     docIndex.mark();
+    //System.out.println("SEPW: startTerm docIndex=" + docIndex);
 
     if (indexOptions != IndexOptions.DOCS_ONLY) {
       freqIndex.mark();
@@ -201,7 +200,7 @@ public final class SepPostingsWriterImpl
   public void startDoc(int docID, int termDocFreq) throws IOException {
 
     final int delta = docID - lastDocID;
-    //System.out.println("SepW startDoc: write doc=" + docID + " delta=" + delta);
+    //System.out.println("SEPW: startDoc: write doc=" + docID + " delta=" + delta + " out.fp=" + docOut);
 
     if (docID < 0 || (df > 0 && delta <= 0)) {
       throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )");
@@ -223,15 +222,6 @@ public final class SepPostingsWriterImpl
     }
   }
 
-  @Override
-  public void flushTermsBlock() throws IOException {
-    //System.out.println("SepW.flushTermsBlock: pendingTermCount=" + pendingTermCount + " bytesUsed=" + indexBytesWriter.getFilePointer());
-    termsOut.writeVLong((int) indexBytesWriter.getFilePointer());
-    indexBytesWriter.writeTo(termsOut);
-    indexBytesWriter.reset();
-    pendingTermCount = 0;
-  }
-
   /** Add a new position & payload */
   @Override
   public void addPosition(int position, BytesRef payload) throws IOException {
@@ -269,6 +259,24 @@ public final class SepPostingsWriterImpl
     lastPosition = 0;
   }
 
+  private static class PendingTerm {
+    public final IntIndexOutput.Index docIndex;
+    public final IntIndexOutput.Index freqIndex;
+    public final IntIndexOutput.Index posIndex;
+    public final long payloadFP;
+    public final long skipFP;
+
+    public PendingTerm(IntIndexOutput.Index docIndex, IntIndexOutput.Index freqIndex, IntIndexOutput.Index posIndex, long payloadFP, long skipFP) {
+      this.docIndex = docIndex;
+      this.freqIndex = freqIndex;
+      this.posIndex = posIndex;
+      this.payloadFP = payloadFP;
+      this.skipFP = skipFP;
+    }
+  }
+
+  private final List<PendingTerm> pendingTerms = new ArrayList<PendingTerm>();
+
   /** Called when we are done adding docs to this term */
   @Override
   public void finishTerm(TermStats stats) throws IOException {
@@ -276,50 +284,107 @@ public final class SepPostingsWriterImpl
     assert stats.docFreq > 0;
     assert stats.docFreq == df;
 
-    final boolean isFirstTerm = pendingTermCount == 0;  
-    //System.out.println("SepW.finishTerm: isFirstTerm=" + isFirstTerm);
-
-    docIndex.write(indexBytesWriter, isFirstTerm);
-    //System.out.println("  docIndex=" + docIndex);
+    final IntIndexOutput.Index docIndexCopy = docOut.index();
+    docIndexCopy.copyFrom(docIndex, false);
 
+    final IntIndexOutput.Index freqIndexCopy;
+    final IntIndexOutput.Index posIndexCopy;
     if (indexOptions != IndexOptions.DOCS_ONLY) {
-      freqIndex.write(indexBytesWriter, isFirstTerm);
-      //System.out.println("  freqIndex=" + freqIndex);
-    }
-
-    if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
-      posIndex.write(indexBytesWriter, isFirstTerm);
-      //System.out.println("  posIndex=" + posIndex);
-      if (storePayloads) {
-        if (isFirstTerm) {
-          indexBytesWriter.writeVLong(payloadStart);
-        } else {
-          indexBytesWriter.writeVLong(payloadStart - lastPayloadStart);
-        }
-        lastPayloadStart = payloadStart;
-        //System.out.println("  payloadFP=" + payloadStart);
+      freqIndexCopy = freqOut.index();
+      freqIndexCopy.copyFrom(freqIndex, false);
+      if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+        posIndexCopy = posOut.index();
+        posIndexCopy.copyFrom(posIndex, false);
+      } else {
+        posIndexCopy = null;
       }
+    } else {
+      freqIndexCopy = null;
+      posIndexCopy = null;
     }
 
+    final long skipFP;
     if (df >= skipMinimum) {
-      //System.out.println("  skipFP=" + skipStart);
-      final long skipFP = skipOut.getFilePointer();
+      skipFP = skipOut.getFilePointer();
+      //System.out.println("  skipFP=" + skipFP);
       skipListWriter.writeSkip(skipOut);
-      //System.out.println("   writeSkip @ " + indexBytesWriter.getFilePointer());
-      if (isFirstTerm) {
-        indexBytesWriter.writeVLong(skipFP);
-      } else {
-        indexBytesWriter.writeVLong(skipFP - lastSkipFP);
-      }
-      lastSkipFP = skipFP;
-    } else if (isFirstTerm) {
-      // lazily write an absolute delta if a term in this block requires skip data.
-      lastSkipFP = 0;
+      //System.out.println("    numBytes=" + (skipOut.getFilePointer()-skipFP));
+    } else {
+      skipFP = -1;
     }
 
     lastDocID = 0;
     df = 0;
-    pendingTermCount++;
+
+    pendingTerms.add(new PendingTerm(docIndexCopy,
+                                     freqIndexCopy,
+                                     posIndexCopy,
+                                     payloadStart,
+                                     skipFP));
+  }
+
+  @Override
+  public void flushTermsBlock(int start, int count) throws IOException {
+    //System.out.println("SEPW: flushTermsBlock: start=" + start + " count=" + count + " pendingTerms.size()=" + pendingTerms.size() + " termsOut.fp=" + termsOut.getFilePointer());
+    assert indexBytesWriter.getFilePointer() == 0;
+    final int absStart = pendingTerms.size() - start;
+    final List<PendingTerm> slice = pendingTerms.subList(absStart, absStart+count);
+
+    long lastPayloadFP = 0;
+    long lastSkipFP = 0;
+
+    if (count == 0) {
+      termsOut.writeByte((byte) 0);
+      return;
+    }
+
+    final PendingTerm firstTerm = slice.get(0);
+    final IntIndexOutput.Index docIndexFlush = firstTerm.docIndex;
+    final IntIndexOutput.Index freqIndexFlush = firstTerm.freqIndex;
+    final IntIndexOutput.Index posIndexFlush = firstTerm.posIndex;
+
+    for(int idx=0;idx<slice.size();idx++) {
+      final boolean isFirstTerm = idx == 0;
+      final PendingTerm t = slice.get(idx);
+      //System.out.println("  write idx=" + idx + " docIndex=" + t.docIndex);
+      docIndexFlush.copyFrom(t.docIndex, false);
+      docIndexFlush.write(indexBytesWriter, isFirstTerm);
+      if (indexOptions != IndexOptions.DOCS_ONLY) {
+        freqIndexFlush.copyFrom(t.freqIndex, false);
+        freqIndexFlush.write(indexBytesWriter, isFirstTerm);
+        //System.out.println("    freqIndex=" + t.freqIndex);
+        if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+          posIndexFlush.copyFrom(t.posIndex, false);
+          posIndexFlush.write(indexBytesWriter, isFirstTerm);
+          //System.out.println("    posIndex=" + t.posIndex);
+          if (storePayloads) {
+            //System.out.println("    payloadFP=" + t.payloadFP);
+            if (isFirstTerm) {
+              indexBytesWriter.writeVLong(t.payloadFP);
+            } else {
+              indexBytesWriter.writeVLong(t.payloadFP - lastPayloadFP);
+            }
+            lastPayloadFP = t.payloadFP;
+          }
+        }
+      }
+
+      if (t.skipFP != -1) {
+        if (isFirstTerm) {
+          indexBytesWriter.writeVLong(t.skipFP);
+        } else {
+          indexBytesWriter.writeVLong(t.skipFP - lastSkipFP);
+        }
+        lastSkipFP = t.skipFP;
+        //System.out.println("    skipFP=" + t.skipFP);
+      }
+    }
+
+    //System.out.println("  numBytes=" + indexBytesWriter.getFilePointer());
+    termsOut.writeVLong((int) indexBytesWriter.getFilePointer());
+    indexBytesWriter.writeTo(termsOut);
+    indexBytesWriter.reset();
+    slice.clear();
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListWriter.java Sat Aug 20 19:20:45 2011
@@ -124,12 +124,12 @@ class SepSkipListWriter extends MultiLev
     Arrays.fill(lastSkipDoc, 0);
     Arrays.fill(lastSkipPayloadLength, -1);  // we don't have to write the first length in the skip list
     for(int i=0;i<numberOfSkipLevels;i++) {
-      docIndex[i].set(topDocIndex);
+      docIndex[i].copyFrom(topDocIndex, true);
       if (freqOutput != null) {
-        freqIndex[i].set(topFreqIndex);
+        freqIndex[i].copyFrom(topFreqIndex, true);
       }
       if (posOutput != null) {
-        posIndex[i].set(topPosIndex);
+        posIndex[i].copyFrom(topPosIndex, true);
       }
     }
     if (payloadOutput != null) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java Sat Aug 20 19:20:45 2011
@@ -22,31 +22,37 @@ import java.util.Set;
 
 import org.apache.lucene.index.PerDocWriteState;
 import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.codecs.PostingsReaderBase;
+import org.apache.lucene.index.codecs.PostingsWriterBase;
+import org.apache.lucene.index.codecs.BlockTreeTermsReader;
+import org.apache.lucene.index.codecs.BlockTreeTermsWriter;
 import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
+import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.PerDocConsumer;
-import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
 import org.apache.lucene.index.codecs.PerDocValues;
-import org.apache.lucene.index.codecs.PostingsWriterBase;
-import org.apache.lucene.index.codecs.PostingsReaderBase;
-import org.apache.lucene.index.codecs.TermsIndexWriterBase;
-import org.apache.lucene.index.codecs.TermsIndexReaderBase;
-import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
-import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
-import org.apache.lucene.index.codecs.BlockTermsWriter;
-import org.apache.lucene.index.codecs.BlockTermsReader;
-import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.store.Directory;
 
 /** Default codec. 
  *  @lucene.experimental */
 public class StandardCodec extends Codec {
 
+  private final int minBlockSize;
+  private final int maxBlockSize;
+
   public StandardCodec() {
+    this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
+  }
+
+  public StandardCodec(int minBlockSize, int maxBlockSize) {
     super("Standard");
+    this.minBlockSize = minBlockSize;
+    assert minBlockSize > 1;
+    this.maxBlockSize = maxBlockSize;
   }
 
   @Override
@@ -57,29 +63,14 @@ public class StandardCodec extends Codec
     // pluggable?  Ie so that this codec would record which
     // index impl was used, and switch on loading?
     // Or... you must make a new Codec for this?
-    TermsIndexWriterBase indexWriter;
     boolean success = false;
     try {
-      indexWriter = new VariableGapTermsIndexWriter(state, new VariableGapTermsIndexWriter.EveryNTermSelector(state.termIndexInterval));
-      success = true;
-    } finally {
-      if (!success) {
-        docs.close();
-      }
-    }
-
-    success = false;
-    try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs);
+      FieldsConsumer ret = new BlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
       success = true;
       return ret;
     } finally {
       if (!success) {
-        try {
-          docs.close();
-        } finally {
-          indexWriter.close();
-        }
+        docs.close();
       }
     }
   }
@@ -89,41 +80,22 @@ public class StandardCodec extends Codec
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
     PostingsReaderBase postings = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
-    TermsIndexReaderBase indexReader;
 
     boolean success = false;
     try {
-      indexReader = new VariableGapTermsIndexReader(state.dir,
+      FieldsProducer ret = new BlockTreeTermsReader(
+                                                    state.dir,
                                                     state.fieldInfos,
                                                     state.segmentInfo.name,
-                                                    state.termsIndexDivisor,
-                                                    state.codecId, state.context);
-      success = true;
-    } finally {
-      if (!success) {
-        postings.close();
-      }
-    }
-
-    success = false;
-    try {
-      FieldsProducer ret = new BlockTermsReader(indexReader,
-                                                state.dir,
-                                                state.fieldInfos,
-                                                state.segmentInfo.name,
-                                                postings,
-                                                state.context,
-                                                TERMS_CACHE_SIZE,
-                                                state.codecId);
+                                                    postings,
+                                                    state.context,
+                                                    state.codecId,
+                                                    state.termsIndexDivisor);
       success = true;
       return ret;
     } finally {
       if (!success) {
-        try {
-          postings.close();
-        } finally {
-          indexReader.close();
-        }
+        postings.close();
       }
     }
   }
@@ -135,11 +107,10 @@ public class StandardCodec extends Codec
   static final String PROX_EXTENSION = "prx";
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
-    StandardPostingsReader.files(dir, segmentInfo, id, files);
-    BlockTermsReader.files(dir, segmentInfo, id, files);
-    VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
-    DefaultDocValuesConsumer.files(dir, segmentInfo, id, files, getDocValuesUseCFS());
+  public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set<String> files) throws IOException {
+    StandardPostingsReader.files(dir, segmentInfo, codecID, files);
+    BlockTreeTermsReader.files(dir, segmentInfo, codecID, files);
+    DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files, getDocValuesUseCFS());
   }
 
   @Override
@@ -151,8 +122,12 @@ public class StandardCodec extends Codec
   public static void getStandardExtensions(Set<String> extensions) {
     extensions.add(FREQ_EXTENSION);
     extensions.add(PROX_EXTENSION);
-    BlockTermsReader.getExtensions(extensions);
-    VariableGapTermsIndexReader.getIndexExtensions(extensions);
+    BlockTreeTermsReader.getExtensions(extensions);
+  }
+
+  @Override
+  public String toString() {
+    return name + "(minBlockSize=" + minBlockSize + " maxBlockSize=" + maxBlockSize + ")";
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java Sat Aug 20 19:20:45 2011
@@ -27,8 +27,8 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.TermState;
-import org.apache.lucene.index.codecs.BlockTermState;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
+import org.apache.lucene.index.codecs.BlockTermState;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -46,22 +46,23 @@ public class StandardPostingsReader exte
 
   private final IndexInput freqIn;
   private final IndexInput proxIn;
+  // public static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
 
   int skipInterval;
   int maxSkipLevels;
   int skipMinimum;
 
-  //private String segment;
+  // private String segment;
 
-  public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext context, int codecId) throws IOException {
+  public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext ioContext, int codecId) throws IOException {
     freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.FREQ_EXTENSION),
-                           context);
-    //this.segment = segmentInfo.name;
+                           ioContext);
+    // this.segment = segmentInfo.name;
     if (segmentInfo.getHasProx()) {
       boolean success = false;
       try {
         proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.PROX_EXTENSION),
-                               context);
+                               ioContext);
         success = true;
       } finally {
         if (!success) {
@@ -73,10 +74,10 @@ public class StandardPostingsReader exte
     }
   }
 
-  public static void files(Directory dir, SegmentInfo segmentInfo, int id, Collection<String> files) throws IOException {
-    files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, StandardCodec.FREQ_EXTENSION));
+  public static void files(Directory dir, SegmentInfo segmentInfo, int codecID, Collection<String> files) throws IOException {
+    files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardCodec.FREQ_EXTENSION));
     if (segmentInfo.getHasProx()) {
-      files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, StandardCodec.PROX_EXTENSION));
+      files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardCodec.PROX_EXTENSION));
     }
   }
 
@@ -100,7 +101,7 @@ public class StandardPostingsReader exte
 
     // Only used by the "primary" TermState -- clones don't
     // copy this (basically they are "transient"):
-    ByteArrayDataInput bytesReader;
+    ByteArrayDataInput bytesReader;  // TODO: should this NOT be in the TermState...?
     byte[] bytes;
 
     @Override
@@ -155,7 +156,8 @@ public class StandardPostingsReader exte
     final StandardTermState termState = (StandardTermState) _termState;
 
     final int len = termsIn.readVInt();
-    //System.out.println("SPR.readTermsBlock termsIn.fp=" + termsIn.getFilePointer());
+
+    // if (DEBUG) System.out.println("  SPR.readTermsBlock bytes=" + len + " ts=" + _termState);
     if (termState.bytes == null) {
       termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
       termState.bytesReader = new ByteArrayDataInput();
@@ -171,21 +173,25 @@ public class StandardPostingsReader exte
   public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState)
     throws IOException {
     final StandardTermState termState = (StandardTermState) _termState;
-    //System.out.println("StandardR.nextTerm seg=" + segment);
-    final boolean isFirstTerm = termState.termCount == 0;
+    // if (DEBUG) System.out.println("SPR: nextTerm seg=" + segment + " tbOrd=" + termState.termBlockOrd + " bytesReader.fp=" + termState.bytesReader.getPosition());
+    final boolean isFirstTerm = termState.termBlockOrd == 0;
 
     if (isFirstTerm) {
       termState.freqOffset = termState.bytesReader.readVLong();
     } else {
       termState.freqOffset += termState.bytesReader.readVLong();
     }
-    //System.out.println("  dF=" + termState.docFreq);
-    //System.out.println("  freqFP=" + termState.freqOffset);
+    /*
+    if (DEBUG) {
+      System.out.println("  dF=" + termState.docFreq);
+      System.out.println("  freqFP=" + termState.freqOffset);
+    }
+    */
     assert termState.freqOffset < freqIn.length();
 
     if (termState.docFreq >= skipMinimum) {
       termState.skipOffset = termState.bytesReader.readVInt();
-      //System.out.println("  skipOffset=" + termState.skipOffset + " vs freqIn.length=" + freqIn.length());
+      // if (DEBUG) System.out.println("  skipOffset=" + termState.skipOffset + " vs freqIn.length=" + freqIn.length());
       assert termState.freqOffset + termState.skipOffset < freqIn.length();
     } else {
       // undefined
@@ -197,7 +203,7 @@ public class StandardPostingsReader exte
       } else {
         termState.proxOffset += termState.bytesReader.readVLong();
       }
-      //System.out.println("  proxFP=" + termState.proxOffset);
+      // if (DEBUG) System.out.println("  proxFP=" + termState.proxOffset);
     }
   }
     
@@ -215,6 +221,7 @@ public class StandardPostingsReader exte
         docsEnum = new SegmentDocsEnum(freqIn);
       }
     }
+    // if (DEBUG) System.out.println("SPR.docs ts=" + termState);
     return docsEnum.reset(fieldInfo, (StandardTermState) termState, liveDocs);
   }
 
@@ -300,7 +307,7 @@ public class StandardPostingsReader exte
       assert limit > 0;
       ord = 0;
       doc = 0;
-      //System.out.println("  sde limit=" + limit + " freqFP=" + freqOffset);
+      // if (DEBUG) System.out.println("  sde limit=" + limit + " freqFP=" + freqOffset);
 
       skipped = false;
 
@@ -309,8 +316,10 @@ public class StandardPostingsReader exte
 
     @Override
     public int nextDoc() throws IOException {
+      //if (DEBUG) System.out.println("    stpr.nextDoc seg=" + segment + " fp=" + freqIn.getFilePointer());
       while(true) {
         if (ord == limit) {
+          //if (DEBUG) System.out.println("      return doc=" + NO_MORE_DOCS);
           return doc = NO_MORE_DOCS;
         }
 
@@ -318,6 +327,7 @@ public class StandardPostingsReader exte
 
         // Decode next doc/freq pair
         final int code = freqIn.readVInt();
+        // if (DEBUG) System.out.println("      code=" + code);
         if (omitTF) {
           doc += code;
         } else {
@@ -334,6 +344,7 @@ public class StandardPostingsReader exte
         }
       }
 
+      //if (DEBUG) System.out.println("    stpr.nextDoc return doc=" + doc);
       return doc;
     }
 
@@ -480,16 +491,17 @@ public class StandardPostingsReader exte
       freqOffset = termState.freqOffset;
       proxOffset = termState.proxOffset;
       skipOffset = termState.skipOffset;
-      //System.out.println("StandardR.D&PE reset seg=" + segment + " limit=" + limit + " freqFP=" + freqOffset + " proxFP=" + proxOffset);
+      // if (DEBUG) System.out.println("StandardR.D&PE reset seg=" + segment + " limit=" + limit + " freqFP=" + freqOffset + " proxFP=" + proxOffset);
 
       return this;
     }
 
     @Override
     public int nextDoc() throws IOException {
+      // if (DEBUG) System.out.println("SPR.nextDoc seg=" + segment + " freqIn.fp=" + freqIn.getFilePointer());
       while(true) {
         if (ord == limit) {
-          //System.out.println("StandardR.D&PE seg=" + segment + " nextDoc return doc=END");
+          // if (DEBUG) System.out.println("  return END");
           return doc = NO_MORE_DOCS;
         }
 
@@ -513,7 +525,7 @@ public class StandardPostingsReader exte
 
       position = 0;
 
-      //System.out.println("StandardR.D&PE nextDoc seg=" + segment + " return doc=" + doc);
+      // if (DEBUG) System.out.println("  return doc=" + doc);
       return doc;
     }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java Sat Aug 20 19:20:45 2011
@@ -21,6 +21,8 @@ package org.apache.lucene.index.codecs.s
  *  index file format */
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocsEnum;
@@ -34,18 +36,19 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
-import org.apache.lucene.util.IOUtils;
 
 /** @lucene.experimental */
 public final class StandardPostingsWriter extends PostingsWriterBase {
-  final static String CODEC = "StandardPostingsWriterImpl";
+  final static String CODEC = "StandardPostingsWriter";
+
+  //private static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
   
   // Increment version to change it:
   final static int VERSION_START = 0;
   final static int VERSION_CURRENT = VERSION_START;
 
-  IndexOutput freqOut;
-  IndexOutput proxOut;
+  final IndexOutput freqOut;
+  final IndexOutput proxOut;
   final DefaultSkipListWriter skipListWriter;
   /** Expert: The fraction of TermDocs entries stored in skip tables,
    * used to accelerate {@link DocsEnum#advance(int)}.  Larger values result in
@@ -70,52 +73,42 @@ public final class StandardPostingsWrite
   IndexOptions indexOptions;
   boolean storePayloads;
   // Starts a new term
-  long lastFreqStart;
   long freqStart;
-  long lastProxStart;
   long proxStart;
   FieldInfo fieldInfo;
   int lastPayloadLength;
   int lastPosition;
 
-  private int pendingCount;
-
-  //private String segment;
-
-  private RAMOutputStream bytesWriter = new RAMOutputStream();
+  // private String segment;
 
   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;
+    // this.segment = state.segmentName;
     String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.FREQ_EXTENSION);
     freqOut = state.directory.createOutput(fileName, state.context);
-    boolean success = false;
-    try {
-      if (state.fieldInfos.hasProx()) {
-        // At least one field does not omit TF, so create the
-        // prox file
-        fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION);
-        proxOut = state.directory.createOutput(fileName, state.context);
-      } else {
-        // Every field omits TF so we will write no prox file
-        proxOut = null;
-      }
-      
-      totalNumDocs = state.numDocs;
-      
-      skipListWriter = new DefaultSkipListWriter(skipInterval, maxSkipLevels,
-          state.numDocs, freqOut, proxOut);
-      success = true;
-    } finally {
-      if (!success) {
-        IOUtils.closeSafely(true, freqOut, proxOut);
-      }
+    if (state.fieldInfos.hasProx()) {
+      // At least one field does not omit TF, so create the
+      // prox file
+      fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION);
+      proxOut = state.directory.createOutput(fileName, state.context);
+    } else {
+      // Every field omits TF so we will write no prox file
+      proxOut = null;
     }
+
+    totalNumDocs = state.numDocs;
+
+    skipListWriter = new DefaultSkipListWriter(skipInterval,
+                                               maxSkipLevels,
+                                               state.numDocs,
+                                               freqOut,
+                                               proxOut);
   }
 
   @Override
@@ -129,8 +122,8 @@ public final class StandardPostingsWrite
 
   @Override
   public void startTerm() {
-    //System.out.println("StandardW: startTerm seg=" + segment + " pendingCount=" + pendingCount);
     freqStart = freqOut.getFilePointer();
+    //if (DEBUG) System.out.println("SPW: startTerm freqOut.fp=" + freqStart);
     if (proxOut != null) {
       proxStart = proxOut.getFilePointer();
       // force first payload to write its length
@@ -144,6 +137,13 @@ public final class StandardPostingsWrite
   @Override
   public void setField(FieldInfo fieldInfo) {
     //System.out.println("SPW: setField");
+    /*
+    if (BlockTreeTermsWriter.DEBUG && fieldInfo.name.equals("id")) {
+      DEBUG = true;
+    } else {
+      DEBUG = false;
+    }
+    */
     this.fieldInfo = fieldInfo;
     indexOptions = fieldInfo.indexOptions;
     storePayloads = fieldInfo.storePayloads;
@@ -158,7 +158,7 @@ public final class StandardPostingsWrite
    *  then we just skip consuming positions/payloads. */
   @Override
   public void startDoc(int docID, int termDocFreq) throws IOException {
-    //System.out.println("StandardW:   startDoc seg=" + segment + " docID=" + docID + " tf=" + termDocFreq);
+    // if (DEBUG) System.out.println("SPW:   startDoc seg=" + segment + " docID=" + docID + " tf=" + termDocFreq + " freqOut.fp=" + freqOut.getFilePointer());
 
     final int delta = docID - lastDocID;
     
@@ -189,13 +189,13 @@ public final class StandardPostingsWrite
   /** Add a new position & payload */
   @Override
   public void addPosition(int position, BytesRef payload) throws IOException {
-    //System.out.println("StandardW:     addPos pos=" + position + " payload=" + (payload == null ? "null" : (payload.length + " bytes")) + " proxFP=" + proxOut.getFilePointer());
+    //if (DEBUG) System.out.println("SPW:     addPos pos=" + position + " payload=" + (payload == null ? "null" : (payload.length + " bytes")) + " proxFP=" + proxOut.getFilePointer());
     assert indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS: "invalid indexOptions: " + indexOptions;
     assert proxOut != null;
 
     final int delta = position - lastPosition;
-
-    assert delta >= 0: "position=" + position + " lastPosition=" + lastPosition;
+    
+    assert delta >= 0: "position=" + position + " lastPosition=" + lastPosition;            // not quite right (if pos=0 is repeated twice we don't catch it)
 
     lastPosition = position;
 
@@ -222,57 +222,104 @@ public final class StandardPostingsWrite
   public void finishDoc() {
   }
 
+  private static class PendingTerm {
+    public final long freqStart;
+    public final long proxStart;
+    public final int skipOffset;
+
+    public PendingTerm(long freqStart, long proxStart, int skipOffset) {
+      this.freqStart = freqStart;
+      this.proxStart = proxStart;
+      this.skipOffset = skipOffset;
+    }
+  }
+
+  private final List<PendingTerm> pendingTerms = new ArrayList<PendingTerm>();
+
   /** Called when we are done adding docs to this term */
   @Override
   public void finishTerm(TermStats stats) throws IOException {
-    //System.out.println("StandardW.finishTerm seg=" + segment);
+
+    // if (DEBUG) System.out.println("SPW: finishTerm seg=" + segment + " freqStart=" + freqStart);
     assert stats.docFreq > 0;
 
     // TODO: wasteful we are counting this (counting # docs
     // for this term) in two places?
     assert stats.docFreq == df;
 
-    final boolean isFirstTerm = pendingCount == 0;
-    //System.out.println("  isFirstTerm=" + isFirstTerm);
-
-    //System.out.println("  freqFP=" + freqStart);
-    if (isFirstTerm) {
-      bytesWriter.writeVLong(freqStart);
+    final int skipOffset;
+    if (df >= skipMinimum) {
+      skipOffset = (int) (skipListWriter.writeSkip(freqOut)-freqStart);
     } else {
-      bytesWriter.writeVLong(freqStart-lastFreqStart);
+      skipOffset = -1;
     }
-    lastFreqStart = freqStart;
 
-    if (df >= skipMinimum) {
-      bytesWriter.writeVInt((int) (skipListWriter.writeSkip(freqOut)-freqStart));
-    }
+    pendingTerms.add(new PendingTerm(freqStart, proxStart, skipOffset));
 
-    if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
-      //System.out.println("  proxFP=" + proxStart);
-      if (isFirstTerm) {
-        bytesWriter.writeVLong(proxStart);
-      } else {
-        bytesWriter.writeVLong(proxStart - lastProxStart);
-      }
-      lastProxStart = proxStart;
-    }
-     
     lastDocID = 0;
     df = 0;
-    pendingCount++;
   }
 
+  private final RAMOutputStream bytesWriter = new RAMOutputStream();
+
   @Override
-  public void flushTermsBlock() throws IOException {
-    //System.out.println("SPW.flushBlock pendingCount=" + pendingCount);
+  public void flushTermsBlock(int start, int count) throws IOException {
+    //if (DEBUG) System.out.println("SPW: flushTermsBlock start=" + start + " count=" + count + " left=" + (pendingTerms.size()-count) + " pendingTerms.size()=" + pendingTerms.size());
+
+    if (count == 0) {
+      termsOut.writeByte((byte) 0);
+      return;
+    }
+
+    assert start <= pendingTerms.size();
+    assert count <= start;
+
+    final int limit = pendingTerms.size() - start + count;
+    final PendingTerm firstTerm = pendingTerms.get(limit - count);
+    // First term in block is abs coded:
+    bytesWriter.writeVLong(firstTerm.freqStart);
+
+    if (firstTerm.skipOffset != -1) {
+      assert firstTerm.skipOffset > 0;
+      bytesWriter.writeVInt(firstTerm.skipOffset);
+    }
+    if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+      bytesWriter.writeVLong(firstTerm.proxStart);
+    }
+    long lastFreqStart = firstTerm.freqStart;
+    long lastProxStart = firstTerm.proxStart;
+    for(int idx=limit-count+1; idx<limit; idx++) {
+      final PendingTerm term = pendingTerms.get(idx);
+      //if (DEBUG) System.out.println("  write term freqStart=" + term.freqStart);
+      // The rest of the terms term are delta coded:
+      bytesWriter.writeVLong(term.freqStart - lastFreqStart);
+      lastFreqStart = term.freqStart;
+      if (term.skipOffset != -1) {
+        assert term.skipOffset > 0;
+        bytesWriter.writeVInt(term.skipOffset);
+      }
+      if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+        bytesWriter.writeVLong(term.proxStart - lastProxStart);
+        lastProxStart = term.proxStart;
+      }
+    }
+
     termsOut.writeVInt((int) bytesWriter.getFilePointer());
     bytesWriter.writeTo(termsOut);
     bytesWriter.reset();
-    pendingCount = 0;
+
+    // Remove the terms we just wrote:
+    pendingTerms.subList(limit-count, limit).clear();
   }
 
   @Override
   public void close() throws IOException {
-    IOUtils.closeSafely(false, freqOut, proxOut);
+    try {
+      freqOut.close();
+    } finally {
+      if (proxOut != null) {
+        proxOut.close();
+      }
+    }
   }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java Sat Aug 20 19:20:45 2011
@@ -22,14 +22,11 @@ import java.io.IOException;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.AutomatonTermsEnum.CompiledAutomaton;
-import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.automaton.Automaton;
-import org.apache.lucene.util.automaton.BasicAutomata;
 import org.apache.lucene.util.automaton.BasicOperations;
-import org.apache.lucene.util.automaton.MinimizationOperations;
-import org.apache.lucene.util.automaton.SpecialOperations;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 /**
  * A {@link Query} that will match terms against a finite-state machine.
@@ -41,7 +38,7 @@ import org.apache.lucene.util.automaton.
  * the standard Lucene wildcard syntax with {@link WildcardQuery}.
  * </p>
  * <p>
- * When the query is executed, it will create an equivalent minimal DFA of the
+ * When the query is executed, it will create an equivalent DFA of the
  * finite-state machine, and will enumerate the term dictionary in an
  * intelligent way to reduce the number of comparisons. For example: the regular
  * expression of <code>[dl]og?</code> will make approximately four comparisons:
@@ -52,20 +49,10 @@ import org.apache.lucene.util.automaton.
 public class AutomatonQuery extends MultiTermQuery {
   /** the automaton to match index terms against */
   protected final Automaton automaton;
+  protected final CompiledAutomaton compiled;
   /** term containing the field, and possibly some pattern structure */
   protected final Term term;
 
-  /** 
-   * abstraction for returning a termsenum:
-   * in the ctor the query computes one of these, the actual
-   * implementation depends upon the automaton's structure.
-   */
-  private abstract class TermsEnumFactory {
-    protected abstract TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException;
-  }
-  
-  private final TermsEnumFactory factory;
-
   /**
    * Create a new AutomatonQuery from an {@link Automaton}.
    * 
@@ -78,73 +65,12 @@ public class AutomatonQuery extends Mult
     super(term.field());
     this.term = term;
     this.automaton = automaton;
-    MinimizationOperations.minimize(automaton);
-    
-    if (BasicOperations.isEmpty(automaton)) {
-      // matches nothing
-      factory = new TermsEnumFactory() {
-        @Override
-        protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-          return TermsEnum.EMPTY;
-        }
-      };
-    } else if (BasicOperations.isTotal(automaton)) {
-      // matches all possible strings
-      factory = new TermsEnumFactory() {
-        @Override
-        protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-          return terms.iterator();
-        }
-      };
-    } else {
-      final String singleton;
-      final String commonPrefix;
-      
-      if (automaton.getSingleton() == null) {
-        commonPrefix = SpecialOperations.getCommonPrefix(automaton);
-        if (commonPrefix.length() > 0 && BasicOperations.sameLanguage(automaton, BasicAutomata.makeString(commonPrefix))) {
-          singleton = commonPrefix;
-        } else {
-          singleton = null;
-        }
-      } else {
-        commonPrefix = null;
-        singleton = automaton.getSingleton();
-      }
-      
-      if (singleton != null) {
-        // matches a fixed string in singleton or expanded representation
-        factory = new TermsEnumFactory() {
-          @Override
-          protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-            return new SingleTermsEnum(terms.iterator(), new Term(field, singleton));
-          }
-        };
-      } else if (BasicOperations.sameLanguage(automaton, BasicOperations.concatenate(
-          BasicAutomata.makeString(commonPrefix), BasicAutomata.makeAnyString()))) {
-        // matches a constant prefix
-        factory = new TermsEnumFactory() {
-          @Override
-          protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-            return new PrefixTermsEnum(terms.iterator(), new Term(field, commonPrefix));
-          }
-        };
-      } else {
-        final AutomatonTermsEnum.CompiledAutomaton compiled = 
-          new CompiledAutomaton(automaton, SpecialOperations.isFinite(automaton));
-        factory = new TermsEnumFactory() {
-          @Override
-          protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-            return new AutomatonTermsEnum(terms.iterator(), compiled);
-          }
-        };
-      }
-    }
+    this.compiled = new CompiledAutomaton(automaton);
   }
 
   @Override
   protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-    return factory.getTermsEnum(terms, atts);
+    return compiled.getTermsEnum(terms);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java Sat Aug 20 19:20:45 2011
@@ -28,6 +28,8 @@ import org.apache.lucene.index.DocsAndPo
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.Bits;
 
+// TODO: move this class to oal.index
+
 /**
  * Abstract class for enumerating a subset of all terms. 
  * 
@@ -42,7 +44,7 @@ import org.apache.lucene.util.Bits;
 public abstract class FilteredTermsEnum extends TermsEnum {
 
   private BytesRef initialSeekTerm = null;
-  private boolean doSeek = true;        
+  private boolean doSeek;
   private BytesRef actualTerm = null;
 
   private final TermsEnum tenum;
@@ -64,8 +66,17 @@ public abstract class FilteredTermsEnum 
    * @param tenum the terms enumeration to filter.
    */
   public FilteredTermsEnum(final TermsEnum tenum) {
+    this(tenum, true);
+  }
+
+  /**
+   * Creates a filtered {@link TermsEnum} on a terms enum.
+   * @param tenum the terms enumeration to filter.
+   */
+  public FilteredTermsEnum(final TermsEnum tenum, final boolean startWithSeek) {
     assert tenum != null;
     this.tenum = tenum;
+    doSeek = startWithSeek;
   }
 
   /**
@@ -190,18 +201,23 @@ public abstract class FilteredTermsEnum 
   @SuppressWarnings("fallthrough")
   @Override
   public BytesRef next() throws IOException {
+    //System.out.println("FTE.next doSeek=" + doSeek);
+    //new Throwable().printStackTrace(System.out);
     for (;;) {
       // Seek or forward the iterator
       if (doSeek) {
         doSeek = false;
         final BytesRef t = nextSeekTerm(actualTerm);
+        //System.out.println("  seek to t=" + (t == null ? "null" : t.utf8ToString()) + " tenum=" + tenum);
         // Make sure we always seek forward:
         assert actualTerm == null || t == null || getComparator().compare(t, actualTerm) > 0: "curTerm=" + actualTerm + " seekTerm=" + t;
         if (t == null || tenum.seekCeil(t, false) == SeekStatus.END) {
           // no more terms to seek to or enum exhausted
+          //System.out.println("  return null");
           return null;
         }
         actualTerm = tenum.term();
+        //System.out.println("  got term=" + actualTerm.utf8ToString());
       } else {
         actualTerm = tenum.next();
         if (actualTerm == null) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java Sat Aug 20 19:20:45 2011
@@ -137,12 +137,10 @@ public class FuzzyQuery extends MultiTer
 
   @Override
   protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-    TermsEnum tenum = terms.iterator();
-    
     if (!termLongEnough) {  // can only match if it's exact
-      return new SingleTermsEnum(tenum, term);
+      return new SingleTermsEnum(terms.iterator(), term.bytes());
     }
-    return new FuzzyTermsEnum(tenum, atts, getTerm(), minimumSimilarity, prefixLength);
+    return new FuzzyTermsEnum(terms, atts, getTerm(), minimumSimilarity, prefixLength);
   }
   
   /**

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Sat Aug 20 19:20:45 2011
@@ -17,12 +17,17 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.AutomatonTermsEnum.CompiledAutomaton;
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeImpl;
 import org.apache.lucene.util.AttributeSource;
@@ -34,13 +39,9 @@ import org.apache.lucene.util.automaton.
 import org.apache.lucene.util.automaton.BasicAutomata;
 import org.apache.lucene.util.automaton.BasicOperations;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.List;
-
 /** Subclass of TermsEnum for enumerating all terms that are similar
  * to the specified filter term.
  *
@@ -72,7 +73,7 @@ public final class FuzzyTermsEnum extend
   private int maxEdits;
   private final boolean raw;
 
-  private final TermsEnum tenum;
+  private final Terms terms;
   private final Term term;
   private final int termText[];
   private final int realPrefixLength;
@@ -94,7 +95,7 @@ public final class FuzzyTermsEnum extend
    * @param prefixLength Length of required common prefix. Default value is 0.
    * @throws IOException
    */
-  public FuzzyTermsEnum(TermsEnum tenum, AttributeSource atts, Term term, 
+  public FuzzyTermsEnum(Terms terms, AttributeSource atts, Term term, 
       final float minSimilarity, final int prefixLength) throws IOException {
     if (minSimilarity >= 1.0f && minSimilarity != (int)minSimilarity)
       throw new IllegalArgumentException("fractional edit distances are not allowed");
@@ -102,7 +103,7 @@ public final class FuzzyTermsEnum extend
       throw new IllegalArgumentException("minimumSimilarity cannot be less than 0");
     if(prefixLength < 0)
       throw new IllegalArgumentException("prefixLength cannot be less than 0");
-    this.tenum = tenum;
+    this.terms = terms;
     this.term = term;
 
     // convert the string into a utf32 int[] representation for fast comparisons
@@ -143,8 +144,10 @@ public final class FuzzyTermsEnum extend
       throws IOException {
     final List<CompiledAutomaton> runAutomata = initAutomata(editDistance);
     if (editDistance < runAutomata.size()) {
-      return new AutomatonFuzzyTermsEnum(runAutomata.subList(0, editDistance + 1)
-          .toArray(new CompiledAutomaton[editDistance + 1]), lastTerm);
+      //if (BlockTreeTermsWriter.DEBUG) System.out.println("FuzzyTE.getAEnum: ed=" + editDistance + " lastTerm=" + (lastTerm==null ? "null" : lastTerm.utf8ToString()));
+      final CompiledAutomaton compiled = runAutomata.get(editDistance);
+      return new AutomatonFuzzyTermsEnum(terms.intersect(compiled, lastTerm == null ? null : compiled.floor(lastTerm, new BytesRef())),
+                                         runAutomata.subList(0, editDistance + 1).toArray(new CompiledAutomaton[editDistance + 1]));
     } else {
       return null;
     }
@@ -153,6 +156,7 @@ public final class FuzzyTermsEnum extend
   /** initialize levenshtein DFAs up to maxDistance, if possible */
   private List<CompiledAutomaton> initAutomata(int maxDistance) {
     final List<CompiledAutomaton> runAutomata = dfaAtt.automata();
+    //System.out.println("cached automata size: " + runAutomata.size());
     if (runAutomata.size() <= maxDistance && 
         maxDistance <= LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
       LevenshteinAutomata builder = 
@@ -160,13 +164,14 @@ public final class FuzzyTermsEnum extend
 
       for (int i = runAutomata.size(); i <= maxDistance; i++) {
         Automaton a = builder.toAutomaton(i);
+        //System.out.println("compute automaton n=" + i);
         // constant prefix
         if (realPrefixLength > 0) {
           Automaton prefix = BasicAutomata.makeString(
             UnicodeUtil.newString(termText, 0, realPrefixLength));
           a = BasicOperations.concatenate(prefix, a);
         }
-        runAutomata.add(new CompiledAutomaton(a, true));
+        runAutomata.add(new CompiledAutomaton(a, true, false));
       }
     }
     return runAutomata;
@@ -301,65 +306,65 @@ public final class FuzzyTermsEnum extend
   public BytesRef term() throws IOException {
     return actualEnum.term();
   }
-  
+
   /**
-   * Implement fuzzy enumeration with automaton.
+   * Implement fuzzy enumeration with Terms.intersect.
    * <p>
    * This is the fastest method as opposed to LinearFuzzyTermsEnum:
    * as enumeration is logarithmic to the number of terms (instead of linear)
    * and comparison is linear to length of the term (rather than quadratic)
    */
-  private class AutomatonFuzzyTermsEnum extends AutomatonTermsEnum {
+  private class AutomatonFuzzyTermsEnum extends FilteredTermsEnum {
     private final ByteRunAutomaton matchers[];
     
     private final BytesRef termRef;
     
-    private final BytesRef lastTerm;
     private final BoostAttribute boostAtt =
       attributes().addAttribute(BoostAttribute.class);
     
-    public AutomatonFuzzyTermsEnum(CompiledAutomaton compiled[], 
-        BytesRef lastTerm) throws IOException {
-      super(tenum, compiled[compiled.length - 1]);
+    public AutomatonFuzzyTermsEnum(TermsEnum tenum, CompiledAutomaton compiled[]) 
+      throws IOException {
+      super(tenum, false);
       this.matchers = new ByteRunAutomaton[compiled.length];
       for (int i = 0; i < compiled.length; i++)
         this.matchers[i] = compiled[i].runAutomaton;
-      this.lastTerm = lastTerm;
       termRef = new BytesRef(term.text());
     }
-    
+
     /** finds the smallest Lev(n) DFA that accepts the term. */
     @Override
     protected AcceptStatus accept(BytesRef term) {    
+      //System.out.println("AFTE.accept term=" + term);
       int ed = matchers.length - 1;
       
-      if (matches(term, ed)) { // we match the outer dfa
-        // now compute exact edit distance
-        while (ed > 0) {
-          if (matches(term, ed - 1)) {
-            ed--;
-          } else {
-            break;
-          }
-        }
-        
-        // scale to a boost and return (if similarity > minSimilarity)
-        if (ed == 0) { // exact match
-          boostAtt.setBoost(1.0F);
-          return AcceptStatus.YES_AND_SEEK;
+      // we are wrapping either an intersect() TermsEnum or an AutomatonTermsENum,
+      // so we know the outer DFA always matches.
+      // now compute exact edit distance
+      while (ed > 0) {
+        if (matches(term, ed - 1)) {
+          ed--;
         } else {
-          final int codePointCount = UnicodeUtil.codePointCount(term);
-          final float similarity = 1.0f - ((float) ed / (float) 
-              (Math.min(codePointCount, termLength)));
-          if (similarity > minSimilarity) {
-            boostAtt.setBoost((similarity - minSimilarity) * scale_factor);
-            return AcceptStatus.YES_AND_SEEK;
-          } else {
-            return AcceptStatus.NO_AND_SEEK;
-          }
+          break;
         }
+      }
+      //System.out.println("CHECK term=" + term.utf8ToString() + " ed=" + ed);
+      
+      // scale to a boost and return (if similarity > minSimilarity)
+      if (ed == 0) { // exact match
+        boostAtt.setBoost(1.0F);
+        //System.out.println("  yes");
+        return AcceptStatus.YES;
       } else {
-        return AcceptStatus.NO_AND_SEEK;
+        final int codePointCount = UnicodeUtil.codePointCount(term);
+        final float similarity = 1.0f - ((float) ed / (float) 
+            (Math.min(codePointCount, termLength)));
+        if (similarity > minSimilarity) {
+          boostAtt.setBoost((similarity - minSimilarity) * scale_factor);
+          //System.out.println("  yes");
+          return AcceptStatus.YES;
+        } else {
+          return AcceptStatus.NO;
+        }
       }
     }
     
@@ -367,16 +372,8 @@ public final class FuzzyTermsEnum extend
     final boolean matches(BytesRef term, int k) {
       return k == 0 ? term.equals(termRef) : matchers[k].run(term.bytes, term.offset, term.length);
     }
-    
-    /** defers to superclass, except can start at an arbitrary location */
-    @Override
-    protected BytesRef nextSeekTerm(BytesRef term) throws IOException {
-      if (term == null)
-        term = lastTerm;
-      return super.nextSeekTerm(term);
-    }
   }
-  
+
   /**
    * Implement fuzzy enumeration with linear brute force.
    */
@@ -408,7 +405,7 @@ public final class FuzzyTermsEnum extend
      * @throws IOException
      */
     public LinearFuzzyTermsEnum() throws IOException {
-      super(tenum);
+      super(terms.iterator());
 
       this.text = new int[termLength - realPrefixLength];
       System.arraycopy(termText, realPrefixLength, text, 0, text.length);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Sat Aug 20 19:20:45 2011
@@ -882,6 +882,6 @@ public class IndexSearcher implements Cl
 
   @Override
   public String toString() {
-    return "IndexSearcher(" + reader + ")";
+    return "IndexSearcher(" + reader + "; executor=" + executor + ")";
   }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixQuery.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixQuery.java Sat Aug 20 19:20:45 2011
@@ -51,7 +51,7 @@ public class PrefixQuery extends MultiTe
       // no prefix -- match all terms for this field:
       return tenum;
     }
-    return new PrefixTermsEnum(tenum, prefix);
+    return new PrefixTermsEnum(tenum, prefix.bytes());
   }
 
   /** Prints a user-readable version of this query. */

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java Sat Aug 20 19:20:45 2011
@@ -19,7 +19,6 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 
@@ -34,9 +33,9 @@ public class PrefixTermsEnum extends Fil
 
   private final BytesRef prefixRef;
 
-  public PrefixTermsEnum(TermsEnum tenum, Term prefix) throws IOException {
+  public PrefixTermsEnum(TermsEnum tenum, BytesRef prefixText) throws IOException {
     super(tenum);
-    setInitialSeekTerm(prefixRef = prefix.bytes());
+    setInitialSeekTerm(this.prefixRef = prefixText);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/SingleTermsEnum.java Sat Aug 20 19:20:45 2011
@@ -19,7 +19,6 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 
@@ -39,10 +38,10 @@ public final class SingleTermsEnum exten
    * After calling the constructor the enumeration is already pointing to the term,
    * if it exists.
    */
-  public SingleTermsEnum(TermsEnum tenum, Term singleTerm) throws IOException {
+  public SingleTermsEnum(TermsEnum tenum, BytesRef termText) throws IOException {
     super(tenum);
-    singleRef = singleTerm.bytes();
-    setInitialSeekTerm(singleRef);
+    singleRef = termText;
+    setInitialSeekTerm(termText);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermQuery.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermQuery.java Sat Aug 20 19:20:45 2011
@@ -98,17 +98,18 @@ public class TermQuery extends Query {
     TermsEnum getTermsEnum(AtomicReaderContext context) throws IOException {
       final TermState state = termStates.get(context.ord);
       if (state == null) { // term is not present in that reader
-        assert termNotInReader(context.reader, term.field(), term.bytes()) : "no termstate found but term exists in reader";
+        assert termNotInReader(context.reader, term.field(), term.bytes()) : "no termstate found but term exists in reader term=" + term;
         return null;
       }
-      final TermsEnum termsEnum = context.reader.terms(term.field())
-          .getThreadTermsEnum();
+      //System.out.println("LD=" + reader.getLiveDocs() + " set?=" + (reader.getLiveDocs() != null ? reader.getLiveDocs().get(0) : "null"));
+      final TermsEnum termsEnum = context.reader.terms(term.field()).getThreadTermsEnum();
       termsEnum.seekExact(term.bytes(), state);
       return termsEnum;
     }
     
     private boolean termNotInReader(IndexReader reader, String field, BytesRef bytes) throws IOException {
       // only called from assert
+      //System.out.println("TQ.termNotInReader reader=" + reader + " term=" + field + ":" + bytes.utf8ToString());
       final Terms terms = reader.terms(field);
       return terms == null || terms.docFreq(bytes) == 0;
     }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermScorer.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermScorer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermScorer.java Sat Aug 20 19:20:45 2011
@@ -70,6 +70,7 @@ final class TermScorer extends Scorer {
   public boolean score(Collector c, int end, int firstDocID) throws IOException {
     c.setScorer(this);
     while (doc < end) {                           // for docs in window
+      //System.out.println("TS: collect doc=" + doc);
       c.collect(doc);                      // collect score
       if (++pointer >= pointerMax) {
         refillBuffer();

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java Sat Aug 20 19:20:45 2011
@@ -78,15 +78,38 @@ public abstract class TopTermsRewrite<Q 
       public void setNextEnum(TermsEnum termsEnum) throws IOException {
         this.termsEnum = termsEnum;
         this.termComp = termsEnum.getComparator();
+        
+        assert compareToLastTerm(null);
+
         // lazy init the initial ScoreTerm because comparator is not known on ctor:
         if (st == null)
           st = new ScoreTerm(this.termComp, new TermContext(topReaderContext));
         boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class);
       }
     
+      // for assert:
+      private BytesRef lastTerm;
+      private boolean compareToLastTerm(BytesRef t) throws IOException {
+        if (lastTerm == null && t != null) {
+          lastTerm = new BytesRef(t);
+        } else if (t == null) {
+          lastTerm = null;
+        } else {
+          assert termsEnum.getComparator().compare(lastTerm, t) < 0: "lastTerm=" + lastTerm + " t=" + t;
+          lastTerm.copy(t);
+        }
+        return true;
+      }
+  
       @Override
       public boolean collect(BytesRef bytes) throws IOException {
         final float boost = boostAtt.getBoost();
+
+        // make sure within a single seg we always collect
+        // terms in order
+        assert compareToLastTerm(bytes);
+
+        //System.out.println("TTR.collect term=" + bytes.utf8ToString() + " boost=" + boost + " ord=" + readerContext.ord);
         // ignore uncompetitive hits
         if (stQueue.size() == maxSize) {
           final ScoreTerm t = stQueue.peek();
@@ -134,9 +157,10 @@ public abstract class TopTermsRewrite<Q 
     final Q q = getTopLevelQuery();
     final ScoreTerm[] scoreTerms = stQueue.toArray(new ScoreTerm[stQueue.size()]);
     ArrayUtil.mergeSort(scoreTerms, scoreTermSortByTermComp);
+    
     for (final ScoreTerm st : scoreTerms) {
       final Term term = new Term(query.field, st.bytes);
-      assert reader.docFreq(term) == st.termState.docFreq() : "reader DF is " + reader.docFreq(term) + " vs " + st.termState.docFreq();
+      assert reader.docFreq(term) == st.termState.docFreq() : "reader DF is " + reader.docFreq(term) + " vs " + st.termState.docFreq() + " term=" + term;
       addClause(q, term, st.termState.docFreq(), query.getBoost() * st.boost, st.termState); // add to query
     }
     query.incTotalNumberOfTerms(scoreTerms.length);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/ByteArrayDataInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/ByteArrayDataInput.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/ByteArrayDataInput.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/ByteArrayDataInput.java Sat Aug 20 19:20:45 2011
@@ -43,16 +43,30 @@ public final class ByteArrayDataInput ex
     reset(bytes, 0, bytes.length);
   }
 
+  // NOTE: sets pos to 0, which is not right if you had
+  // called reset w/ non-zero offset!!
+  public void rewind() {
+    pos = 0;
+  }
+
   public int getPosition() {
     return pos;
   }
 
+  public void setPosition(int pos) {
+    this.pos = pos;
+  }
+
   public void reset(byte[] bytes, int offset, int len) {
     this.bytes = bytes;
     pos = offset;
     limit = offset + len;
   }
 
+  public int length() {
+    return limit;
+  }
+
   public boolean eof() {
     return pos == limit;
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java Sat Aug 20 19:20:45 2011
@@ -19,10 +19,6 @@ package org.apache.lucene.store;
 
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.Lock;
 import org.apache.lucene.util.IOUtils;
 
 import java.util.Collection;
@@ -189,14 +185,14 @@ public abstract class CompoundFileDirect
   }
   
   @Override
-  public synchronized IndexInput openInput(String id, IOContext context) throws IOException {
+  public synchronized IndexInput openInput(String fileName, IOContext context) throws IOException {
     ensureOpen();
     assert !openForWrite;
-    id = IndexFileNames.stripSegmentName(id);
+    final String id = IndexFileNames.stripSegmentName(fileName);
     final FileEntry entry = entries.get(id);
-    if (entry == null)
-      throw new IOException("No sub-file with id " + id + " found (files: " + entries.keySet() + ")");
-    
+    if (entry == null) {
+      throw new IOException("No sub-file with id " + id + " found (fileName=" + fileName + " files: " + entries.keySet() + ")");
+    }
     return openInputSlice(id, entry.offset, entry.length, readBufferSize);
   }
   

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FSDirectory.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/FSDirectory.java Sat Aug 20 19:20:45 2011
@@ -448,6 +448,7 @@ public abstract class FSDirectory extend
     /** output methods: */
     @Override
     public void flushBuffer(byte[] b, int offset, int size) throws IOException {
+      assert isOpen;
       if (rateLimiter != null) {
         rateLimiter.pause(size);
       }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRef.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BytesRef.java Sat Aug 20 19:20:45 2011
@@ -65,6 +65,18 @@ public final class BytesRef implements C
     this.bytes = new byte[capacity];
   }
 
+  /** Incoming IntsRef values must be Byte.MIN_VALUE -
+   *  Byte.MAX_VALUE. */
+  public BytesRef(IntsRef intsRef) {
+    bytes = new byte[intsRef.length];
+    for(int idx=0;idx<intsRef.length;idx++) {
+      final int v = intsRef.ints[intsRef.offset + idx];
+      assert v >= Byte.MIN_VALUE && v <= Byte.MAX_VALUE;
+      bytes[idx] = (byte) v;
+    }
+    length = intsRef.length;
+  }
+
   /**
    * @param text Initialize the byte[] from the UTF8 bytes
    * for the provided Sring.  This must be well-formed

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/TermContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/TermContext.java?rev=1159905&r1=1159904&r2=1159905&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/TermContext.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/TermContext.java Sat Aug 20 19:20:45 2011
@@ -21,14 +21,13 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
-import org.apache.lucene.index.IndexReader.ReaderContext;
-import org.apache.lucene.index.TermsEnum.SeekStatus;
 
 /**
  * Maintains a {@link IndexReader} {@link TermState} view over
@@ -45,6 +44,9 @@ public final class TermContext {
   private int docFreq;
   private long totalTermFreq;
 
+  //public static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+
+
   /**
    * Creates an empty {@link TermContext} from a {@link ReaderContext}
    */
@@ -85,7 +87,9 @@ public final class TermContext {
     final BytesRef bytes = term.bytes();
     final TermContext perReaderTermState = new TermContext(context);
     final AtomicReaderContext[] leaves = ReaderUtil.leaves(context);
+    //if (DEBUG) System.out.println("prts.build term=" + term);
     for (int i = 0; i < leaves.length; i++) {
+      //if (DEBUG) System.out.println("  r=" + leaves[i].reader);
       final Fields fields = leaves[i].reader.fields();
       if (fields != null) {
         final Terms terms = fields.terms(field);
@@ -93,6 +97,7 @@ public final class TermContext {
           final TermsEnum termsEnum = terms.getThreadTermsEnum(); // thread-private don't share!
           if (termsEnum.seekExact(bytes, cache)) { 
             final TermState termState = termsEnum.termState();
+            //if (DEBUG) System.out.println("    found");
             perReaderTermState.register(termState, leaves[i].ord, termsEnum.docFreq(), termsEnum.totalTermFreq());
           }
         }

Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java?rev=1159905&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java (added)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java Sat Aug 20 19:20:45 2011
@@ -0,0 +1,313 @@
+package org.apache.lucene.util.automaton;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+  
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.PrefixTermsEnum;
+import org.apache.lucene.search.SingleTermsEnum;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Immutable class holding compiled details for a given
+ * Automaton.  The Automaton is deterministic, must not have
+ * dead states but may not be minimal.
+ *
+ * @lucene.experimental
+ */
+public class CompiledAutomaton {
+  public enum AUTOMATON_TYPE {NONE, ALL, SINGLE, PREFIX, NORMAL};
+  public final AUTOMATON_TYPE type;
+
+  // For PREFIX, this is the prefix term; for SINGLE this is
+  // the singleton term:
+  public final BytesRef term;
+
+  // NOTE: the next 4 members are only non-null if type ==
+  // NORMAL:
+  public final ByteRunAutomaton runAutomaton;
+  // TODO: would be nice if these sortedTransitions had "int
+  // to;" instead of "State to;" somehow:
+  public final Transition[][] sortedTransitions;
+  public final BytesRef commonSuffixRef;
+  public final Boolean finite;
+
+  public CompiledAutomaton(Automaton automaton) {
+    this(automaton, null, true);
+  }
+
+  public CompiledAutomaton(Automaton automaton, Boolean finite, boolean simplify) {
+
+    if (simplify) {
+      // Test whether the automaton is a "simple" form and
+      // if so, don't create a runAutomaton.  Note that on a
+      // large automaton these tests could be costly:
+      if (BasicOperations.isEmpty(automaton)) {
+        // matches nothing
+        type = AUTOMATON_TYPE.NONE;
+        term = null;
+        commonSuffixRef = null;
+        runAutomaton = null;
+        sortedTransitions = null;
+        this.finite = null;
+        return;
+      } else if (BasicOperations.isTotal(automaton)) {
+        // matches all possible strings
+        type = AUTOMATON_TYPE.ALL;
+        term = null;
+        commonSuffixRef = null;
+        runAutomaton = null;
+        sortedTransitions = null;
+        this.finite = null;
+        return;
+      } else {
+        final String commonPrefix;
+        final String singleton;
+        if (automaton.getSingleton() == null) {
+          commonPrefix = SpecialOperations.getCommonPrefix(automaton);
+          if (commonPrefix.length() > 0 && BasicOperations.sameLanguage(automaton, BasicAutomata.makeString(commonPrefix))) {
+            singleton = commonPrefix;
+          } else {
+            singleton = null;
+          }
+        } else {
+          commonPrefix = null;
+          singleton = automaton.getSingleton();
+        }
+      
+        if (singleton != null) {
+          // matches a fixed string in singleton or expanded
+          // representation
+          type = AUTOMATON_TYPE.SINGLE;
+          term = new BytesRef(singleton);
+          commonSuffixRef = null;
+          runAutomaton = null;
+          sortedTransitions = null;
+          this.finite = null;
+          return;
+        } else if (BasicOperations.sameLanguage(automaton, BasicOperations.concatenate(
+                                                                                       BasicAutomata.makeString(commonPrefix), BasicAutomata.makeAnyString()))) {
+          // matches a constant prefix
+          type = AUTOMATON_TYPE.PREFIX;
+          term = new BytesRef(commonPrefix);
+          commonSuffixRef = null;
+          runAutomaton = null;
+          sortedTransitions = null;
+          this.finite = null;
+          return;
+        }
+      }
+    }
+
+    type = AUTOMATON_TYPE.NORMAL;
+    term = null;
+    if (finite == null) {
+      this.finite = SpecialOperations.isFinite(automaton);
+    } else {
+      this.finite = finite;
+    }
+    Automaton utf8 = new UTF32ToUTF8().convert(automaton);
+    if (this.finite) {
+      commonSuffixRef = null;
+    } else {
+      commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(utf8);
+    }
+    runAutomaton = new ByteRunAutomaton(utf8, true);
+    sortedTransitions = utf8.getSortedTransitions();
+  }
+  
+  //private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+
+  private BytesRef addTail(int state, BytesRef term, int idx, int leadLabel) {
+
+    // Find biggest transition that's < label
+    // TODO: use binary search here
+    Transition maxTransition = null;
+    for (Transition transition : sortedTransitions[state]) {
+      if (transition.min < leadLabel) {
+        maxTransition = transition;
+      }
+    }
+
+    assert maxTransition != null;
+
+    // Append floorLabel
+    final int floorLabel;
+    if (maxTransition.max > leadLabel-1) {
+      floorLabel = leadLabel-1;
+    } else {
+      floorLabel = maxTransition.max;
+    }
+    if (idx >= term.bytes.length) {
+      term.grow(1+idx);
+    }
+    //if (DEBUG) System.out.println("  add floorLabel=" + (char) floorLabel + " idx=" + idx);
+    term.bytes[idx] = (byte) floorLabel;
+
+    state = maxTransition.to.getNumber();
+    idx++;
+
+    // Push down to last accept state
+    while (true) {
+      Transition[] transitions = sortedTransitions[state];
+      if (transitions.length == 0) {
+        assert runAutomaton.isAccept(state);
+        term.length = idx;
+        //if (DEBUG) System.out.println("  return " + term.utf8ToString());
+        return term;
+      } else {
+        // We are pushing "top" -- so get last label of
+        // last transition:
+        assert transitions.length != 0;
+        Transition lastTransition = transitions[transitions.length-1];
+        if (idx >= term.bytes.length) {
+          term.grow(1+idx);
+        }
+        //if (DEBUG) System.out.println("  push maxLabel=" + (char) lastTransition.max + " idx=" + idx);
+        term.bytes[idx] = (byte) lastTransition.max;
+        state = lastTransition.to.getNumber();
+        idx++;
+      }
+    }
+  }
+
+  // TODO: should this take startTerm too?  This way
+  // Terms.intersect could forward to this method if type !=
+  // NORMAL:
+  public TermsEnum getTermsEnum(Terms terms) throws IOException {
+    switch(type) {
+    case NONE:
+      return TermsEnum.EMPTY;
+    case ALL:
+      return terms.iterator();
+    case SINGLE:
+      return new SingleTermsEnum(terms.iterator(), term);
+    case PREFIX:
+      // TODO: this is very likely faster than .intersect,
+      // but we should test and maybe cutover
+      return new PrefixTermsEnum(terms.iterator(), term);
+    case NORMAL:
+      return terms.intersect(this, null);
+    default:
+      // unreachable
+      throw new RuntimeException("unhandled case");
+    }
+  }
+
+  /** Finds largest term accepted by this Automaton, that's
+   *  <= the provided input term.  The result is placed in
+   *  output; it's fine for output and input to point to
+   *  the same BytesRef.  The returned result is either the
+   *  provided output, or null if there is no floor term
+   *  (ie, the provided input term is before the first term
+   *  accepted by this Automaton). */
+  public BytesRef floor(BytesRef input, BytesRef output) {
+
+    output.offset = 0;
+    //if (DEBUG) System.out.println("CA.floor input=" + input.utf8ToString());
+
+    int state = runAutomaton.getInitialState();
+
+    // Special case empty string:
+    if (input.length == 0) {
+      if (runAutomaton.isAccept(state)) {
+        output.length = 0;
+        return output;
+      } else {
+        return null;
+      }
+    }
+
+    final List<Integer> stack = new ArrayList<Integer>();
+
+    int idx = 0;
+    while (true) {
+      int label = input.bytes[input.offset + idx] & 0xff;
+      int nextState = runAutomaton.step(state, label);
+      //if (DEBUG) System.out.println("  cycle label=" + (char) label + " nextState=" + nextState);
+
+      if (idx == input.length-1) {
+        if (nextState != -1 && runAutomaton.isAccept(nextState)) {
+          // Input string is accepted
+          if (idx >= output.bytes.length) {
+            output.grow(1+idx);
+          }
+          output.bytes[idx] = (byte) label;
+          output.length = input.length;
+          //if (DEBUG) System.out.println("  input is accepted; return term=" + output.utf8ToString());
+          return output;
+        } else {
+          nextState = -1;
+        }
+      }
+
+      if (nextState == -1) {
+
+        // Pop back to a state that has a transition
+        // <= our label:
+        while (true) {
+          Transition[] transitions = sortedTransitions[state];
+          if (transitions.length == 0) {
+            assert runAutomaton.isAccept(state);
+            output.length = idx;
+            //if (DEBUG) System.out.println("  return " + output.utf8ToString());
+            return output;
+          } else if (label-1 < transitions[0].min) {
+
+            if (runAutomaton.isAccept(state)) {
+              output.length = idx;
+              //if (DEBUG) System.out.println("  return " + output.utf8ToString());
+              return output;
+            }
+            // pop
+            if (stack.size() == 0) {
+              //if (DEBUG) System.out.println("  pop ord=" + idx + " return null");
+              return null;
+            } else {
+              state = stack.remove(stack.size()-1);
+              idx--;
+              //if (DEBUG) System.out.println("  pop ord=" + (idx+1) + " label=" + (char) label + " first trans.min=" + (char) transitions[0].min);
+              label = input.bytes[input.offset + idx] & 0xff;
+            }
+
+          } else {
+            //if (DEBUG) System.out.println("  stop pop ord=" + idx + " first trans.min=" + (char) transitions[0].min);
+            break;
+          }
+        }
+
+        //if (DEBUG) System.out.println("  label=" + (char) label + " idx=" + idx);
+
+        return addTail(state, output, idx, label);
+        
+      } else {
+        if (idx >= output.bytes.length) {
+          output.grow(1+idx);
+        }
+        output.bytes[idx] = (byte) label;
+        stack.add(state);
+        state = nextState;
+        idx++;
+      }
+    }
+  }
+}