You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by bu...@apache.org on 2011/01/13 20:53:39 UTC

svn commit: r1058718 [8/18] - in /lucene/dev/branches/realtime_search: ./ lucene/ lucene/contrib/ lucene/contrib/ant/src/java/org/apache/lucene/ant/ lucene/contrib/ant/src/test/org/apache/lucene/ant/ lucene/contrib/benchmark/ lucene/contrib/demo/src/ja...

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSource.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSource.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSource.java Thu Jan 13 19:53:21 2011
@@ -17,7 +17,10 @@ package org.apache.lucene.search.functio
  * limitations under the License.
  */
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.CompositeReaderContext;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.search.function.DocValues;
 
 import java.io.IOException;
@@ -39,11 +42,23 @@ public abstract class ValueSource implem
 
   /**
    * Return the DocValues used by the function query.
-   * @param reader the IndexReader used to read these values.
+   * @param context the IndexReader used to read these values.
    * If any caching is involved, that caching would also be IndexReader based.  
    * @throws IOException for any error.
    */
-  public abstract DocValues getValues(IndexReader reader) throws IOException;
+  public abstract DocValues getValues(AtomicReaderContext context) throws IOException;
+  
+  /**
+   * Return the DocValues used by the function query.
+   * @deprecated (4.0) This method is temporary, to ease the migration to segment-based
+   * searching. Please change your code to not pass {@link CompositeReaderContext} to these
+   * APIs. Use {@link #getValues(AtomicReaderContext)} instead
+   */
+  @Deprecated
+  public DocValues getValues(ReaderContext context) throws IOException {
+    return getValues((AtomicReaderContext) context);
+  }
+
 
   /** 
    * description of field, used in explain() 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java Thu Jan 13 19:53:21 2011
@@ -19,8 +19,9 @@ package org.apache.lucene.search.functio
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.*;
+import org.apache.lucene.search.Weight.ScorerContext;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.Bits;
 
@@ -68,7 +69,7 @@ public class ValueSourceQuery extends Qu
     float queryNorm;
     float queryWeight;
 
-    public ValueSourceWeight(Searcher searcher) {
+    public ValueSourceWeight(IndexSearcher searcher) {
       this.similarity = getSimilarity(searcher);
     }
 
@@ -99,14 +100,14 @@ public class ValueSourceQuery extends Qu
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
-      return new ValueSourceScorer(similarity, reader, this);
+    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
+      return new ValueSourceScorer(similarity, context, this);
     }
 
     /*(non-Javadoc) @see org.apache.lucene.search.Weight#explain(org.apache.lucene.index.IndexReader, int) */
     @Override
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
-      DocValues vals = valSrc.getValues(reader);
+    public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
+      DocValues vals = valSrc.getValues(context);
       float sc = queryWeight * vals.floatVal(doc);
 
       Explanation result = new ComplexExplanation(
@@ -133,12 +134,13 @@ public class ValueSourceQuery extends Qu
     private int doc = -1;
 
     // constructor
-    private ValueSourceScorer(Similarity similarity, IndexReader reader, ValueSourceWeight w) throws IOException {
+    private ValueSourceScorer(Similarity similarity, AtomicReaderContext context, ValueSourceWeight w) throws IOException {
       super(similarity,w);
+      final IndexReader reader = context.reader;
       qWeight = w.getValue();
       // this is when/where the values are first created.
-      vals = valSrc.getValues(reader);
-      delDocs = MultiFields.getDeletedDocs(reader);
+      vals = valSrc.getValues(context);
+      delDocs = reader.getDeletedDocs();
       maxDoc = reader.maxDoc();
     }
 
@@ -173,7 +175,7 @@ public class ValueSourceQuery extends Qu
   }
 
   @Override
-  public Weight createWeight(Searcher searcher) {
+  public Weight createWeight(IndexSearcher searcher) {
     return new ValueSourceQuery.ValueSourceWeight(searcher);
   }
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/package.html?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/package.html (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/package.html Thu Jan 13 19:53:21 2011
@@ -36,8 +36,8 @@ Code to search indices.
 Search over indices.
 
 Applications usually call {@link
-org.apache.lucene.search.Searcher#search(Query,int)} or {@link
-org.apache.lucene.search.Searcher#search(Query,Filter,int)}.
+org.apache.lucene.search.IndexSearcher#search(Query,int)} or {@link
+org.apache.lucene.search.IndexSearcher#search(Query,Filter,int)}.
 
     <!-- FILL IN MORE HERE -->   
 </p>

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java Thu Jan 13 19:53:21 2011
@@ -17,10 +17,10 @@ package org.apache.lucene.search.payload
  * limitations under the License.
  */
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.spans.NearSpansOrdered;
@@ -66,7 +66,7 @@ public class PayloadNearQuery extends Sp
   }
 
   @Override
-  public Weight createWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
     return new PayloadNearSpanWeight(this, searcher);
   }
 
@@ -137,16 +137,15 @@ public class PayloadNearQuery extends Sp
   }
 
   public class PayloadNearSpanWeight extends SpanWeight {
-    public PayloadNearSpanWeight(SpanQuery query, Searcher searcher)
+    public PayloadNearSpanWeight(SpanQuery query, IndexSearcher searcher)
         throws IOException {
       super(query, searcher);
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder,
-        boolean topScorer) throws IOException {
-      return new PayloadNearSpanScorer(query.getSpans(reader), this,
-          similarity, reader.norms(query.getField()));
+    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
+      return new PayloadNearSpanScorer(query.getSpans(context.reader), this,
+          similarity, context.reader.norms(query.getField()));
     }
   }
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java Thu Jan 13 19:53:21 2011
@@ -17,10 +17,10 @@ package org.apache.lucene.search.payload
  * limitations under the License.
  */
 
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.Similarity;
@@ -62,22 +62,21 @@ public class PayloadTermQuery extends Sp
   }
 
   @Override
-  public Weight createWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
     return new PayloadTermWeight(this, searcher);
   }
 
   protected class PayloadTermWeight extends SpanWeight {
 
-    public PayloadTermWeight(PayloadTermQuery query, Searcher searcher)
+    public PayloadTermWeight(PayloadTermQuery query, IndexSearcher searcher)
         throws IOException {
       super(query, searcher);
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder,
-        boolean topScorer) throws IOException {
-      return new PayloadTermSpanScorer((TermSpans) query.getSpans(reader),
-          this, similarity, reader.norms(query.getField()));
+    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
+      return new PayloadTermSpanScorer((TermSpans) query.getSpans(context.reader),
+          this, similarity, context.reader.norms(query.getField()));
     }
 
     protected class PayloadTermSpanScorer extends SpanScorer {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java Thu Jan 13 19:53:21 2011
@@ -24,7 +24,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Weight;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.util.ToStringUtils;
 
@@ -102,12 +102,12 @@ public class FieldMaskingSpanQuery exten
   }  
 
   @Override
-  public Weight createWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
     return maskedQuery.createWeight(searcher);
   }
 
   @Override
-  public Similarity getSimilarity(Searcher searcher) {
+  public Similarity getSimilarity(IndexSearcher searcher) {
     return maskedQuery.getSimilarity(searcher);
   }
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java Thu Jan 13 19:53:21 2011
@@ -26,6 +26,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TopTermsRewrite;
 import org.apache.lucene.search.ScoringRewrite;
 import org.apache.lucene.search.BooleanClause.Occur; // javadocs only
+import org.apache.lucene.util.PerReaderTermState;
 
 /**
  * Wraps any {@link MultiTermQuery} as a {@link SpanQuery}, 
@@ -153,7 +154,7 @@ public class SpanMultiTermQueryWrapper<Q
       }
     
       @Override
-      protected void addClause(SpanOrQuery topLevel, Term term, int docCount, float boost) {
+      protected void addClause(SpanOrQuery topLevel, Term term, int docCount, float boost, PerReaderTermState states) {
         final SpanTermQuery q = new SpanTermQuery(term);
         q.setBoost(boost);
         topLevel.addClause(q);
@@ -202,7 +203,7 @@ public class SpanMultiTermQueryWrapper<Q
         }
 
         @Override
-        protected void addClause(SpanOrQuery topLevel, Term term, int docFreq, float boost) {
+        protected void addClause(SpanOrQuery topLevel, Term term, int docFreq, float boost, PerReaderTermState states) {
           final SpanTermQuery q = new SpanTermQuery(term);
           q.setBoost(boost);
           topLevel.addClause(q);

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java Thu Jan 13 19:53:21 2011
@@ -21,7 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Weight;
 
 /** Base class for span-based queries. */
@@ -34,7 +34,7 @@ public abstract class SpanQuery extends 
   public abstract String getField();
 
   @Override
-  public Weight createWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
     return new SpanWeight(this, searcher);
   }
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java Thu Jan 13 19:53:21 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.*;
 import org.apache.lucene.search.Explanation.IDFExplanation;
@@ -40,7 +40,7 @@ public class SpanWeight extends Weight {
   protected SpanQuery query;
   private IDFExplanation idfExp;
 
-  public SpanWeight(SpanQuery query, Searcher searcher)
+  public SpanWeight(SpanQuery query, IndexSearcher searcher)
     throws IOException {
     this.similarity = query.getSimilarity(searcher);
     this.query = query;
@@ -72,13 +72,13 @@ public class SpanWeight extends Weight {
   }
 
   @Override
-  public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
-    return new SpanScorer(query.getSpans(reader), this, similarity, reader
+  public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
+    return new SpanScorer(query.getSpans(context.reader), this, similarity, context.reader
         .norms(query.getField()));
   }
 
   @Override
-  public Explanation explain(IndexReader reader, int doc)
+  public Explanation explain(AtomicReaderContext context, int doc)
     throws IOException {
 
     ComplexExplanation result = new ComplexExplanation();
@@ -111,12 +111,12 @@ public class SpanWeight extends Weight {
     fieldExpl.setDescription("fieldWeight("+field+":"+query.toString(field)+
                              " in "+doc+"), product of:");
 
-    Explanation tfExpl = ((SpanScorer)scorer(reader, true, false)).explain(doc);
+    Explanation tfExpl = ((SpanScorer)scorer(context, ScorerContext.def())).explain(doc);
     fieldExpl.addDetail(tfExpl);
     fieldExpl.addDetail(idfExpl);
 
     Explanation fieldNormExpl = new Explanation();
-    byte[] fieldNorms = reader.norms(field);
+    byte[] fieldNorms = context.reader.norms(field);
     float fieldNorm =
       fieldNorms!=null ? similarity.decodeNormValue(fieldNorms[doc]) : 1.0f;
     fieldNormExpl.setValue(fieldNorm);

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/RAMInputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/RAMInputStream.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/RAMInputStream.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/RAMInputStream.java Thu Jan 13 19:53:21 2011
@@ -83,6 +83,7 @@ class RAMInputStream extends IndexInput 
   }
 
   private final void switchCurrentBuffer(boolean enforceEOF) throws IOException {
+    bufferStart = (long) BUFFER_SIZE * (long) currentBufferIndex;
     if (currentBufferIndex >= file.numBuffers()) {
       // end of file reached, no more buffers left
       if (enforceEOF)
@@ -95,7 +96,6 @@ class RAMInputStream extends IndexInput 
     } else {
       currentBuffer = file.getBuffer(currentBufferIndex);
       bufferPosition = 0;
-      bufferStart = (long) BUFFER_SIZE * (long) currentBufferIndex;
       long buflen = length - bufferStart;
       bufferLength = buflen > BUFFER_SIZE ? BUFFER_SIZE : (int) buflen;
     }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/ReaderUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/ReaderUtil.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/ReaderUtil.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/ReaderUtil.java Thu Jan 13 19:53:21 2011
@@ -22,6 +22,9 @@ import java.util.List;
 import java.io.IOException;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.CompositeReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 
 /**
  * Common util methods for dealing with {@link IndexReader}s.
@@ -148,7 +151,94 @@ public final class ReaderUtil {
         .toArray(new IndexReader[subReadersList.size()]);
     return subReaders[subIndex];
   }
+  
+  public static ReaderContext buildReaderContext(IndexReader reader) {
+    return new ReaderContextBuilder(reader).build();
+  }
+  
+  public static class ReaderContextBuilder {
+    private final IndexReader reader;
+    private final AtomicReaderContext[] leaves;
+    private int leafOrd = 0;
+    private int leafDocBase = 0;
+    public ReaderContextBuilder(IndexReader reader) {
+      this.reader = reader;
+      leaves = new AtomicReaderContext[numLeaves(reader)];
+    }
+    
+    public ReaderContext build() {
+      return build(null, reader, 0, 0);
+    }
+    
+    private ReaderContext build(CompositeReaderContext parent, IndexReader reader, int ord, int docBase) {
+      IndexReader[] sequentialSubReaders = reader.getSequentialSubReaders();
+      if (sequentialSubReaders == null) {
+        AtomicReaderContext atomic = new AtomicReaderContext(parent, reader, ord, docBase, leafOrd, leafDocBase);
+        leaves[leafOrd++] = atomic;
+        leafDocBase += reader.maxDoc();
+        return atomic;
+      } else {
+        ReaderContext[] children = new ReaderContext[sequentialSubReaders.length];
+        final CompositeReaderContext newParent;
+        if (parent == null) {
+          newParent = new CompositeReaderContext(reader, children, leaves);
+        } else {
+          newParent = new CompositeReaderContext(parent, reader, ord, docBase, children);
+        }
+        
+        int newDocBase = 0;
+        for (int i = 0; i < sequentialSubReaders.length; i++) {
+          build(newParent, sequentialSubReaders[i], i, newDocBase);
+          newDocBase += sequentialSubReaders[i].maxDoc();
+        }
+        return newParent;
+      }
+    }
+    
+    private int numLeaves(IndexReader reader) {
+      final int[] numLeaves = new int[1];
+      try {
+        new Gather(reader) {
+          @Override
+          protected void add(int base, IndexReader r) {
+            numLeaves[0]++;
+          }
+        }.run();
+      } catch (IOException ioe) {
+        // won't happen
+        throw new RuntimeException(ioe);
+      }
+      return numLeaves[0];
+    }
+    
+  }
 
+  /**
+   * Returns the context's leaves or the context itself as the only element of
+   * the returned array. If the context's #leaves() method returns
+   * <code>null</code> the given context must be an instance of
+   * {@link AtomicReaderContext}
+   */
+  public static AtomicReaderContext[] leaves(ReaderContext context) {
+    assert context != null && context.isTopLevel : "context must be non-null & top-level";
+    final AtomicReaderContext[] leaves = context.leaves();
+    if (leaves == null) {
+      assert context.isAtomic : "top-level context without leaves must be atomic";
+      return new AtomicReaderContext[] { (AtomicReaderContext) context };
+    }
+    return leaves;
+  }
+  
+  /**
+   * Walks up the reader tree and return the given context's top level reader
+   * context, or in other words the reader tree's root context.
+   */
+  public static ReaderContext getTopLevelContext(ReaderContext context) {
+    while (context.parent != null) {
+      context = context.parent;
+    }
+    return context;
+  }
 
   /**
    * Returns index of the searcher/reader for document <code>n</code> in the
@@ -175,4 +265,30 @@ public final class ReaderUtil {
     }
     return hi;
   }
+  
+  /**
+   * Returns index of the searcher/reader for document <code>n</code> in the
+   * array used to construct this searcher/reader.
+   */
+  public static int subIndex(int n, AtomicReaderContext[] leaves) { // find
+    // searcher/reader for doc n:
+    int size = leaves.length;
+    int lo = 0; // search starts array
+    int hi = size - 1; // for first element less than n, return its index
+    while (hi >= lo) {
+      int mid = (lo + hi) >>> 1;
+      int midValue = leaves[mid].docBase;
+      if (n < midValue)
+        hi = mid - 1;
+      else if (n > midValue)
+        lo = mid + 1;
+      else { // found a match
+        while (mid + 1 < size && leaves[mid + 1].docBase == midValue) {
+          mid++; // scan to last match
+        }
+        return mid;
+      }
+    }
+    return hi;
+  }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java Thu Jan 13 19:53:21 2011
@@ -256,7 +256,7 @@ public class Builder<T> {
   }
 
   public void add(IntsRef input, T output) throws IOException {
-    //System.out.println("\nADD: " + input.utf8ToString());
+    //System.out.println("\nFST ADD: input=" + input + " output=" + fst.outputs.outputToString(output));
     assert lastInput.length == 0 || input.compareTo(lastInput) > 0: "inputs are added out of order lastInput=" + lastInput + " vs input=" + input;
     assert validOutput(output);
 
@@ -361,7 +361,7 @@ public class Builder<T> {
     compilePrevTail(1);
     //System.out.println("finish: inputCount=" + frontier[0].inputCount);
     if (frontier[0].inputCount < minSuffixCount1 || frontier[0].inputCount < minSuffixCount2 || frontier[0].numArcs == 0) {
-      if (fst.getEmptyOutput() == null) {
+      if (fst.emptyOutput == null) {
         return null;
       } else if (minSuffixCount1 > 0 || minSuffixCount2 > 0) {
         // empty string got pruned

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java Thu Jan 13 19:53:21 2011
@@ -17,173 +17,31 @@ package org.apache.lucene.util.automaton
  * limitations under the License.
  */
 
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.RamUsageEstimator;
-
 import java.io.IOException;
 
+import org.apache.lucene.util.BytesRef;
+
 /** Can next() and advance() through the terms in an FST
   * @lucene.experimental
 */
 
-public class BytesRefFSTEnum<T> {
-  private final FST<T> fst;
-
-  private BytesRef current = new BytesRef(10);
-  @SuppressWarnings("unchecked") private FST.Arc<T>[] arcs = new FST.Arc[10];
-  // outputs are cumulative
-  @SuppressWarnings("unchecked") private T[] output = (T[]) new Object[10];
-
-  private boolean lastFinal;
-  private boolean didEmpty;
-  private final T NO_OUTPUT;
+public final class BytesRefFSTEnum<T> extends FSTEnum<T> {
+  private final BytesRef current = new BytesRef(10);
   private final InputOutput<T> result = new InputOutput<T>();
+  private BytesRef target;
 
   public static class InputOutput<T> {
     public BytesRef input;
     public T output;
   }
-  
+
+  /** doFloor controls the behavior of advance: if it's true
+   *  doFloor is true, advance positions to the biggest
+   *  term before target.  */
   public BytesRefFSTEnum(FST<T> fst) {
-    this.fst = fst;
+    super(fst);
     result.input = current;
-    NO_OUTPUT = fst.outputs.getNoOutput();
-  }
-
-  public void reset() {
-    lastFinal = false;
-    didEmpty = false;
-    current.length = 0;
-    result.output = NO_OUTPUT;
-  }
-
-  /** NOTE: target must be >= where we are already
-   *  positioned */
-  public InputOutput<T> advance(BytesRef target) throws IOException {
-
-    assert target.compareTo(current) >= 0;
-
-    //System.out.println("    advance len=" + target.length + " curlen=" + current.length);
-
-    // special case empty string
-    if (current.length == 0) {
-      if (target.length == 0) {
-        final T output = fst.getEmptyOutput();      
-        if (output != null) {
-          if (!didEmpty) {
-            current.length = 0;
-            lastFinal = true;
-            result.output = output;
-            didEmpty = true;
-          }
-          return result;
-        } else {
-          return next();
-        }
-      }
-      
-      if (fst.noNodes()) {
-        return null;
-      }
-    }
-
-    // TODO: possibly caller could/should provide common
-    // prefix length?  ie this work may be redundant if
-    // caller is in fact intersecting against its own
-    // automaton
-
-    // what prefix does target share w/ current
-    int idx = 0;
-    while (idx < current.length && idx < target.length) {
-      if (current.bytes[idx] != target.bytes[target.offset + idx]) {
-        break;
-      }
-      idx++;
-    }
-
-    //System.out.println("  shared " + idx);
-
-    FST.Arc<T> arc;
-    if (current.length == 0) {
-      // new enum (no seek/next yet)
-      arc = fst.readFirstArc(fst.getStartNode(), getArc(0));
-      //System.out.println("  new enum");
-    } else if (idx < current.length) {
-      // roll back to shared point
-      lastFinal = false;
-      current.length = idx;
-      arc = arcs[idx];
-      if (arc.isLast()) {
-        if (idx == 0) {
-          return null;
-        } else {
-          return next();
-        }
-      }
-      arc = fst.readNextArc(arc);
-    } else if (idx == target.length) {
-      // degenerate case -- seek to term we are already on
-      assert target.equals(current);
-      return result;
-    } else {
-      // current is a full prefix of target
-      if (lastFinal) {
-        arc = fst.readFirstArc(arcs[current.length-1].target, getArc(current.length));
-      } else {
-        return next();
-      }
-    }
-
-    lastFinal = false;
-
-    assert arc == arcs[current.length];
-    int targetLabel = target.bytes[target.offset+current.length] & 0xFF;
-
-    while(true) {
-      //System.out.println("    cycle len=" + current.length + " target=" + ((char) targetLabel) + " vs " + ((char) arc.label));
-      if (arc.label == targetLabel) {
-        grow();
-        current.bytes[current.length] = (byte) arc.label;
-        appendOutput(arc.output);
-        current.length++;
-        grow();
-        if (current.length == target.length) {
-          result.output = output[current.length-1];
-          if (arc.isFinal()) {
-            // target is exact match
-            if (fst.hasArcs(arc.target)) {
-              // target is also a proper prefix of other terms
-              lastFinal = true;
-              appendFinalOutput(arc.nextFinalOutput);
-            }
-          } else {
-            // target is not a match but is a prefix of
-            // other terms
-            current.length--;
-            push();
-          }
-          return result;
-        } else if (!fst.hasArcs(arc.target)) {
-          // we only match a prefix of the target
-          return next();
-        } else {
-          targetLabel = target.bytes[target.offset+current.length] & 0xFF;
-          arc = fst.readFirstArc(arc.target, getArc(current.length));
-        }
-      } else if (arc.label > targetLabel) {
-        // we are now past the target
-        push();
-        return result;
-      } else if (arc.isLast()) {
-        if (current.length == 0) {
-          return null;
-        }
-        return next();
-      } else {
-        arc = fst.readNextArc(getArc(current.length));
-      }
-    }
+    current.offset = 1;
   }
 
   public InputOutput<T> current() {
@@ -192,124 +50,58 @@ public class BytesRefFSTEnum<T> {
 
   public InputOutput<T> next() throws IOException {
     //System.out.println("  enum.next");
-
-    if (current.length == 0) {
-      final T output = fst.getEmptyOutput();
-      if (output != null) {
-        if (!didEmpty) {
-          current.length = 0;
-          lastFinal = true;
-          result.output = output;
-          didEmpty = true;
-          return result;
-        } else {
-          lastFinal = false;
-        }
-      }
-      if (fst.noNodes()) {
-        return null;
-      }
-      fst.readFirstArc(fst.getStartNode(), getArc(0));
-      push();
-    } else if (lastFinal) {
-      lastFinal = false;
-      assert current.length > 0;
-      // resume pushing
-      fst.readFirstArc(arcs[current.length-1].target, getArc(current.length));
-      push();
-    } else {
-      //System.out.println("    pop/push");
-      pop();
-      if (current.length == 0) {
-        // enum done
-        return null;
-      } else {
-        current.length--;
-        fst.readNextArc(arcs[current.length]);
-        push();
-      }
-    }
-
-    return result;
-  }
-
-  private void grow() {
-    final int l = current.length + 1;
-    current.grow(l);
-    if (arcs.length < l) {
-      @SuppressWarnings("unchecked") final FST.Arc<T>[] newArcs =
-        new FST.Arc[ArrayUtil.oversize(l, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      System.arraycopy(arcs, 0, newArcs, 0, arcs.length);
-      arcs = newArcs;
-    }
-    if (output.length < l) {
-      @SuppressWarnings("unchecked") final T[] newOutput =
-        (T[]) new Object[ArrayUtil.oversize(l, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      System.arraycopy(output, 0, newOutput, 0, output.length);
-      output = newOutput;
-    }
+    doNext();
+    return setResult();
   }
 
-  private void appendOutput(T addedOutput) {
-    T newOutput;
-    if (current.length == 0) {
-      newOutput = addedOutput;
-    } else if (addedOutput == NO_OUTPUT) {
-      output[current.length] = output[current.length-1];
-      return;
+  /** Seeks to smallest term that's >= target. */
+  public InputOutput<T> seekCeil(BytesRef target) throws IOException {
+    this.target = target;
+    targetLength = target.length;
+    super.doSeekCeil();
+    return setResult();
+  }
+
+  /** Seeks to biggest term that's <= target. */
+  public InputOutput<T> seekFloor(BytesRef target) throws IOException {
+    this.target = target;
+    targetLength = target.length;
+    super.doSeekFloor();
+    return setResult();
+  }
+
+  @Override
+  protected int getTargetLabel() {
+    if (upto-1 == target.length) {
+      return FST.END_LABEL;
     } else {
-      newOutput = fst.outputs.add(output[current.length-1], addedOutput);
+      return target.bytes[target.offset + upto - 1] & 0xFF;
     }
-    output[current.length] = newOutput;
   }
 
-  private void appendFinalOutput(T addedOutput) {
-    if (current.length == 0) {
-      result.output = addedOutput;
-    } else {
-      result.output = fst.outputs.add(output[current.length-1], addedOutput);
-    }
+  @Override
+  protected int getCurrentLabel() {
+    // current.offset fixed at 1
+    return current.bytes[upto] & 0xFF;
   }
 
-  private void push() throws IOException {
-
-    FST.Arc<T> arc = arcs[current.length];
-    assert arc != null;
-
-    while(true) {
-      grow();
-      
-      current.bytes[current.length] = (byte) arc.label;
-      appendOutput(arc.output);
-      //System.out.println("    push: append label=" + ((char) arc.label) + " output=" + fst.outputs.outputToString(arc.output));
-      current.length++;
-      grow();
-
-      if (!fst.hasArcs(arc.target)) {
-        break;
-      }
-
-      if (arc.isFinal()) {
-        appendFinalOutput(arc.nextFinalOutput);
-        lastFinal = true;
-        return;
-      }
-
-      arc = fst.readFirstArc(arc.target, getArc(current.length));
-    }
-    result.output = output[current.length-1];
+  @Override
+  protected void setCurrentLabel(int label) {
+    current.bytes[upto] = (byte) label;
   }
 
-  private void pop() {
-    while (current.length > 0 && arcs[current.length-1].isLast()) {
-      current.length--;
-    }
+  @Override
+  protected void grow() {
+    current.grow(upto+1);
   }
 
-  private FST.Arc<T> getArc(int idx) {
-    if (arcs[idx] == null) {
-      arcs[idx] = new FST.Arc<T>();
+  private InputOutput<T> setResult() {
+    if (upto == 0) {
+      return null;
+    } else {
+      current.length = upto-1;
+      result.output = output[upto];
+      return result;
     }
-    return arcs[idx];
   }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java Thu Jan 13 19:53:21 2011
@@ -18,20 +18,13 @@ package org.apache.lucene.util.automaton
  */
 
 import java.io.IOException;
-import java.io.PrintStream;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
 
 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.BytesRef;
 import org.apache.lucene.util.CodecUtil;
-import org.apache.lucene.util.IntsRef;
 
 /** Represents an FST using a compact byte[] format.
  *  <p> The format is similar to what's used by Morfologik
@@ -40,7 +33,7 @@ import org.apache.lucene.util.IntsRef;
  */
 public class FST<T> {
   public static enum INPUT_TYPE {BYTE1, BYTE2, BYTE4};
-  private final INPUT_TYPE inputType;
+  public final INPUT_TYPE inputType;
 
   private final static int BIT_FINAL_ARC = 1 << 0;
   private final static int BIT_LAST_ARC = 1 << 1;
@@ -76,7 +69,7 @@ public class FST<T> {
 
   // if non-null, this FST accepts the empty string and
   // produces this output
-  private T emptyOutput;
+  T emptyOutput;
   private byte[] emptyOutputBytes;
 
   private byte[] bytes;
@@ -94,11 +87,16 @@ public class FST<T> {
   public int arcCount;
   public int arcWithOutputCount;
 
+  // If arc has this label then that arc is final/accepted
+  public static int END_LABEL = -1;
+
   public final static class Arc<T> {
-    int label;  // really a "unsigned" byte
+    public int label;
+    public T output;
+
     int target;
+
     byte flags;
-    T output;
     T nextFinalOutput;
     int nextArc;
 
@@ -108,13 +106,26 @@ public class FST<T> {
     int arcIdx;
     int numArcs;
 
-    // Must call this before re-using an Arc instance on a
-    // new node
-    public void reset() {
-      bytesPerArc = 0;
+    /** Returns this */
+    public Arc<T> copyFrom(Arc<T> other) {
+      label = other.label;
+      target = other.target;
+      flags = other.flags;
+      output = other.output;
+      nextFinalOutput = other.nextFinalOutput;
+      nextArc = other.nextArc;
+      if (other.bytesPerArc != 0) {
+        bytesPerArc = other.bytesPerArc;
+        posArcsStart = other.posArcsStart;
+        arcIdx = other.arcIdx;
+        numArcs = other.numArcs;
+      } else {
+        bytesPerArc = 0;
+      }
+      return this;
     }
 
-    public boolean flag(int flag) {
+    boolean flag(int flag) {
       return FST.flag(flags, flag);
     }
 
@@ -122,7 +133,7 @@ public class FST<T> {
       return flag(BIT_LAST_ARC);
     }
 
-    public boolean isFinal() {
+    boolean isFinal() {
       return flag(BIT_FINAL_ARC);
     }
   };
@@ -156,7 +167,7 @@ public class FST<T> {
       // messy
       bytes = new byte[numBytes];
       in.readBytes(bytes, 0, numBytes);
-      emptyOutput = outputs.read(new BytesReader(numBytes-1));
+      emptyOutput = outputs.read(getBytesReader(numBytes-1));
     } else {
       emptyOutput = null;
     }
@@ -203,9 +214,9 @@ public class FST<T> {
     this.startNode = startNode;
   }
 
-  public void setEmptyOutput(T v) throws IOException {
-    if (emptyOutput != null) {
-      throw new IllegalStateException("empty output is already set");
+  void setEmptyOutput(T v) throws IOException {
+    if (emptyOutput != null && !emptyOutput.equals(v)) {
+      throw new IllegalStateException("empty output is already set: " + outputs.outputToString(emptyOutput) + " vs " + outputs.outputToString(v));
     }
     emptyOutput = v;
 
@@ -271,7 +282,7 @@ public class FST<T> {
     }
   }
 
-  private int readLabel(DataInput in) throws IOException {
+  int readLabel(DataInput in) throws IOException {
     final int v;
     if (inputType == INPUT_TYPE.BYTE1) {
       v = in.readByte()&0xFF;
@@ -285,21 +296,8 @@ public class FST<T> {
 
   // returns true if the node at this address has any
   // outgoing arcs
-  public boolean hasArcs(int address) {
-    return address != FINAL_END_NODE && address != NON_FINAL_END_NODE;
-  }
-
-  public int getStartNode() {
-    if (startNode == -1) {
-      throw new IllegalStateException("call finish first");
-    }
-    return startNode;
-  }
-
-  // returns null if this FST does not accept the empty
-  // string, else, the output for the empty string
-  public T getEmptyOutput() {
-    return emptyOutput;
+  public boolean targetHasArcs(Arc<T> arc) {
+    return arc.target > 0;
   }
 
   // serializes new node by appending its bytes to the end
@@ -364,7 +362,7 @@ public class FST<T> {
         assert arc.nextFinalOutput == NO_OUTPUT;
       }
 
-      boolean targetHasArcs = hasArcs(target.address);
+      boolean targetHasArcs = target.address > 0;
 
       if (!targetHasArcs) {
         flags += BIT_STOP_NODE;
@@ -453,10 +451,49 @@ public class FST<T> {
     return endAddress-1;
   }
 
-  public Arc<T> readFirstArc(int address, Arc<T> arc) throws IOException {
-    //System.out.println("readFirstArc addr=" + address);
+  /** Fills virtual 'start' arc, ie, an empty incoming arc to
+   *  the FST's start node */
+  public Arc<T> getFirstArc(Arc<T> arc) {
+    if (emptyOutput != null) {
+      arc.flags = BIT_FINAL_ARC | BIT_LAST_ARC;
+      arc.nextFinalOutput = emptyOutput;
+    } else {
+      arc.flags = BIT_LAST_ARC;
+    }
+
+    // If there are no nodes, ie, the FST only accepts the
+    // empty string, then startNode is 0, and then readFirstTargetArc
+    arc.target = startNode;
+    return arc;
+  }
+
+  /** Follow the follow arc and read the first arc of its
+   *  target; this changes the provide arc (2nd arg) in-place
+   *  and returns it. */
+  public Arc<T> readFirstTargetArc(Arc<T> follow, Arc<T> arc) throws IOException {
     //int pos = address;
-    final BytesReader in = new BytesReader(address);
+    //System.out.println("    readFirstTarget follow.target=" + follow.target + " isFinal=" + follow.isFinal());
+    if (follow.isFinal()) {
+      // Insert "fake" final first arc:
+      arc.label = -1;
+      arc.output = follow.nextFinalOutput;
+      if (follow.target <= 0) {
+        arc.flags = BIT_LAST_ARC;
+      } else {
+        arc.flags = 0;
+        arc.nextArc = follow.target;
+      }
+      //System.out.println("    insert isFinal; nextArc=" + follow.target + " isLast=" + arc.isLast() + " output=" + outputs.outputToString(arc.output));
+      return arc;
+    } else {
+      return readFirstRealArc(follow.target, arc);
+    }
+  }
+
+  // Not private beacaus NodeHash needs access:
+  Arc<T> readFirstRealArc(int address, Arc<T> arc) throws IOException {
+
+    final BytesReader in = getBytesReader(address);
 
     arc.flags = in.readByte();
 
@@ -473,19 +510,66 @@ public class FST<T> {
       arc.bytesPerArc = 0;
     }
     arc.nextArc = in.pos;
+    arc.label = 0;
     return readNextArc(arc);
   }
 
+  /** In-place read; returns the arc. */
   public Arc<T> readNextArc(Arc<T> arc) throws IOException {
+    if (arc.label == -1) {
+      // This was a fake inserted "final" arc
+      if (arc.nextArc <= 0) {
+        // This arc went to virtual final node, ie has no outgoing arcs
+        return null;
+      }
+      return readFirstRealArc(arc.nextArc, arc);
+    } else {
+      return readNextRealArc(arc);
+    }
+  }
+
+  /** Peeks at next arc's label; does not alter arc.  Do
+   *  not call this if arc.isLast()! */
+  public int readNextArcLabel(Arc<T> arc) throws IOException {
+    assert !arc.isLast();
+
+    final BytesReader in;
+    if (arc.label == END_LABEL) {
+      //System.out.println("    nextArc fake " + arc.nextArc);
+      in = getBytesReader(arc.nextArc);
+      byte flags = bytes[in.pos];
+      if (flag(flags, BIT_ARCS_AS_FIXED_ARRAY)) {
+        //System.out.println("    nextArc fake array");
+        in.pos--;
+        in.readVInt();
+        in.readByte();
+      }
+    } else {
+      if (arc.bytesPerArc != 0) {
+        //System.out.println("    nextArc real array");
+        // arcs are at fixed entries
+        in = getBytesReader(arc.posArcsStart - (1+arc.arcIdx)*arc.bytesPerArc);
+      } else {
+        // arcs are packed
+        //System.out.println("    nextArc real packed");
+        in = getBytesReader(arc.nextArc);
+      }
+    }
+    // skip flags
+    in.readByte();
+    return readLabel(in);
+  }
+
+  Arc<T> readNextRealArc(Arc<T> arc) throws IOException {
     // this is a continuing arc in a fixed array
     final BytesReader in;
     if (arc.bytesPerArc != 0) {
       // arcs are at fixed entries
       arc.arcIdx++;
-      in = new BytesReader(arc.posArcsStart - arc.arcIdx*arc.bytesPerArc);
+      in = getBytesReader(arc.posArcsStart - arc.arcIdx*arc.bytesPerArc);
     } else {
       // arcs are packed
-      in = new BytesReader(arc.nextArc);
+      in = getBytesReader(arc.nextArc);
     }
     arc.flags = in.readByte();
     arc.label = readLabel(in);
@@ -504,6 +588,7 @@ public class FST<T> {
 
     if (arc.flag(BIT_STOP_NODE)) {
       arc.target = FINAL_END_NODE;
+      arc.flags |= BIT_FINAL_ARC;
       arc.nextArc = in.pos;
     } else if (arc.flag(BIT_TARGET_NEXT)) {
       arc.nextArc = in.pos;
@@ -524,14 +609,30 @@ public class FST<T> {
     return arc;
   }
 
-  public Arc<T> findArc(int address, int labelToMatch, Arc<T> arc) throws IOException {
+  /** Finds an arc leaving the incoming arc, replacing the arc in place.
+   *  This returns null if the arc was not found, else the incoming arc. */
+  public Arc<T> findTargetArc(int labelToMatch, Arc<T> follow, Arc<T> arc) throws IOException {
+
+    if (labelToMatch == END_LABEL) {
+      if (follow.isFinal()) {
+        arc.output = follow.nextFinalOutput;
+        arc.label = END_LABEL;
+        return arc;
+      } else {
+        return null;
+      }
+    }
+
+    if (!targetHasArcs(follow)) {
+      return null;
+    }
+
     // TODO: maybe make an explicit thread state that holds
     // reusable stuff eg BytesReader:
-    final BytesReader in = new BytesReader(address);
+    final BytesReader in = getBytesReader(follow.target);
 
     if ((in.readByte() & BIT_ARCS_AS_FIXED_ARRAY) != 0) {
       // Arcs are full array; do binary search:
-      //System.out.println("findArc: array label=" + labelToMatch);
       arc.numArcs = in.readVInt();
       arc.bytesPerArc = in.readByte() & 0xFF;
       arc.posArcsStart = in.pos;
@@ -548,19 +649,20 @@ public class FST<T> {
           high = mid - 1;
         else {
           arc.arcIdx = mid-1;
-          return readNextArc(arc);
+          return readNextRealArc(arc);
         }
       }
 
       return null;
     }
-    //System.out.println("findArc: scan");
-
-    readFirstArc(address, arc);
 
+    // Linear scan
+    readFirstTargetArc(follow, arc);
     while(true) {
       if (arc.label == labelToMatch) {
         return arc;
+      } else if (arc.label > labelToMatch) {
+        return null;
       } else if (arc.isLast()) {
         return null;
       } else {
@@ -569,191 +671,6 @@ public class FST<T> {
     }
   }
 
-  /** Looks up the output for this input, or null if the
-   *  input is not accepted. FST must be
-   *  INPUT_TYPE.BYTE4. */
-  public T get(IntsRef input) throws IOException {
-    assert inputType == INPUT_TYPE.BYTE4;
-
-    if (input.length == 0) {
-      return getEmptyOutput();
-    }
-
-    // TODO: would be nice not to alloc this on every lookup
-    final FST.Arc<T> arc = new FST.Arc<T>();
-    int node = getStartNode();
-    T output = NO_OUTPUT;
-    for(int i=0;i<input.length;i++) {
-      if (!hasArcs(node)) {
-        // hit end of FST before input end
-        return null;
-      }
-
-      if (findArc(node, input.ints[input.offset + i], arc) != null) {
-        node = arc.target;
-        if (arc.output != NO_OUTPUT) {
-          output = outputs.add(output, arc.output);
-        }
-      } else {
-        return null;
-      }
-    }
-
-    if (!arc.isFinal()) {
-      // hit input's end before end node
-      return null;
-    }
-
-    if (arc.nextFinalOutput != NO_OUTPUT) {
-      output = outputs.add(output, arc.nextFinalOutput);
-    }
-
-    return output;
-  }
-
-  /** Logically casts input to UTF32 ints then looks up the output
-   *  or null if the input is not accepted.  FST must be
-   *  INPUT_TYPE.BYTE4.  */
-  public T get(char[] input, int offset, int length) throws IOException {
-    assert inputType == INPUT_TYPE.BYTE4;
-
-    if (length == 0) {
-      return getEmptyOutput();
-    }
-
-    // TODO: would be nice not to alloc this on every lookup
-    final FST.Arc<T> arc = new FST.Arc<T>();
-    int node = getStartNode();
-    int charIdx = offset;
-    final int charLimit = offset + length;
-    T output = NO_OUTPUT;
-    while(charIdx < charLimit) {
-      if (!hasArcs(node)) {
-        // hit end of FST before input end
-        return null;
-      }
-
-      final int utf32 = Character.codePointAt(input, charIdx);
-      charIdx += Character.charCount(utf32);
-
-      if (findArc(node, utf32, arc) != null) {
-        node = arc.target;
-        if (arc.output != NO_OUTPUT) {
-          output = outputs.add(output, arc.output);
-        }
-      } else {
-        return null;
-      }
-    }
-
-    if (!arc.isFinal()) {
-      // hit input's end before end node
-      return null;
-    }
-
-    if (arc.nextFinalOutput != NO_OUTPUT) {
-      output = outputs.add(output, arc.nextFinalOutput);
-    }
-
-    return output;
-  }
-
-
-  /** Logically casts input to UTF32 ints then looks up the output
-   *  or null if the input is not accepted.  FST must be
-   *  INPUT_TYPE.BYTE4.  */
-  public T get(CharSequence input) throws IOException {
-    assert inputType == INPUT_TYPE.BYTE4;
-
-    final int len = input.length();
-    if (len == 0) {
-      return getEmptyOutput();
-    }
-
-    // TODO: would be nice not to alloc this on every lookup
-    final FST.Arc<T> arc = new FST.Arc<T>();
-    int node = getStartNode();
-    int charIdx = 0;
-    final int charLimit = input.length();
-    T output = NO_OUTPUT;
-    while(charIdx < charLimit) {
-      if (!hasArcs(node)) {
-        // hit end of FST before input end
-        return null;
-      }
-
-      final int utf32 = Character.codePointAt(input, charIdx);
-      charIdx += Character.charCount(utf32);
-
-      if (findArc(node, utf32, arc) != null) {
-        node = arc.target;
-        if (arc.output != NO_OUTPUT) {
-          output = outputs.add(output, arc.output);
-        }
-      } else {
-        return null;
-      }
-    }
-
-    if (!arc.isFinal()) {
-      // hit input's end before end node
-      return null;
-    }
-
-    if (arc.nextFinalOutput != NO_OUTPUT) {
-      output = outputs.add(output, arc.nextFinalOutput);
-    }
-
-    return output;
-  }
-
-  /** Looks up the output for this input, or null if the
-   *  input is not accepted */
-  public T get(BytesRef input) throws IOException {
-    assert inputType == INPUT_TYPE.BYTE1;
-
-    if (input.length == 0) {
-      return getEmptyOutput();
-    }
-
-    // TODO: would be nice not to alloc this on every lookup
-    final FST.Arc<T> arc = new FST.Arc<T>();
-    int node = getStartNode();
-    T output = NO_OUTPUT;
-    for(int i=0;i<input.length;i++) {
-      if (!hasArcs(node)) {
-        // hit end of FST before input end
-        return null;
-      }
-
-      if (findArc(node, input.bytes[i+input.offset], arc) != null) {
-        node = arc.target;
-        if (arc.output != NO_OUTPUT) {
-          output = outputs.add(output, arc.output);
-        }
-      } else {
-        return null;
-      }
-    }
-
-    if (!arc.isFinal()) {
-      // hit input's end before end node
-      return null;
-    }
-
-    if (arc.nextFinalOutput != NO_OUTPUT) {
-      output = outputs.add(output, arc.nextFinalOutput);
-    }
-
-    return output;
-  }
-
-  /** Returns true if this FST has no nodes */
-  public boolean noNodes() {
-    //System.out.println("isempty startNode=" + startNode);
-    return startNode == 0;
-  }
-
   private void seekToNextNode(BytesReader in) throws IOException {
 
     while(true) {
@@ -779,85 +696,6 @@ public class FST<T> {
     }
   }
 
-  // NOTE: this consumes alot of RAM!
-  // final arcs have a flat end (not arrow)
-  // arcs w/ NEXT opto are in blue
-  /*
-    eg:
-      PrintStream ps = new PrintStream("out.dot");
-      fst.toDot(ps);
-      ps.close();
-      System.out.println("SAVED out.dot");
-      
-    then dot -Tpng out.dot > /x/tmp/out.png
-  */
-  public void toDot(PrintStream out) throws IOException {
-
-    final List<Integer> queue = new ArrayList<Integer>();
-    queue.add(startNode);
-
-    final Set<Integer> seen = new HashSet<Integer>();
-    seen.add(startNode);
-    
-    out.println("digraph FST {");
-    out.println("  rankdir = LR;");
-    //out.println("  " + startNode + " [shape=circle label=" + startNode + "];");
-    out.println("  " + startNode + " [label=\"\" shape=circle];");
-    out.println("  initial [shape=point color=white label=\"\"];");
-    if (emptyOutput != null) {
-      out.println("  initial -> " + startNode + " [arrowhead=tee label=\"(" + outputs.outputToString(emptyOutput) + ")\"];");
-    } else {
-      out.println("  initial -> " + startNode);
-    }
-
-    final Arc<T> arc = new Arc<T>();
-
-    while(queue.size() != 0) {
-      Integer node = queue.get(queue.size()-1);
-      queue.remove(queue.size()-1);
-
-      if (node == FINAL_END_NODE || node == NON_FINAL_END_NODE) {
-        continue;
-      }
-
-      // scan all arcs
-      readFirstArc(node, arc);
-      while(true) {
-
-        if (!seen.contains(arc.target)) {
-          //out.println("  " + arc.target + " [label=" + arc.target + "];");
-          out.println("  " + arc.target + " [label=\"\" shape=circle];");
-          seen.add(arc.target);
-          queue.add(arc.target);
-        }
-        String outs;
-        if (arc.output != NO_OUTPUT) {
-          outs = "/" + outputs.outputToString(arc.output);
-        } else {
-          outs = "";
-        }
-        if (arc.isFinal() && arc.nextFinalOutput != NO_OUTPUT) {
-          outs += " (" + outputs.outputToString(arc.nextFinalOutput) + ")";
-        }
-        out.print("  " + node + " -> " + arc.target + " [label=\"" + arc.label + outs + "\"");
-        if (arc.isFinal()) {
-          out.print(" arrowhead=tee");
-        }
-        if (arc.flag(BIT_TARGET_NEXT)) {
-          out.print(" color=blue");
-        }
-        out.println("];");
-        
-        if (arc.isLast()) {
-          break;
-        } else {
-          readNextArc(arc);
-        }
-      }
-    }
-    out.println("}");
-  }
-  
   public int getNodeCount() {
     // 1+ in order to count the -1 implicit final node
     return 1+nodeCount;
@@ -872,7 +710,7 @@ public class FST<T> {
   }
 
   // Non-static: writes to FST's byte[]
-  private class BytesWriter extends DataOutput {
+  class BytesWriter extends DataOutput {
     int posWrite;
 
     public BytesWriter() {
@@ -899,8 +737,13 @@ public class FST<T> {
     }
   }
 
+  final BytesReader getBytesReader(int pos) {
+    // TODO: maybe re-use via ThreadLocal?
+    return new BytesReader(pos);
+  }
+
   // Non-static: reads byte[] from FST
-  private class BytesReader extends DataInput {
+  class BytesReader extends DataInput {
     int pos;
 
     public BytesReader(int pos) {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/IntsRefFSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/IntsRefFSTEnum.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/IntsRefFSTEnum.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/IntsRefFSTEnum.java Thu Jan 13 19:53:21 2011
@@ -17,9 +17,7 @@ package org.apache.lucene.util.automaton
  * limitations under the License.
  */
 
-import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IntsRef;
-import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.IOException;
 
@@ -27,163 +25,23 @@ import java.io.IOException;
   * @lucene.experimental
 */
 
-public class IntsRefFSTEnum<T> {
-  private final FST<T> fst;
-
-  private IntsRef current = new IntsRef(10);
-  @SuppressWarnings("unchecked") private FST.Arc<T>[] arcs = new FST.Arc[10];
-  // outputs are cumulative
-  @SuppressWarnings("unchecked") private T[] output = (T[]) new Object[10];
-
-  private boolean lastFinal;
-  private boolean didEmpty;
-  private final T NO_OUTPUT;
+public final class IntsRefFSTEnum<T> extends FSTEnum<T> {
+  private final IntsRef current = new IntsRef(10);
   private final InputOutput<T> result = new InputOutput<T>();
+  private IntsRef target;
 
   public static class InputOutput<T> {
     public IntsRef input;
     public T output;
   }
-  
+
+  /** doFloor controls the behavior of advance: if it's true
+   *  doFloor is true, advance positions to the biggest
+   *  term before target.  */
   public IntsRefFSTEnum(FST<T> fst) {
-    this.fst = fst;
+    super(fst);
     result.input = current;
-    NO_OUTPUT = fst.outputs.getNoOutput();
-  }
-
-  public void reset() {
-    lastFinal = false;
-    didEmpty = false;
-    current.length = 0;
-    result.output = NO_OUTPUT;
-  }
-
-  /** NOTE: target must be >= where we are already
-   *  positioned */
-  public InputOutput<T> advance(IntsRef target) throws IOException {
-
-    assert target.compareTo(current) >= 0;
-
-    //System.out.println("    advance len=" + target.length + " curlen=" + current.length);
-
-    // special case empty string
-    if (current.length == 0) {
-      if (target.length == 0) {
-        final T output = fst.getEmptyOutput();      
-        if (output != null) {
-          if (!didEmpty) {
-            current.length = 0;
-            lastFinal = true;
-            result.output = output;
-            didEmpty = true;
-          }
-          return result;
-        } else {
-          return next();
-        }
-      }
-      
-      if (fst.noNodes()) {
-        return null;
-      }
-    }
-
-    // TODO: possibly caller could/should provide common
-    // prefix length?  ie this work may be redundant if
-    // caller is in fact intersecting against its own
-    // automaton
-
-    // what prefix does target share w/ current
-    int idx = 0;
-    while (idx < current.length && idx < target.length) {
-      if (current.ints[idx] != target.ints[target.offset + idx]) {
-        break;
-      }
-      idx++;
-    }
-
-    //System.out.println("  shared " + idx);
-
-    FST.Arc<T> arc;
-    if (current.length == 0) {
-      // new enum (no seek/next yet)
-      arc = fst.readFirstArc(fst.getStartNode(), getArc(0));
-      //System.out.println("  new enum");
-    } else if (idx < current.length) {
-      // roll back to shared point
-      lastFinal = false;
-      current.length = idx;
-      arc = arcs[idx];
-      if (arc.isLast()) {
-        if (idx == 0) {
-          return null;
-        } else {
-          return next();
-        }
-      }
-      arc = fst.readNextArc(arc);
-    } else if (idx == target.length) {
-      // degenerate case -- seek to term we are already on
-      assert target.equals(current);
-      return result;
-    } else {
-      // current is a full prefix of target
-      if (lastFinal) {
-        arc = fst.readFirstArc(arcs[current.length-1].target, getArc(current.length));
-      } else {
-        return next();
-      }
-    }
-
-    lastFinal = false;
-
-    assert arc == arcs[current.length];
-    int targetLabel = target.ints[target.offset+current.length];
-
-    while(true) {
-      //System.out.println("    cycle len=" + current.length + " target=" + ((char) targetLabel) + " vs " + ((char) arc.label));
-      if (arc.label == targetLabel) {
-        grow();
-        current.ints[current.length] = arc.label;
-        appendOutput(arc.output);
-        current.length++;
-        grow();
-        if (current.length == target.length) {
-          result.output = output[current.length-1];
-          if (arc.isFinal()) {
-            // target is exact match
-            if (fst.hasArcs(arc.target)) {
-              // target is also a proper prefix of other terms
-              lastFinal = true;
-              appendFinalOutput(arc.nextFinalOutput);
-            }
-          } else {
-            // target is not a match but is a prefix of
-            // other terms
-            current.length--;
-            push();
-          }
-          return result;
-        } else if (!fst.hasArcs(arc.target)) {
-          // we only match a prefix of the target
-          return next();
-        } else {
-          targetLabel = target.ints[target.offset+current.length];
-          arc = fst.readFirstArc(arc.target, getArc(current.length));
-        }
-      } else if (arc.label > targetLabel) {
-        // we are now past the target
-        push();
-        return result;
-      } else if (arc.isLast()) {
-        if (current.length == 0) {
-          return null;
-        }
-        return next();
-      } else {
-        arc = fst.readNextArc(getArc(current.length));
-      }
-    }
+    current.offset = 1;
   }
 
   public InputOutput<T> current() {
@@ -192,124 +50,58 @@ public class IntsRefFSTEnum<T> {
 
   public InputOutput<T> next() throws IOException {
     //System.out.println("  enum.next");
-
-    if (current.length == 0) {
-      final T output = fst.getEmptyOutput();
-      if (output != null) {
-        if (!didEmpty) {
-          current.length = 0;
-          lastFinal = true;
-          result.output = output;
-          didEmpty = true;
-          return result;
-        } else {
-          lastFinal = false;
-        }
-      }
-      if (fst.noNodes()) {
-        return null;
-      }
-      fst.readFirstArc(fst.getStartNode(), getArc(0));
-      push();
-    } else if (lastFinal) {
-      lastFinal = false;
-      assert current.length > 0;
-      // resume pushing
-      fst.readFirstArc(arcs[current.length-1].target, getArc(current.length));
-      push();
-    } else {
-      //System.out.println("    pop/push");
-      pop();
-      if (current.length == 0) {
-        // enum done
-        return null;
-      } else {
-        current.length--;
-        fst.readNextArc(arcs[current.length]);
-        push();
-      }
-    }
-
-    return result;
+    doNext();
+    return setResult();
   }
 
-  private void grow() {
-    final int l = current.length + 1;
-    current.grow(l);
-    if (arcs.length < l) {
-      @SuppressWarnings("unchecked") final FST.Arc<T>[] newArcs =
-        new FST.Arc[ArrayUtil.oversize(l, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      System.arraycopy(arcs, 0, newArcs, 0, arcs.length);
-      arcs = newArcs;
-    }
-    if (output.length < l) {
-      @SuppressWarnings("unchecked") final T[] newOutput =
-        (T[]) new Object[ArrayUtil.oversize(l, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      System.arraycopy(output, 0, newOutput, 0, output.length);
-      output = newOutput;
-    }
-  }
-
-  private void appendOutput(T addedOutput) {
-    T newOutput;
-    if (current.length == 0) {
-      newOutput = addedOutput;
-    } else if (addedOutput == NO_OUTPUT) {
-      output[current.length] = output[current.length-1];
-      return;
+  /** Seeks to smallest term that's >= target. */
+  public InputOutput<T> seekCeil(IntsRef target) throws IOException {
+    this.target = target;
+    targetLength = target.length;
+    super.doSeekCeil();
+    return setResult();
+  }
+
+  /** Seeks to biggest term that's <= target. */
+  public InputOutput<T> seekFloor(IntsRef target) throws IOException {
+    this.target = target;
+    targetLength = target.length;
+    super.doSeekFloor();
+    return setResult();
+  }
+
+  @Override
+  protected int getTargetLabel() {
+    if (upto-1 == target.length) {
+      return FST.END_LABEL;
     } else {
-      newOutput = fst.outputs.add(output[current.length-1], addedOutput);
+      return target.ints[target.offset + upto - 1];
     }
-    output[current.length] = newOutput;
   }
 
-  private void appendFinalOutput(T addedOutput) {
-    if (current.length == 0) {
-      result.output = addedOutput;
-    } else {
-      result.output = fst.outputs.add(output[current.length-1], addedOutput);
-    }
+  @Override
+  protected int getCurrentLabel() {
+    // current.offset fixed at 1
+    return current.ints[upto];
   }
 
-  private void push() throws IOException {
-
-    FST.Arc<T> arc = arcs[current.length];
-    assert arc != null;
-
-    while(true) {
-      grow();
-      
-      current.ints[current.length] = arc.label;
-      appendOutput(arc.output);
-      //System.out.println("    push: append label=" + ((char) arc.label) + " output=" + fst.outputs.outputToString(arc.output));
-      current.length++;
-      grow();
-
-      if (!fst.hasArcs(arc.target)) {
-        break;
-      }
-
-      if (arc.isFinal()) {
-        appendFinalOutput(arc.nextFinalOutput);
-        lastFinal = true;
-        return;
-      }
-
-      arc = fst.readFirstArc(arc.target, getArc(current.length));
-    }
-    result.output = output[current.length-1];
+  @Override
+  protected void setCurrentLabel(int label) {
+    current.ints[upto] = label;
   }
 
-  private void pop() {
-    while (current.length > 0 && arcs[current.length-1].isLast()) {
-      current.length--;
-    }
+  @Override
+  protected void grow() {
+    current.grow(upto+1);
   }
 
-  private FST.Arc<T> getArc(int idx) {
-    if (arcs[idx] == null) {
-      arcs[idx] = new FST.Arc<T>();
+  private InputOutput<T> setResult() {
+    if (upto == 0) {
+      return null;
+    } else {
+      current.length = upto-1;
+      result.output = output[upto];
+      return result;
     }
-    return arcs[idx];
   }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java Thu Jan 13 19:53:21 2011
@@ -35,7 +35,7 @@ final class NodeHash<T> {
   }
 
   private boolean nodesEqual(Builder.UnCompiledNode<T> node, int address) throws IOException {
-    fst.readFirstArc(address, scratchArc);
+    fst.readFirstRealArc(address, scratchArc);
     if (scratchArc.bytesPerArc != 0 && node.numArcs != scratchArc.numArcs) {
       return false;
     }
@@ -56,7 +56,7 @@ final class NodeHash<T> {
           return false;
         }
       }
-      fst.readNextArc(scratchArc);
+      fst.readNextRealArc(scratchArc);
     }
 
     return false;
@@ -89,7 +89,7 @@ final class NodeHash<T> {
     final int PRIME = 31;
     //System.out.println("hash frozen");
     int h = 0;
-    fst.readFirstArc(node, scratchArc);
+    fst.readFirstRealArc(node, scratchArc);
     while(true) {
       //System.out.println("  label=" + scratchArc.label + " target=" + scratchArc.target + " h=" + h + " output=" + fst.outputs.outputToString(scratchArc.output) + " next?=" + scratchArc.flag(4) + " final?=" + scratchArc.isFinal());
       h = PRIME * h + scratchArc.label;
@@ -102,7 +102,7 @@ final class NodeHash<T> {
       if (scratchArc.isLast()) {
         break;
       }
-      fst.readNextArc(scratchArc);
+      fst.readNextRealArc(scratchArc);
     }
     //System.out.println("  ret " + (h&Integer.MAX_VALUE));
     return h & Integer.MAX_VALUE;

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestExternalCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestExternalCodecs.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestExternalCodecs.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestExternalCodecs.java Thu Jan 13 19:53:21 2011
@@ -18,6 +18,7 @@ package org.apache.lucene;
  */
 
 import org.apache.lucene.util.*;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.index.*;
 import org.apache.lucene.document.*;
 import org.apache.lucene.search.*;
@@ -330,10 +331,6 @@ public class TestExternalCodecs extends 
       }
 
       @Override
-      public void cacheCurrentTerm() {
-      }
-
-      @Override
       public DocsEnum docs(Bits skipDocs, DocsEnum reuse) {
         return new RAMDocsEnum(ramField.termToDocs.get(current), skipDocs);
       }

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearch.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearch.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearch.java Thu Jan 13 19:53:21 2011
@@ -94,7 +94,7 @@ public class TestSearch extends LuceneTe
       }
       writer.close();
 
-      Searcher searcher = new IndexSearcher(directory, true);
+      IndexSearcher searcher = new IndexSearcher(directory, true);
 
       String[] queries = {
         "a b",

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/TestSearchForDuplicates.java Thu Jan 13 19:53:21 2011
@@ -102,7 +102,7 @@ public class TestSearchForDuplicates ext
       writer.close();
 
       // try a search without OR
-      Searcher searcher = new IndexSearcher(directory, true);
+      IndexSearcher searcher = new IndexSearcher(directory, true);
 
       QueryParser parser = new QueryParser(TEST_VERSION_CURRENT, PRIORITY_FIELD, analyzer);
 
@@ -133,7 +133,7 @@ public class TestSearchForDuplicates ext
   }
 
 
-  private void printHits(PrintWriter out, ScoreDoc[] hits, Searcher searcher ) throws IOException {
+  private void printHits(PrintWriter out, ScoreDoc[] hits, IndexSearcher searcher) throws IOException {
     out.println(hits.length + " total results\n");
     for (int i = 0 ; i < hits.length; i++) {
       if ( i < 10 || (i > 94 && i < 105) ) {
@@ -143,11 +143,11 @@ public class TestSearchForDuplicates ext
     }
   }
 
-  private void checkHits(ScoreDoc[] hits, int expectedCount, Searcher searcher) throws IOException {
+  private void checkHits(ScoreDoc[] hits, int expectedCount, IndexSearcher searcher) throws IOException {
     assertEquals("total results", expectedCount, hits.length);
     for (int i = 0 ; i < hits.length; i++) {
       if ( i < 10 || (i > 94 && i < 105) ) {
-      Document d = searcher.doc(hits[i].doc);
+        Document d = searcher.doc(hits[i].doc);
         assertEquals("check " + i, String.valueOf(i), d.get(ID_FIELD));
       }
     }

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestDocument.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestDocument.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestDocument.java Thu Jan 13 19:53:21 2011
@@ -6,7 +6,6 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
-import org.apache.lucene.search.Searcher;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
@@ -157,7 +156,7 @@ public class TestDocument extends Lucene
     writer.addDocument(makeDocumentWithFields());
     IndexReader reader = writer.getReader();
     
-    Searcher searcher = new IndexSearcher(reader);
+    IndexSearcher searcher = new IndexSearcher(reader);
     
     // search for something that does exists
     Query query = new TermQuery(new Term("keyword", "test1"));
@@ -239,7 +238,7 @@ public class TestDocument extends Lucene
     writer.addDocument(doc);
     
     IndexReader reader = writer.getReader();
-    Searcher searcher = new IndexSearcher(reader);
+    IndexSearcher searcher = new IndexSearcher(reader);
     
     Query query = new TermQuery(new Term("keyword", "test"));
     

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java Thu Jan 13 19:53:21 2011
@@ -87,6 +87,7 @@ public class RandomIndexWriter implement
     if (LuceneTestCase.VERBOSE) {
       System.out.println("RIW config=" + w.getConfig());
       System.out.println("codec default=" + w.getConfig().getCodecProvider().getDefaultFieldCodec());
+      w.setInfoStream(System.out);
     }
   } 
 

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/Test2BTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/Test2BTerms.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/Test2BTerms.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/Test2BTerms.java Thu Jan 13 19:53:21 2011
@@ -143,6 +143,12 @@ public class Test2BTerms extends LuceneT
             setMergePolicy(newLogMergePolicy(false, 10))
     );
 
+    MergePolicy mp = w.getConfig().getMergePolicy();
+    if (mp instanceof LogByteSizeMergePolicy) {
+      // 1 petabyte:
+      ((LogByteSizeMergePolicy) mp).setMaxMergeMB(1024*1024*1024);
+    }
+
     Document doc = new Document();
     Field field = new Field("field", new MyTokenStream(TERMS_PER_DOC));
     field.setOmitTermFreqAndPositions(true);

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Thu Jan 13 19:53:21 2011
@@ -43,6 +43,7 @@ import org.apache.lucene.search.FieldCac
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.NumericRangeQuery;
 import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.Similarity;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
@@ -216,6 +217,9 @@ public class TestBackwardsCompatibility 
   
   public void testOptimizeOldIndex() throws Exception {
     for(int i=0;i<oldNames.length;i++) {
+      if (VERBOSE) {
+        System.out.println("\nTEST: index=" + oldNames[i]);
+      }
       unzip(getDataFile("index." + oldNames[i] + ".zip"), oldNames[i]);
 
       String fullPath = fullDir(oldNames[i]);
@@ -223,6 +227,7 @@ public class TestBackwardsCompatibility 
 
       IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(
           TEST_VERSION_CURRENT, new MockAnalyzer()));
+      w.setInfoStream(VERBOSE ? System.out : null);
       w.optimize();
       w.close();
 
@@ -360,7 +365,7 @@ public class TestBackwardsCompatibility 
 
     // First document should be #21 since it's norm was
     // increased:
-    Document d = searcher.doc(hits[0].doc);
+    Document d = searcher.getIndexReader().document(hits[0].doc);
     assertEquals("didn't get the right document first", "21", d.get("id"));
 
     doTestHits(hits, 34, searcher.getIndexReader());
@@ -408,7 +413,7 @@ public class TestBackwardsCompatibility 
     // make sure searching sees right # hits
     IndexSearcher searcher = new IndexSearcher(dir, true);
     ScoreDoc[] hits = searcher.search(new TermQuery(new Term("content", "aaa")), null, 1000).scoreDocs;
-    Document d = searcher.doc(hits[0].doc);
+    Document d = searcher.getIndexReader().document(hits[0].doc);
     assertEquals("wrong first document", "21", d.get("id"));
     doTestHits(hits, 44, searcher.getIndexReader());
     searcher.close();
@@ -419,7 +424,7 @@ public class TestBackwardsCompatibility 
     Term searchTerm = new Term("id", "6");
     int delCount = reader.deleteDocuments(searchTerm);
     assertEquals("wrong delete count", 1, delCount);
-    reader.setNorm(searcher.search(new TermQuery(new Term("id", "22")), 10).scoreDocs[0].doc, "content", (float) 2.0);
+    reader.setNorm(searcher.search(new TermQuery(new Term("id", "22")), 10).scoreDocs[0].doc, "content", Similarity.getDefault().encodeNormValue(2.0f));
     reader.close();
     searcher.close();
 
@@ -467,7 +472,7 @@ public class TestBackwardsCompatibility 
     Term searchTerm = new Term("id", "6");
     int delCount = reader.deleteDocuments(searchTerm);
     assertEquals("wrong delete count", 1, delCount);
-    reader.setNorm(22, "content", (float) 2.0);
+    reader.setNorm(22, "content", Similarity.getDefault().encodeNormValue(2.0f));
     reader.close();
 
     // make sure they "took":
@@ -526,7 +531,7 @@ public class TestBackwardsCompatibility 
     assertEquals("didn't delete the right number of documents", 1, delCount);
 
     // Set one norm so we get a .s0 file:
-    reader.setNorm(21, "content", (float) 1.5);
+    reader.setNorm(21, "content", Similarity.getDefault().encodeNormValue(1.5f));
     reader.close();
   }
 
@@ -563,7 +568,7 @@ public class TestBackwardsCompatibility 
       assertEquals("didn't delete the right number of documents", 1, delCount);
 
       // Set one norm so we get a .s0 file:
-      reader.setNorm(21, "content", (float) 1.5);
+      reader.setNorm(21, "content", Similarity.getDefault().encodeNormValue(1.5f));
       reader.close();
 
       // The numbering of fields can vary depending on which