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

svn commit: r1058390 [6/16] - in /lucene/dev/branches/bulkpostings: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/ dev-tools/idea/lucene/contrib/ dev-tools/idea/lucene/contr...

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java Thu Jan 13 02:09:33 2011
@@ -19,8 +19,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 
 /** 
  * Constrains search results to only match those which also match a provided
@@ -48,12 +47,15 @@ public class QueryWrapperFilter extends 
   }
 
   @Override
-  public DocIdSet getDocIdSet(final IndexReader reader) throws IOException {
-    final Weight weight = query.weight(new IndexSearcher(reader));
+  public DocIdSet getDocIdSet(final AtomicReaderContext context) throws IOException {
+    // get a private context that is used to rewrite, createWeight and score eventually
+    assert context.reader.getTopReaderContext().isAtomic;
+    final AtomicReaderContext privateContext = (AtomicReaderContext) context.reader.getTopReaderContext();
+    final Weight weight = query.weight(new IndexSearcher(privateContext));
     return new DocIdSet() {
       @Override
       public DocIdSetIterator iterator() throws IOException {
-        return weight.scorer(reader, true, false);
+        return weight.scorer(privateContext, true, false);
       }
       @Override
       public boolean isCacheable() { return false; }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java Thu Jan 13 02:09:33 2011
@@ -43,7 +43,7 @@ public class ScoreCachingWrappingScorer 
   }
 
   @Override
-  protected boolean score(Collector collector, int max, int firstDocID) throws IOException {
+  public boolean score(Collector collector, int max, int firstDocID) throws IOException {
     return scorer.score(collector, max, firstDocID);
   }
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ScoreDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ScoreDoc.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ScoreDoc.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ScoreDoc.java Thu Jan 13 02:09:33 2011
@@ -24,7 +24,7 @@ public class ScoreDoc implements java.io
   public float score;
 
   /** Expert: A hit document's number.
-   * @see Searcher#doc(int)
+   * @see IndexSearcher#doc(int)
    */
   public int doc;
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Scorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Scorer.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Scorer.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Scorer.java Thu Jan 13 02:09:33 2011
@@ -90,7 +90,7 @@ public abstract class Scorer extends Doc
    *          this method.
    * @return true if more matching documents may remain.
    */
-  protected boolean score(Collector collector, int max, int firstDocID) throws IOException {
+  public boolean score(Collector collector, int max, int firstDocID) throws IOException {
     collector.setScorer(this);
     int doc = firstDocID;
     while (doc < max) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java Thu Jan 13 02:09:33 2011
@@ -89,7 +89,7 @@ public abstract class ScoringRewrite<Q e
       if (bq.clauses().isEmpty())
         return bq;
       // strip the scores off
-      final Query result = new ConstantScoreQuery(new QueryWrapperFilter(bq));
+      final Query result = new ConstantScoreQuery(bq);
       result.setBoost(query.getBoost());
       return result;
     }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Similarity.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Similarity.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Similarity.java Thu Jan 13 02:09:33 2011
@@ -462,12 +462,14 @@ import org.apache.lucene.util.SmallFloat
  *        {@link org.apache.lucene.document.Fieldable#setBoost(float) field.setBoost()}
  *        before adding the field to a document.
  *        </li>
- *        <li>{@link #lengthNorm(String, int) <b>lengthNorm</b>(field)} - computed
+ *        <li><b>lengthNorm</b> - computed
  *        when the document is added to the index in accordance with the number of tokens
  *        of this field in the document, so that shorter fields contribute more to the score.
  *        LengthNorm is computed by the Similarity class in effect at indexing.
  *        </li>
  *      </ul>
+ *      The {@link #computeNorm} method is responsible for
+ *      combining all of these factors into a single float.
  *
  *      <p>
  *      When a document is added to the index, all the above factors are multiplied.
@@ -480,7 +482,7 @@ import org.apache.lucene.util.SmallFloat
  *            norm(t,d) &nbsp; = &nbsp;
  *            {@link org.apache.lucene.document.Document#getBoost() doc.getBoost()}
  *            &nbsp;&middot;&nbsp;
- *            {@link #lengthNorm(String, int) lengthNorm(field)}
+ *            lengthNorm
  *            &nbsp;&middot;&nbsp;
  *          </td>
  *          <td valign="bottom" align="center" rowspan="1">
@@ -521,8 +523,8 @@ import org.apache.lucene.util.SmallFloat
  * </ol>
  *
  * @see #setDefault(Similarity)
- * @see org.apache.lucene.index.IndexWriter#setSimilarity(Similarity)
- * @see Searcher#setSimilarity(Similarity)
+ * @see org.apache.lucene.index.IndexWriterConfig#setSimilarity(Similarity)
+ * @see IndexSearcher#setSimilarity(Similarity)
  */
 public abstract class Similarity implements Serializable {
   
@@ -535,8 +537,8 @@ public abstract class Similarity impleme
   /** Set the default Similarity implementation used by indexing and search
    * code.
    *
-   * @see Searcher#setSimilarity(Similarity)
-   * @see org.apache.lucene.index.IndexWriter#setSimilarity(Similarity)
+   * @see IndexSearcher#setSimilarity(Similarity)
+   * @see org.apache.lucene.index.IndexWriterConfig#setSimilarity(Similarity)
    */
   public static void setDefault(Similarity similarity) {
     Similarity.defaultImpl = similarity;
@@ -547,8 +549,8 @@ public abstract class Similarity impleme
    *
    * <p>This is initially an instance of {@link DefaultSimilarity}.
    *
-   * @see Searcher#setSimilarity(Similarity)
-   * @see org.apache.lucene.index.IndexWriter#setSimilarity(Similarity)
+   * @see IndexSearcher#setSimilarity(Similarity)
+   * @see org.apache.lucene.index.IndexWriterConfig#setSimilarity(Similarity)
    */
   public static Similarity getDefault() {
     return Similarity.defaultImpl;
@@ -563,6 +565,11 @@ public abstract class Similarity impleme
   }
 
   /** Decodes a normalization factor stored in an index.
+   * <p>
+   * <b>WARNING: If you override this method, you should change the default
+   *    Similarity to your implementation with {@link Similarity#setDefault(Similarity)}. 
+   *    Otherwise, your method may not always be called, especially if you omit norms 
+   *    for some fields.</b>
    * @see #encodeNormValue(float)
    */
   public float decodeNormValue(byte b) {
@@ -570,12 +577,23 @@ public abstract class Similarity impleme
   }
 
   /**
-   * Compute the normalization value for a field, given the accumulated
+   * Computes the normalization value for a field, given the accumulated
    * state of term processing for this field (see {@link FieldInvertState}).
    * 
    * <p>Implementations should calculate a float value based on the field
    * state and then return that value.
    *
+   * <p>Matches in longer fields are less precise, so implementations of this
+   * method usually return smaller values when <code>state.getLength()</code> is large,
+   * and larger values when <code>state.getLength()</code> is small.
+   * 
+   * <p>Note that the return values are computed under 
+   * {@link org.apache.lucene.index.IndexWriter#addDocument(org.apache.lucene.document.Document)} 
+   * and then stored using
+   * {@link #encodeNormValue(float)}.  
+   * Thus they have limited precision, and documents
+   * must be re-indexed if this method is altered.
+   *
    * <p>For backward compatibility this method by default calls
    * {@link #lengthNorm(String, int)} passing
    * {@link FieldInvertState#getLength()} as the second argument, and
@@ -587,9 +605,7 @@ public abstract class Similarity impleme
    * @param state current processing state for this field
    * @return the calculated float norm
    */
-  public float computeNorm(String field, FieldInvertState state) {
-    return (state.getBoost() * lengthNorm(field, state.getLength()));
-  }
+  public abstract float computeNorm(String field, FieldInvertState state);
   
   /** Computes the normalization value for a field given the total number of
    * terms contained in a field.  These values, together with field boosts, are
@@ -613,8 +629,13 @@ public abstract class Similarity impleme
    * @return a normalization factor for hits on this field of this document
    *
    * @see org.apache.lucene.document.Field#setBoost(float)
+   *
+   * @deprecated Please override computeNorm instead
    */
-  public abstract float lengthNorm(String fieldName, int numTokens);
+  @Deprecated
+  public final float lengthNorm(String fieldName, int numTokens) {
+    throw new UnsupportedOperationException("please use computeNorm instead");
+  }
 
   /** Computes the normalization value for a query given the sum of the squared
    * weights of each of the query terms.  This value is multiplied into the
@@ -641,7 +662,11 @@ public abstract class Similarity impleme
    * are rounded down to the largest representable value.  Positive values too
    * small to represent are rounded up to the smallest positive representable
    * value.
-   *
+   * <p>
+   * <b>WARNING: If you override this method, you should change the default
+   * Similarity to your implementation with {@link Similarity#setDefault(Similarity)}. 
+   * Otherwise, your method may not always be called, especially if you omit norms 
+   * for some fields.</b>
    * @see org.apache.lucene.document.Field#setBoost(float)
    * @see org.apache.lucene.util.SmallFloat
    */
@@ -709,11 +734,11 @@ public abstract class Similarity impleme
    * idf(docFreq, searcher.maxDoc());
    * </pre>
    * 
-   * Note that {@link Searcher#maxDoc()} is used instead of
+   * Note that {@link IndexSearcher#maxDoc()} is used instead of
    * {@link org.apache.lucene.index.IndexReader#numDocs() IndexReader#numDocs()} because also 
-   * {@link Searcher#docFreq(Term)} is used, and when the latter 
-   * is inaccurate, so is {@link Searcher#maxDoc()}, and in the same direction.
-   * In addition, {@link Searcher#maxDoc()} is more efficient to compute
+   * {@link IndexSearcher#docFreq(Term)} is used, and when the latter 
+   * is inaccurate, so is {@link IndexSearcher#maxDoc()}, and in the same direction.
+   * In addition, {@link IndexSearcher#maxDoc()} is more efficient to compute
    *   
    * @param term the term in question
    * @param searcher the document collection being searched
@@ -722,7 +747,7 @@ public abstract class Similarity impleme
              and an explanation for the term.
    * @throws IOException
    */
-  public IDFExplanation idfExplain(final Term term, final Searcher searcher, int docFreq) throws IOException {
+  public IDFExplanation idfExplain(final Term term, final IndexSearcher searcher, int docFreq) throws IOException {
     final int df = docFreq;
     final int max = searcher.maxDoc();
     final float idf = idf(df, max);
@@ -736,16 +761,16 @@ public abstract class Similarity impleme
         public float getIdf() {
           return idf;
         }};
-   }
+  }
 
   /**
    * This method forwards to {@link
-   * #idfExplain(Term,Searcher,int)} by passing
+   * #idfExplain(Term,IndexSearcher,int)} by passing
    * <code>searcher.docFreq(term)</code> as the docFreq.
    */
-  public IDFExplanation idfExplain(final Term term, final Searcher searcher) throws IOException {
+  public IDFExplanation idfExplain(final Term term, final IndexSearcher searcher) throws IOException {
     return idfExplain(term, searcher, searcher.docFreq(term));
-   }
+  }
 
   /**
    * Computes a score factor for a phrase.
@@ -761,7 +786,7 @@ public abstract class Similarity impleme
    *         for each term.
    * @throws IOException
    */
-  public IDFExplanation idfExplain(Collection<Term> terms, Searcher searcher) throws IOException {
+  public IDFExplanation idfExplain(Collection<Term> terms, IndexSearcher searcher) throws IOException {
     final int max = searcher.maxDoc();
     float idf = 0.0f;
     final StringBuilder exp = new StringBuilder();

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Sort.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Sort.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Sort.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Sort.java Thu Jan 13 02:09:33 2011
@@ -103,7 +103,7 @@ implements Serializable {
   /**
    * Represents sorting by computed relevance. Using this sort criteria returns
    * the same results as calling
-   * {@link Searcher#search(Query,int) Searcher#search()}without a sort criteria,
+   * {@link IndexSearcher#search(Query,int) IndexSearcher#search()}without a sort criteria,
    * only with slightly more overhead.
    */
   public static final Sort RELEVANCE = new Sort();
@@ -116,7 +116,7 @@ implements Serializable {
 
   /**
    * Sorts by computed relevance. This is the same sort criteria as calling
-   * {@link Searcher#search(Query,int) Searcher#search()}without a sort criteria,
+   * {@link IndexSearcher#search(Query,int) IndexSearcher#search()}without a sort criteria,
    * only with slightly more overhead.
    */
   public Sort() {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java Thu Jan 13 02:09:33 2011
@@ -17,6 +17,7 @@ package org.apache.lucene.search;
 
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.Spans;
 import org.apache.lucene.util.OpenBitSet;
@@ -52,8 +53,8 @@ public class SpanQueryFilter extends Spa
   }
 
   @Override
-  public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
-    SpanFilterResult result = bitSpans(reader);
+  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
+    SpanFilterResult result = bitSpans(context.reader);
     return result.getDocIdSet();
   }
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermQuery.java Thu Jan 13 02:09:33 2011
@@ -22,8 +22,10 @@ import java.util.Set;
 
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.BulkPostingsEnum;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.index.BulkPostingsEnum.BlockReader;
 import org.apache.lucene.search.Explanation.IDFExplanation;
 import org.apache.lucene.util.ToStringUtils;
@@ -42,9 +44,11 @@ public class TermQuery extends Query {
     private float queryNorm;
     private float queryWeight;
     private IDFExplanation idfExp;
+    private transient ReaderContext weightContext; // only set if -ea for assert in scorer()
 
-    public TermWeight(Searcher searcher)
+    public TermWeight(IndexSearcher searcher)
       throws IOException {
+      assert setWeightContext(searcher);
       this.similarity = getSimilarity(searcher);
       if (docFreq != -1) {
         idfExp = similarity.idfExplain(term, searcher, docFreq);
@@ -77,7 +81,9 @@ public class TermQuery extends Query {
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+      assert assertTopReaderContext(context);
+      final IndexReader reader = context.reader;
       assert reader.getSequentialSubReaders() == null;
       BulkPostingsEnum docs = reader.bulkTermPostingsEnum(term.field(),
                                                           term.bytes(),
@@ -98,10 +104,23 @@ public class TermQuery extends Query {
             reader.getDeletedDocs(), similarity, reader.norms(term.field()));
       }
     }
+    
+    private boolean assertTopReaderContext(ReaderContext context) {
+      while (context.parent != null) {
+        context = context.parent;
+      }
+      return weightContext == context;
+    }
+    
+    private boolean setWeightContext(IndexSearcher searcher) {
+      weightContext = searcher.getTopReaderContext();
+      return true;
+    }
 
     @Override
-    public Explanation explain(IndexReader reader, int doc)
+    public Explanation explain(AtomicReaderContext context, int doc)
       throws IOException {
+      final IndexReader reader = context.reader;
 
       ComplexExplanation result = new ComplexExplanation();
       result.setDescription("weight("+getQuery()+" in "+doc+"), product of:");
@@ -192,7 +211,7 @@ public class TermQuery extends Query {
   public Term getTerm() { return term; }
 
   @Override
-  public Weight createWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
     return new TermWeight(searcher);
   }
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermScorer.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermScorer.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermScorer.java Thu Jan 13 02:09:33 2011
@@ -88,7 +88,7 @@ final class TermScorer extends Scorer {
 
   // firstDocID is ignored since nextDoc() sets 'doc'
   @Override
-  protected boolean score(Collector c, int end, int firstDocID) throws IOException {
+  public boolean score(Collector c, int end, int firstDocID) throws IOException {
     c.setScorer(this);
     // nocommit -- this can leave scorer on a deleted doc...
     while (doc < end) {                           // for docs in window

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TopDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TopDocs.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TopDocs.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TopDocs.java Thu Jan 13 02:09:33 2011
@@ -18,8 +18,8 @@ package org.apache.lucene.search;
  */
 
 /** Represents hits returned by {@link
- * Searcher#search(Query,Filter,int)} and {@link
- * Searcher#search(Query,int)}. */
+ * IndexSearcher#search(Query,Filter,int)} and {@link
+ * IndexSearcher#search(Query,int)}. */
 public class TopDocs implements java.io.Serializable {
   /** The total number of hits for the query.
   */

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TopFieldDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TopFieldDocs.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TopFieldDocs.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TopFieldDocs.java Thu Jan 13 02:09:33 2011
@@ -19,7 +19,7 @@ package org.apache.lucene.search;
 
 
 /** Represents hits returned by {@link
- * Searcher#search(Query,Filter,int,Sort)}.
+ * IndexSearcher#search(Query,Filter,int,Sort)}.
  */
 public class TopFieldDocs
 extends TopDocs {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Weight.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Weight.java Thu Jan 13 02:09:33 2011
@@ -21,27 +21,35 @@ import java.io.IOException;
 import java.io.Serializable;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 
 /**
  * Expert: Calculate query weights and build query scorers.
  * <p>
- * The purpose of {@link Weight} is to ensure searching does not
- * modify a {@link Query}, so that a {@link Query} instance can be reused. <br>
- * {@link Searcher} dependent state of the query should reside in the
+ * The purpose of {@link Weight} is to ensure searching does not modify a
+ * {@link Query}, so that a {@link Query} instance can be reused. <br>
+ * {@link IndexSearcher} dependent state of the query should reside in the
  * {@link Weight}. <br>
  * {@link IndexReader} dependent state should reside in the {@link Scorer}.
  * <p>
+ * Since {@link Weight} creates {@link Scorer} instances for a given
+ * {@link AtomicReaderContext} ({@link #scorer(AtomicReaderContext, boolean, boolean)})
+ * callers must maintain the relationship between the searcher's top-level
+ * {@link ReaderContext} and the context used to create a {@link Scorer}. 
+ * <p>
  * A <code>Weight</code> is used in the following way:
  * <ol>
  * <li>A <code>Weight</code> is constructed by a top-level query, given a
- * <code>Searcher</code> ({@link Query#createWeight(Searcher)}).
+ * <code>IndexSearcher</code> ({@link Query#createWeight(IndexSearcher)}).
  * <li>The {@link #sumOfSquaredWeights()} method is called on the
  * <code>Weight</code> to compute the query normalization factor
  * {@link Similarity#queryNorm(float)} of the query clauses contained in the
  * query.
  * <li>The query normalization factor is passed to {@link #normalize(float)}. At
  * this point the weighting is complete.
- * <li>A <code>Scorer</code> is constructed by {@link #scorer(IndexReader,boolean,boolean)}.
+ * <li>A <code>Scorer</code> is constructed by
+ * {@link #scorer(AtomicReaderContext,boolean,boolean)}.
  * </ol>
  * 
  * @since 2.9
@@ -51,12 +59,12 @@ public abstract class Weight implements 
   /**
    * An explanation of the score computation for the named document.
    * 
-   * @param reader sub-reader containing the give doc
-   * @param doc
+   * @param context the readers context to create the {@link Explanation} for.
+   * @param doc the document's id relative to the given context's reader
    * @return an Explanation for the score
-   * @throws IOException
+   * @throws IOException if an {@link IOException} occurs
    */
-  public abstract Explanation explain(IndexReader reader, int doc) throws IOException;
+  public abstract Explanation explain(AtomicReaderContext context, int doc) throws IOException;
 
   /** The query that this concerns. */
   public abstract Query getQuery();
@@ -79,8 +87,8 @@ public abstract class Weight implements 
    * <b>NOTE:</b> null can be returned if no documents will be scored by this
    * query.
    * 
-   * @param reader
-   *          the {@link IndexReader} for which to return the {@link Scorer}.
+   * @param context
+   *          the {@link AtomicReaderContext} for which to return the {@link Scorer}.
    * @param scoreDocsInOrder
    *          specifies whether in-order scoring of documents is required. Note
    *          that if set to false (i.e., out-of-order scoring is required),
@@ -96,7 +104,7 @@ public abstract class Weight implements 
    * @return a {@link Scorer} which scores documents in/out-of order.
    * @throws IOException
    */
-  public abstract Scorer scorer(IndexReader reader, boolean scoreDocsInOrder,
+  public abstract Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
       boolean topScorer) throws IOException;
   
   /** The sum of squared weights of contained query clauses. */
@@ -106,7 +114,7 @@ public abstract class Weight implements 
    * Returns true iff this implementation scores docs only out of order. This
    * method is used in conjunction with {@link Collector}'s
    * {@link Collector#acceptsDocsOutOfOrder() acceptsDocsOutOfOrder} and
-   * {@link #scorer(org.apache.lucene.index.IndexReader, boolean, boolean)} to
+   * {@link #scorer(AtomicReaderContext, boolean, boolean)} to
    * create a matching {@link Scorer} instance for a given {@link Collector}, or
    * vice versa.
    * <p>

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/ByteValuesCreator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/ByteValuesCreator.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/ByteValuesCreator.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/ByteValuesCreator.java Thu Jan 13 02:09:33 2011
@@ -74,7 +74,7 @@ public class ByteValuesCreator extends C
   }
 
   @Override
-  public ByteValues validate(ByteValues entry, IndexReader reader) throws IOException {
+  public synchronized ByteValues validate(ByteValues entry, IndexReader reader) throws IOException {
     boolean ok = false;
     
     if( hasOption(OPTION_CACHE_VALUES) ) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/DocTermsIndexCreator.java Thu Jan 13 02:09:33 2011
@@ -240,8 +240,28 @@ public class DocTermsIndexCreator extend
 
       @Override
       public SeekStatus seek(BytesRef text, boolean useCache) throws IOException {
-        // TODO - we can support with binary search
-        throw new UnsupportedOperationException();
+        int low = 1;
+        int high = numOrd-1;
+        
+        while (low <= high) {
+          int mid = (low + high) >>> 1;
+          seek(mid);
+          int cmp = term.compareTo(text);
+
+          if (cmp < 0)
+            low = mid + 1;
+          else if (cmp > 0)
+            high = mid - 1;
+          else
+            return SeekStatus.FOUND; // key found
+        }
+        
+        if (low == numOrd) {
+          return SeekStatus.END;
+        } else {
+          seek(low);
+          return SeekStatus.NOT_FOUND;
+        }
       }
 
       @Override
@@ -315,7 +335,7 @@ public class DocTermsIndexCreator extend
 
       @Override
       public Comparator<BytesRef> getComparator() throws IOException {
-        throw new UnsupportedOperationException();
+        return BytesRef.getUTF8SortedAsUnicodeComparator();
       }
     }
   }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/DoubleValuesCreator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/DoubleValuesCreator.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/DoubleValuesCreator.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/DoubleValuesCreator.java Thu Jan 13 02:09:33 2011
@@ -74,7 +74,7 @@ public class DoubleValuesCreator extends
   }
 
   @Override
-  public DoubleValues validate(DoubleValues entry, IndexReader reader) throws IOException {
+  public synchronized DoubleValues validate(DoubleValues entry, IndexReader reader) throws IOException {
     boolean ok = false;
     
     if( hasOption(OPTION_CACHE_VALUES) ) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/FloatValuesCreator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/FloatValuesCreator.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/FloatValuesCreator.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/FloatValuesCreator.java Thu Jan 13 02:09:33 2011
@@ -75,7 +75,7 @@ public class FloatValuesCreator extends 
   }
 
   @Override
-  public FloatValues validate(FloatValues entry, IndexReader reader) throws IOException {
+  public synchronized FloatValues validate(FloatValues entry, IndexReader reader) throws IOException {
     boolean ok = false;
     
     if( hasOption(OPTION_CACHE_VALUES) ) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/IntValuesCreator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/IntValuesCreator.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/IntValuesCreator.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/IntValuesCreator.java Thu Jan 13 02:09:33 2011
@@ -75,7 +75,7 @@ public class IntValuesCreator extends Ca
   }
 
   @Override
-  public IntValues validate(IntValues entry, IndexReader reader) throws IOException {
+  public synchronized IntValues validate(IntValues entry, IndexReader reader) throws IOException {
     boolean ok = false;
     
     if( hasOption(OPTION_CACHE_VALUES) ) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/LongValuesCreator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/LongValuesCreator.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/LongValuesCreator.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/LongValuesCreator.java Thu Jan 13 02:09:33 2011
@@ -75,7 +75,7 @@ public class LongValuesCreator extends C
   }
 
   @Override
-  public LongValues validate(LongValues entry, IndexReader reader) throws IOException {
+  public synchronized LongValues validate(LongValues entry, IndexReader reader) throws IOException {
     boolean ok = false;
     
     if( hasOption(OPTION_CACHE_VALUES) ) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/ShortValuesCreator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/ShortValuesCreator.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/ShortValuesCreator.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/cache/ShortValuesCreator.java Thu Jan 13 02:09:33 2011
@@ -75,7 +75,7 @@ public class ShortValuesCreator extends 
   }
 
   @Override
-  public ShortValues validate(ShortValues entry, IndexReader reader) throws IOException {
+  public synchronized ShortValues validate(ShortValues entry, IndexReader reader) throws IOException {
     boolean ok = false;
     
     if( hasOption(OPTION_CACHE_VALUES) ) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/function/CustomScoreQuery.java Thu Jan 13 02:09:33 2011
@@ -22,13 +22,14 @@ import java.util.Set;
 import java.util.Arrays;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.ComplexExplanation;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Weight;
 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.util.ToStringUtils;
 
@@ -187,7 +188,7 @@ public class CustomScoreQuery extends Qu
     Weight[] valSrcWeights;
     boolean qStrict;
 
-    public CustomWeight(Searcher searcher) throws IOException {
+    public CustomWeight(IndexSearcher searcher) throws IOException {
       this.similarity = getSimilarity(searcher);
       this.subQueryWeight = subQuery.weight(searcher);
       this.valSrcWeights = new Weight[valSrcQueries.length];
@@ -239,40 +240,40 @@ public class CustomScoreQuery extends Qu
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
       // Pass true for "scoresDocsInOrder", because we
       // require in-order scoring, even if caller does not,
       // since we call advance on the valSrcScorers.  Pass
       // false for "topScorer" because we will not invoke
       // score(Collector) on these scorers:
-      Scorer subQueryScorer = subQueryWeight.scorer(reader, true, false);
+      Scorer subQueryScorer = subQueryWeight.scorer(context, true, false);
       if (subQueryScorer == null) {
         return null;
       }
       Scorer[] valSrcScorers = new Scorer[valSrcWeights.length];
       for(int i = 0; i < valSrcScorers.length; i++) {
-         valSrcScorers[i] = valSrcWeights[i].scorer(reader, true, topScorer);
+         valSrcScorers[i] = valSrcWeights[i].scorer(context, true, topScorer);
       }
-      return new CustomScorer(similarity, reader, this, subQueryScorer, valSrcScorers);
+      return new CustomScorer(similarity, context.reader, this, subQueryScorer, valSrcScorers);
     }
 
     @Override
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
-      Explanation explain = doExplain(reader, doc);
+    public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
+      Explanation explain = doExplain(context, doc);
       return explain == null ? new Explanation(0.0f, "no matching docs") : explain;
     }
     
-    private Explanation doExplain(IndexReader reader, int doc) throws IOException {
-      Explanation subQueryExpl = subQueryWeight.explain(reader, doc);
+    private Explanation doExplain(AtomicReaderContext info, int doc) throws IOException {
+      Explanation subQueryExpl = subQueryWeight.explain(info, doc);
       if (!subQueryExpl.isMatch()) {
         return subQueryExpl;
       }
       // match
       Explanation[] valSrcExpls = new Explanation[valSrcWeights.length];
       for(int i = 0; i < valSrcWeights.length; i++) {
-        valSrcExpls[i] = valSrcWeights[i].explain(reader, doc);
+        valSrcExpls[i] = valSrcWeights[i].explain(info, doc);
       }
-      Explanation customExp = CustomScoreQuery.this.getCustomScoreProvider(reader).customExplain(doc,subQueryExpl,valSrcExpls);
+      Explanation customExp = CustomScoreQuery.this.getCustomScoreProvider(info.reader).customExplain(doc,subQueryExpl,valSrcExpls);
       float sc = getValue() * customExp.getValue();
       Explanation res = new ComplexExplanation(
         true, sc, CustomScoreQuery.this.toString() + ", product of:");
@@ -350,7 +351,7 @@ public class CustomScoreQuery extends Qu
   }
 
   @Override
-  public Weight createWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
     return new CustomWeight(searcher);
   }
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/function/ValueSourceQuery.java Thu Jan 13 02:09:33 2011
@@ -19,8 +19,8 @@ package org.apache.lucene.search.functio
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.*;
+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 +68,7 @@ public class ValueSourceQuery extends Qu
     float queryNorm;
     float queryWeight;
 
-    public ValueSourceWeight(Searcher searcher) {
+    public ValueSourceWeight(IndexSearcher searcher) {
       this.similarity = getSimilarity(searcher);
     }
 
@@ -99,14 +99,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, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+      return new ValueSourceScorer(similarity, context.reader, 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.reader);
       float sc = queryWeight * vals.floatVal(doc);
 
       Explanation result = new ComplexExplanation(
@@ -138,7 +138,7 @@ public class ValueSourceQuery extends Qu
       qWeight = w.getValue();
       // this is when/where the values are first created.
       vals = valSrc.getValues(reader);
-      delDocs = MultiFields.getDeletedDocs(reader);
+      delDocs = reader.getDeletedDocs();
       maxDoc = reader.maxDoc();
     }
 
@@ -173,7 +173,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/bulkpostings/lucene/src/java/org/apache/lucene/search/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/package.html?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/package.html (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/package.html Thu Jan 13 02:09:33 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/bulkpostings/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java Thu Jan 13 02:09:33 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,16 @@ 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,
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
         boolean topScorer) throws IOException {
-      return new PayloadNearSpanScorer(query.getSpans(reader), this,
-          similarity, reader.norms(query.getField()));
+      return new PayloadNearSpanScorer(query.getSpans(context.reader), this,
+          similarity, context.reader.norms(query.getField()));
     }
   }
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java Thu Jan 13 02:09:33 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,22 @@ 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,
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
         boolean topScorer) throws IOException {
-      return new PayloadTermSpanScorer((TermSpans) query.getSpans(reader),
-          this, similarity, reader.norms(query.getField()));
+      return new PayloadTermSpanScorer((TermSpans) query.getSpans(context.reader),
+          this, similarity, context.reader.norms(query.getField()));
     }
 
     protected class PayloadTermSpanScorer extends SpanScorer {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java Thu Jan 13 02:09:33 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/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java Thu Jan 13 02:09:33 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/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java Thu Jan 13 02:09:33 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, boolean scoreDocsInOrder, boolean topScorer) 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, true, false)).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/bulkpostings/lucene/src/java/org/apache/lucene/store/RAMInputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/RAMInputStream.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/RAMInputStream.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/RAMInputStream.java Thu Jan 13 02:09:33 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/bulkpostings/lucene/src/java/org/apache/lucene/util/ReaderUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/ReaderUtil.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/ReaderUtil.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/ReaderUtil.java Thu Jan 13 02:09:33 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,83 @@ 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;
+  }
 
   /**
    * Returns index of the searcher/reader for document <code>n</code> in the
@@ -175,4 +254,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/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java Thu Jan 13 02:09:33 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/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java Thu Jan 13 02:09:33 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];
   }
 }