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/10/01 05:05:07 UTC

svn commit: r1177888 [5/16] - in /lucene/dev/branches/lucene2621: ./ dev-tools/eclipse/ dev-tools/idea/lucene/contrib/ dev-tools/maven/ lucene/ lucene/contrib/ lucene/contrib/demo/src/java/org/apache/lucene/demo/ lucene/contrib/demo/src/java/org/apache...

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java Sat Oct  1 03:04:53 2011
@@ -164,10 +164,11 @@ public class MultiPhraseQuery extends Qu
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+        boolean topScorer, Bits acceptDocs) throws IOException {
       assert !termArrays.isEmpty();
       final IndexReader reader = context.reader;
-      final Bits liveDocs = reader.getLiveDocs();
+      final Bits liveDocs = acceptDocs;
       
       PhraseQuery.PostingsAndFreq[] postingsFreqs = new PhraseQuery.PostingsAndFreq[termArrays.size()];
 
@@ -227,7 +228,7 @@ public class MultiPhraseQuery extends Qu
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      Scorer scorer = scorer(context, ScorerContext.def());
+      Scorer scorer = scorer(context, true, false, context.reader.getLiveDocs());
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/PhraseQuery.java Sat Oct  1 03:04:53 2011
@@ -212,10 +212,11 @@ public class PhraseQuery extends Query {
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+        boolean topScorer, Bits acceptDocs) throws IOException {
       assert !terms.isEmpty();
       final IndexReader reader = context.reader;
-      final Bits liveDocs = reader.getLiveDocs();
+      final Bits liveDocs = acceptDocs;
       PostingsAndFreq[] postingsFreqs = new PostingsAndFreq[terms.size()];
       for (int i = 0; i < terms.size(); i++) {
         final Term t = terms.get(i);
@@ -267,7 +268,7 @@ public class PhraseQuery extends Query {
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      Scorer scorer = scorer(context, ScorerContext.def());
+      Scorer scorer = scorer(context, true, false, context.reader.getLiveDocs());
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/QueryTermVector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/QueryTermVector.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/QueryTermVector.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/QueryTermVector.java Sat Oct  1 03:04:53 2011
@@ -57,7 +57,7 @@ public class QueryTermVector implements 
     {
       TokenStream stream;
       try {
-        stream = analyzer.reusableTokenStream("", new StringReader(queryString));
+        stream = analyzer.tokenStream("", new StringReader(queryString));
       } catch (IOException e1) {
         stream = null;
       }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java Sat Oct  1 03:04:53 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.search;
 import java.io.IOException;
 
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
-import org.apache.lucene.search.Weight.ScorerContext;
 
 /** 
  * Constrains search results to only match those which also match a provided
@@ -56,7 +55,7 @@ public class QueryWrapperFilter extends 
     return new DocIdSet() {
       @Override
       public DocIdSetIterator iterator() throws IOException {
-        return weight.scorer(privateContext, ScorerContext.def());
+        return weight.scorer(privateContext, true, false, privateContext.reader.getLiveDocs());
       }
       @Override
       public boolean isCacheable() { return false; }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java Sat Oct  1 03:04:53 2011
@@ -61,7 +61,7 @@ public class SpanQueryFilter extends Spa
   public SpanFilterResult bitSpans(AtomicReaderContext context) throws IOException {
 
     final FixedBitSet bits = new FixedBitSet(context.reader.maxDoc());
-    Spans spans = query.getSpans(context);
+    Spans spans = query.getSpans(context, context.reader.getLiveDocs());
     List<SpanFilterResult.PositionInfo> tmp = new ArrayList<SpanFilterResult.PositionInfo>(20);
     int currentDoc = -1;
     SpanFilterResult.PositionInfo currentInfo = null;

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/TermQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/TermQuery.java Sat Oct  1 03:04:53 2011
@@ -30,6 +30,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.similarities.Similarity.ExactDocScorer;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.TermContext;
 import org.apache.lucene.util.ReaderUtil;
@@ -73,14 +74,15 @@ public class TermQuery extends Query {
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+        boolean topScorer, Bits acceptDocs) throws IOException {
       assert termStates.topReaderContext == ReaderUtil.getTopLevelContext(context) : "The top-reader used to create Weight (" + termStates.topReaderContext + ") is not the same as the current reader's top-reader (" + ReaderUtil.getTopLevelContext(context);
       final TermsEnum termsEnum = getTermsEnum(context);
       if (termsEnum == null) {
         return null;
       }
       // TODO should we reuse the DocsEnum here? 
-      final DocsEnum docs = termsEnum.docs(context.reader.getLiveDocs(), null);
+      final DocsEnum docs = termsEnum.docs(acceptDocs, null);
       assert docs != null;
       return new TermScorer(this, docs, createDocScorer(context));
     }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/Weight.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/Weight.java Sat Oct  1 03:04:53 2011
@@ -23,6 +23,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.search.similarities.SimilarityProvider;
+import org.apache.lucene.util.Bits;
 
 /**
  * Expert: Calculate query weights and build query scorers.
@@ -34,7 +35,8 @@ import org.apache.lucene.search.similari
  * {@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(IndexReader.AtomicReaderContext, ScorerContext)})
+ * {@link AtomicReaderContext} ({@link #scorer(IndexReader.AtomicReaderContext, 
+ * boolean, boolean, Bits)})
  * callers must maintain the relationship between the searcher's top-level
  * {@link ReaderContext} and the context used to create a {@link Scorer}. 
  * <p>
@@ -49,7 +51,7 @@ import org.apache.lucene.search.similari
  * <li>The query normalization factor is passed to {@link #normalize(float, float)}. At
  * this point the weighting is complete.
  * <li>A <code>Scorer</code> is constructed by
- * {@link #scorer(IndexReader.AtomicReaderContext, ScorerContext)}.
+ * {@link #scorer(IndexReader.AtomicReaderContext, boolean, boolean, Bits)}.
  * </ol>
  * 
  * @since 2.9
@@ -89,17 +91,33 @@ public abstract class Weight {
    * 
    * @param context
    *          the {@link AtomicReaderContext} for which to return the {@link Scorer}.
-   * @param scorerContext the {@link ScorerContext} holding the scores context variables
+   * @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),
+   *          this method can return whatever scoring mode it supports, as every
+   *          in-order scorer is also an out-of-order one. However, an
+   *          out-of-order scorer may not support {@link Scorer#nextDoc()}
+   *          and/or {@link Scorer#advance(int)}, therefore it is recommended to
+   *          request an in-order scorer if use of these methods is required.
+   * @param topScorer
+   *          if true, {@link Scorer#score(Collector)} will be called; if false,
+   *          {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)} will
+   *          be called.
+   * @param acceptDocs
+   *          Bits that represent the allowable docs to match (typically deleted docs
+   *          but possibly filtering other documents)
+   *          
    * @return a {@link Scorer} which scores documents in/out-of order.
    * @throws IOException
    */
-  public abstract Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException;
+  public abstract Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+      boolean topScorer, Bits acceptDocs) throws IOException;
 
   /**
    * 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(IndexReader.AtomicReaderContext, ScorerContext)} to
+   * {@link #scorer(IndexReader.AtomicReaderContext, boolean, boolean, Bits)} to
    * create a matching {@link Scorer} instance for a given {@link Collector}, or
    * vice versa.
    * <p>
@@ -107,83 +125,4 @@ public abstract class Weight {
    * the <code>Scorer</code> scores documents in-order.
    */
   public boolean scoresDocsOutOfOrder() { return false; }
-
-  /**
-   * A struct like class encapsulating a scorer's context variables.
-   * ScorerContex is a strictly immutable struct that follows a
-   * <tt>create on modification</tt> pattern. If a context variable changes
-   * through one of the modifiers like {@link #topScorer(boolean)} a new
-   * {@link ScorerContext} instance is creates. If the modifier call doesn't
-   * change the instance the method call has no effect and the same instance is
-   * returned from the modifier.
-   * 
-   * @lucene.experimental
-   */
-  public static final class ScorerContext {
-   
-    /**
-     * Specifies whether in-order scoring of documents is required. Note that if
-     * set to false (i.e., out-of-order scoring is required), this method can
-     * return whatever scoring mode it supports, as every in-order scorer is
-     * also an out-of-order one. However, an out-of-order scorer may not support
-     * {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)}, therefore it
-     * is recommended to request an in-order scorer if use of these methods is
-     * required.
-     */
-    public final boolean scoreDocsInOrder;
-    
-    /**
-     * if <code>true</code>, {@link Scorer#score(Collector)} will be called; if
-     * false, {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)} will
-     * be called instead.
-     */
-    public final boolean topScorer;
-    
-    
-    private static final ScorerContext DEFAULT_CONTEXT = new ScorerContext(true, false);
-
-    /**
-     * Returns a default {@link ScorerContext} template initialized with:
-     * <ul>
-     * <li>{@link #scoreDocsInOrder} = <code>true</code></li>
-     * <li>{@link #topScorer} = <code>false</code></li>
-     * </ul>
-     */
-    public static ScorerContext def() {
-      return DEFAULT_CONTEXT;
-    }
-    
-    private ScorerContext(boolean scoreDocsInOrder, boolean topScorer) {
-      this.scoreDocsInOrder = scoreDocsInOrder;
-      this.topScorer = topScorer;
-    }
-
-    /**
-     * Creates and returns a copy of this context with the given value for
-     * {@link #scoreDocsInOrder} and returns a new instance of
-     * {@link ScorerContext} iff the given value differs from the
-     * {@link #scoreDocsInOrder}. Otherwise, this method has no effect and
-     * returns this instance.
-     */
-    public ScorerContext scoreDocsInOrder(boolean scoreDocsInOrder) {
-      if (this.scoreDocsInOrder == scoreDocsInOrder) {
-        return this;
-      }
-      return new ScorerContext(scoreDocsInOrder, topScorer);
-    }
-    
-    /**
-     * Creates and returns a copy of this context with the given value for
-     * {@link #topScorer} and returns a new instance of
-     * {@link ScorerContext} iff the given value differs from the
-     * {@link #topScorer}. Otherwise, this method has no effect and
-     * returns this instance.
-     */
-    public ScorerContext topScorer(boolean topScorer) {
-      if (this.topScorer == topScorer) {
-        return this;
-      }
-      return new ScorerContext(scoreDocsInOrder, topScorer);
-    }
-  }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java Sat Oct  1 03:04:53 2011
@@ -33,6 +33,7 @@ import org.apache.lucene.search.spans.Sp
 import org.apache.lucene.search.spans.SpanScorer;
 import org.apache.lucene.search.spans.SpanWeight;
 import org.apache.lucene.search.spans.Spans;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ToStringUtils;
 
@@ -147,14 +148,15 @@ public class PayloadNearQuery extends Sp
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
-      return new PayloadNearSpanScorer(query.getSpans(context), this,
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+        boolean topScorer, Bits acceptDocs) throws IOException {
+      return new PayloadNearSpanScorer(query.getSpans(context, acceptDocs), this,
           similarity, similarity.sloppyDocScorer(stats, query.getField(), context));
     }
     
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, ScorerContext.def());
+      PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, true, false, context.reader.getLiveDocs());
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java Sat Oct  1 03:04:53 2011
@@ -176,7 +176,7 @@ public class PayloadSpanUtil {
       throws IOException {
     final AtomicReaderContext[] leaves = ReaderUtil.leaves(context);
     for (AtomicReaderContext atomicReaderContext : leaves) {
-      final Spans spans = query.getSpans(atomicReaderContext);
+      final Spans spans = query.getSpans(atomicReaderContext, atomicReaderContext.reader.getLiveDocs());
       while (spans.next() == true) {
         if (spans.isPayloadAvailable()) {
           Collection<byte[]> payload = spans.getPayload();

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java Sat Oct  1 03:04:53 2011
@@ -25,7 +25,6 @@ import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.ComplexExplanation;
-import org.apache.lucene.search.Weight.ScorerContext;
 import org.apache.lucene.search.payloads.PayloadNearQuery.PayloadNearSpanScorer;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.search.similarities.Similarity;
@@ -34,6 +33,7 @@ import org.apache.lucene.search.spans.Te
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.search.spans.SpanWeight;
 import org.apache.lucene.search.spans.SpanScorer;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 
 import java.io.IOException;
@@ -79,8 +79,9 @@ public class PayloadTermQuery extends Sp
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
-      return new PayloadTermSpanScorer((TermSpans) query.getSpans(context),
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+        boolean topScorer, Bits acceptDocs) throws IOException {
+      return new PayloadTermSpanScorer((TermSpans) query.getSpans(context, acceptDocs),
           this, similarity.sloppyDocScorer(stats, query.getField(), context));
     }
 
@@ -174,7 +175,7 @@ public class PayloadTermQuery extends Sp
     
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, ScorerContext.def());
+      PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, true, false, context.reader.getLiveDocs());
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java Sat Oct  1 03:04:53 2011
@@ -26,6 +26,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
 
 /**
@@ -92,8 +93,8 @@ public class FieldMaskingSpanQuery exten
   // ...this is done to be more consistent with things like SpanFirstQuery
   
   @Override
-  public Spans getSpans(AtomicReaderContext context) throws IOException {
-    return maskedQuery.getSpans(context);
+  public Spans getSpans(AtomicReaderContext context, Bits acceptDocs) throws IOException {
+    return maskedQuery.getSpans(context, acceptDocs);
   }
 
   @Override

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java Sat Oct  1 03:04:53 2011
@@ -19,6 +19,7 @@ package org.apache.lucene.search.spans;
 
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.Bits;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -77,11 +78,11 @@ public class NearSpansOrdered extends Sp
   private SpanNearQuery query;
   private boolean collectPayloads = true;
   
-  public NearSpansOrdered(SpanNearQuery spanNearQuery, AtomicReaderContext context) throws IOException {
-    this(spanNearQuery, context, true);
+  public NearSpansOrdered(SpanNearQuery spanNearQuery, AtomicReaderContext context, Bits acceptDocs) throws IOException {
+    this(spanNearQuery, context, acceptDocs, true);
   }
 
-  public NearSpansOrdered(SpanNearQuery spanNearQuery, AtomicReaderContext context, boolean collectPayloads)
+  public NearSpansOrdered(SpanNearQuery spanNearQuery, AtomicReaderContext context, Bits acceptDocs, boolean collectPayloads)
   throws IOException {
     if (spanNearQuery.getClauses().length < 2) {
       throw new IllegalArgumentException("Less than 2 clauses: "
@@ -94,7 +95,7 @@ public class NearSpansOrdered extends Sp
     matchPayload = new LinkedList<byte[]>();
     subSpansByDoc = new Spans[clauses.length];
     for (int i = 0; i < clauses.length; i++) {
-      subSpans[i] = clauses[i].getSpans(context);
+      subSpans[i] = clauses[i].getSpans(context, acceptDocs);
       subSpansByDoc[i] = subSpans[i]; // used in toSameDoc()
     }
     query = spanNearQuery; // kept for toString() only.

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java Sat Oct  1 03:04:53 2011
@@ -18,6 +18,7 @@ package org.apache.lucene.search.spans;
  */
 
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.PriorityQueue;
 
 import java.io.IOException;
@@ -131,7 +132,7 @@ public class NearSpansUnordered extends 
   }
 
 
-  public NearSpansUnordered(SpanNearQuery query, AtomicReaderContext context)
+  public NearSpansUnordered(SpanNearQuery query, AtomicReaderContext context, Bits acceptDocs)
     throws IOException {
     this.query = query;
     this.slop = query.getSlop();
@@ -141,7 +142,7 @@ public class NearSpansUnordered extends 
     subSpans = new Spans[clauses.length];    
     for (int i = 0; i < clauses.length; i++) {
       SpansCell cell =
-        new SpansCell(clauses[i].getSpans(context), i);
+        new SpansCell(clauses[i].getSpans(context, acceptDocs), i);
       ordered.add(cell);
       subSpans[i] = cell.spans;
     }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java Sat Oct  1 03:04:53 2011
@@ -27,6 +27,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.Bits;
 import org.apache.lucene.util.TermContext;
 
 /**
@@ -89,7 +90,7 @@ public class SpanMultiTermQueryWrapper<Q
   }
   
   @Override
-  public Spans getSpans(AtomicReaderContext context) throws IOException {
+  public Spans getSpans(AtomicReaderContext context, Bits acceptDocs) throws IOException {
     throw new UnsupportedOperationException("Query should have been rewritten");
   }
 

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanNearQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanNearQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanNearQuery.java Sat Oct  1 03:04:53 2011
@@ -30,6 +30,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
 
 /** Matches spans which are near one another.  One can specify <i>slop</i>, the
@@ -117,16 +118,16 @@ public class SpanNearQuery extends SpanQ
   }
 
   @Override
-  public Spans getSpans(final AtomicReaderContext context) throws IOException {
+  public Spans getSpans(final AtomicReaderContext context, Bits acceptDocs) throws IOException {
     if (clauses.size() == 0)                      // optimize 0-clause case
-      return new SpanOrQuery(getClauses()).getSpans(context);
+      return new SpanOrQuery(getClauses()).getSpans(context, acceptDocs);
 
     if (clauses.size() == 1)                      // optimize 1-clause case
-      return clauses.get(0).getSpans(context);
+      return clauses.get(0).getSpans(context, acceptDocs);
 
     return inOrder
-            ? (Spans) new NearSpansOrdered(this, context, collectPayloads)
-            : (Spans) new NearSpansUnordered(this, context);
+            ? (Spans) new NearSpansOrdered(this, context, acceptDocs, collectPayloads)
+            : (Spans) new NearSpansUnordered(this, context, acceptDocs);
   }
 
   @Override

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanNotQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanNotQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanNotQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanNotQuery.java Sat Oct  1 03:04:53 2011
@@ -21,6 +21,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
 
 import java.io.IOException;
@@ -75,12 +76,12 @@ public class SpanNotQuery extends SpanQu
   }
 
   @Override
-  public Spans getSpans(final AtomicReaderContext context) throws IOException {
+  public Spans getSpans(final AtomicReaderContext context, final Bits acceptDocs) throws IOException {
     return new Spans() {
-        private Spans includeSpans = include.getSpans(context);
+        private Spans includeSpans = include.getSpans(context, acceptDocs);
         private boolean moreInclude = true;
 
-        private Spans excludeSpans = exclude.getSpans(context);
+        private Spans excludeSpans = exclude.getSpans(context, acceptDocs);
         private boolean moreExclude = excludeSpans.next();
 
         @Override

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java Sat Oct  1 03:04:53 2011
@@ -28,6 +28,7 @@ import java.util.Set;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.search.Query;
@@ -163,9 +164,9 @@ public class SpanOrQuery extends SpanQue
   }
 
   @Override
-  public Spans getSpans(final AtomicReaderContext context) throws IOException {
+  public Spans getSpans(final AtomicReaderContext context, final Bits acceptDocs) throws IOException {
     if (clauses.size() == 1)                      // optimize 1-clause case
-      return (clauses.get(0)).getSpans(context);
+      return (clauses.get(0)).getSpans(context, acceptDocs);
 
     return new Spans() {
         private SpanQueue queue = null;
@@ -174,7 +175,7 @@ public class SpanOrQuery extends SpanQue
           queue = new SpanQueue(clauses.size());
           Iterator<SpanQuery> i = clauses.iterator();
           while (i.hasNext()) {
-            Spans spans = i.next().getSpans(context);
+            Spans spans = i.next().getSpans(context, acceptDocs);
             if (   ((target == -1) && spans.next())
                 || ((target != -1) && spans.skipTo(target))) {
               queue.add(spans);

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java Sat Oct  1 03:04:53 2011
@@ -21,6 +21,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.util.Bits;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -81,8 +82,8 @@ public abstract class SpanPositionCheckQ
   protected abstract AcceptStatus acceptPosition(Spans spans) throws IOException;
 
   @Override
-  public Spans getSpans(final AtomicReaderContext context) throws IOException {
-    return new PositionCheckSpan(context);
+  public Spans getSpans(final AtomicReaderContext context, Bits acceptDocs) throws IOException {
+    return new PositionCheckSpan(context, acceptDocs);
   }
 
 
@@ -106,8 +107,8 @@ public abstract class SpanPositionCheckQ
   protected class PositionCheckSpan extends Spans {
     private Spans spans;
 
-    public PositionCheckSpan(AtomicReaderContext context) throws IOException {
-      spans = match.getSpans(context);
+    public PositionCheckSpan(AtomicReaderContext context, Bits acceptDocs) throws IOException {
+      spans = match.getSpans(context, acceptDocs);
     }
 
     @Override

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java Sat Oct  1 03:04:53 2011
@@ -23,12 +23,13 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.Bits;
 
 /** Base class for span-based queries. */
 public abstract class SpanQuery extends Query {
   /** Expert: Returns the matches for this query in an index.  Used internally
    * to search for spans. */
-  public abstract Spans getSpans(AtomicReaderContext context) throws IOException;
+  public abstract Spans getSpans(AtomicReaderContext context, Bits acceptDocs) throws IOException;
 
   /** Returns the name of the field matched by this query.*/
   public abstract String getField();

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java Sat Oct  1 03:04:53 2011
@@ -21,6 +21,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
 
 import java.io.IOException;
@@ -81,9 +82,9 @@ public class SpanTermQuery extends SpanQ
   }
 
   @Override
-  public Spans getSpans(final AtomicReaderContext context) throws IOException {
+  public Spans getSpans(final AtomicReaderContext context, Bits acceptDocs) throws IOException {
     final IndexReader reader = context.reader;
-    final DocsAndPositionsEnum postings = reader.termPositionsEnum(reader.getLiveDocs(),
+    final DocsAndPositionsEnum postings = reader.termPositionsEnum(acceptDocs,
                                                                    term.field(),
                                                                    term.bytes());
 

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java Sat Oct  1 03:04:53 2011
@@ -23,6 +23,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.*;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.similarities.Similarity.SloppyDocScorer;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.TermContext;
 
 import java.io.IOException;
@@ -67,13 +68,14 @@ public class SpanWeight extends Weight {
   }
 
   @Override
-  public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
-    return new SpanScorer(query.getSpans(context), this, similarity.sloppyDocScorer(stats, query.getField(), context));
+  public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+      boolean topScorer, Bits acceptDocs) throws IOException {
+    return new SpanScorer(query.getSpans(context, acceptDocs), this, similarity.sloppyDocScorer(stats, query.getField(), context));
   }
 
   @Override
   public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-    Scorer scorer = scorer(context, ScorerContext.def());
+    Scorer scorer = scorer(context, true, false, context.reader.getLiveDocs());
     if (scorer != null) {
       int newDoc = scorer.advance(doc);
       if (newDoc == doc) {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/ByteBlockPool.java Sat Oct  1 03:04:53 2011
@@ -243,6 +243,18 @@ public final class ByteBlockPool {
     assert term.length >= 0;
     return term;
   }
+  /**
+   * Dereferences the byte block according to {@link BytesRef} offset. The offset 
+   * is interpreted as the absolute offset into the {@link ByteBlockPool}.
+   */
+  public final BytesRef deref(BytesRef bytes) {
+    final int offset = bytes.offset;
+    byte[] buffer = buffers[offset >> BYTE_BLOCK_SHIFT];
+    int pos = offset & BYTE_BLOCK_MASK;
+    bytes.bytes = buffer;
+    bytes.offset = pos;
+    return bytes;
+  }
   
   /**
    * Copies the given {@link BytesRef} at the current positions (

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/BytesRef.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/BytesRef.java Sat Oct  1 03:04:53 2011
@@ -238,6 +238,12 @@ public final class BytesRef implements C
     return sb.toString();
   }
 
+  /**
+   * Copies the given {@link BytesRef}
+   * <p>
+   * NOTE: this method resets the offset to 0 and resizes the reference array
+   * if needed.
+   */
   public void copy(BytesRef other) {
     if (bytes.length < other.length) {
       bytes = new byte[other.length];
@@ -247,6 +253,93 @@ public final class BytesRef implements C
     offset = 0;
   }
 
+  /**
+   * Copies the given long value and encodes it as 8 byte Big-Endian.
+   * <p>
+   * NOTE: this method resets the offset to 0, length to 8 and resizes the reference array
+   * if needed.
+   */
+  public void copy(long value) {
+    if (bytes.length < 8) {
+      bytes = new byte[8];
+    }
+    copyInternal((int) (value >> 32), offset = 0);
+    copyInternal((int) value, 4);
+    length = 8;
+  }
+  
+  /**
+   * Copies the given int value and encodes it as 4 byte Big-Endian.
+   * <p>
+   * NOTE: this method resets the offset to 0, length to 4 and resizes the reference array
+   * if needed.
+   */
+  public void copy(int value) {
+    if (bytes.length < 4) {
+      bytes = new byte[4];
+    }
+    copyInternal(value, offset = 0);
+    length = 4;
+  }
+
+  /**
+   * Copies the given short value and encodes it as a 2 byte Big-Endian.
+   * <p>
+   * NOTE: this method resets the offset to 0, length to 2 and resizes the reference array
+   * if needed.
+   */
+  public void copy(short value) {
+    if (bytes.length < 2) {
+      bytes = new byte[2];
+    }
+    bytes[offset] = (byte) (value >> 8);
+    bytes[offset + 1] = (byte) (value);
+
+  }
+  
+  /**
+   * Converts 2 consecutive bytes from the current offset to a short. Bytes are
+   * interpreted as Big-Endian (most significant bit first)
+   * <p>
+   * NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
+   */
+  public short asShort() {
+    int pos = offset;
+    return (short) (0xFFFF & ((bytes[pos++] & 0xFF) << 8) | (bytes[pos] & 0xFF));
+  }
+
+  /**
+   * Converts 4 consecutive bytes from the current offset to an int. Bytes are
+   * interpreted as Big-Endian (most significant bit first)
+   * <p>
+   * NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
+   */
+  public int asInt() {
+    return asIntInternal(offset);
+  }
+
+  /**
+   * Converts 8 consecutive bytes from the current offset to a long. Bytes are
+   * interpreted as Big-Endian (most significant bit first)
+   * <p>
+   * NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
+   */
+  public long asLong() {
+    return (((long) asIntInternal(offset) << 32) | asIntInternal(offset + 4) & 0xFFFFFFFFL);
+  }
+
+  private void copyInternal(int value, int startOffset) {
+    bytes[startOffset] = (byte) (value >> 24);
+    bytes[startOffset + 1] = (byte) (value >> 16);
+    bytes[startOffset + 2] = (byte) (value >> 8);
+    bytes[startOffset + 3] = (byte) (value);
+  }
+
+  private int asIntInternal(int pos) {
+    return ((bytes[pos++] & 0xFF) << 24) | ((bytes[pos++] & 0xFF) << 16)
+        | ((bytes[pos++] & 0xFF) << 8) | (bytes[pos] & 0xFF);
+  }
+
   public void append(BytesRef other) {
     int newLen = length + other.length;
     if (bytes.length < newLen) {
@@ -284,7 +377,7 @@ public final class BytesRef implements C
     // One is a prefix of the other, or, they are equal:
     return this.length - other.length;
   }
-
+  
   private final static Comparator<BytesRef> utf8SortedAsUnicodeSortOrder = new UTF8SortedAsUnicodeComparator();
 
   public static Comparator<BytesRef> getUTF8SortedAsUnicodeComparator() {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct16.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct16.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct16.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct16.java Sat Oct  1 03:04:53 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.DataInput;
 import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.IOException;
@@ -38,7 +38,7 @@ public class Direct16 extends PackedInts
     values = new short[valueCount];
   }
 
-  public Direct16(IndexInput in, int valueCount) throws IOException {
+  public Direct16(DataInput in, int valueCount) throws IOException {
     super(valueCount, BITS_PER_VALUE);
     short[] values = new short[valueCount];
     for(int i=0;i<valueCount;i++) {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct32.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct32.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct32.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct32.java Sat Oct  1 03:04:53 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.DataInput;
 import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.IOException;
@@ -38,7 +38,7 @@ public class Direct32 extends PackedInts
     values = new int[valueCount];
   }
 
-  public Direct32(IndexInput in, int valueCount) throws IOException {
+  public Direct32(DataInput in, int valueCount) throws IOException {
     super(valueCount, BITS_PER_VALUE);
     int[] values = new int[valueCount];
     for(int i=0;i<valueCount;i++) {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct64.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct64.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct64.java Sat Oct  1 03:04:53 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.DataInput;
 import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.IOException;
@@ -38,7 +38,7 @@ public class Direct64 extends PackedInts
     values = new long[valueCount];
   }
 
-  public Direct64(IndexInput in, int valueCount) throws IOException {
+  public Direct64(DataInput in, int valueCount) throws IOException {
     super(valueCount, BITS_PER_VALUE);
     long[] values = new long[valueCount];
     for(int i=0;i<valueCount;i++) {

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct8.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct8.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct8.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Direct8.java Sat Oct  1 03:04:53 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.DataInput;
 import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.IOException;
@@ -38,7 +38,7 @@ public class Direct8 extends PackedInts.
     values = new byte[valueCount];
   }
 
-  public Direct8(IndexInput in, int valueCount)
+  public Direct8(DataInput in, int valueCount)
           throws IOException {
     super(valueCount, BITS_PER_VALUE);
     byte[] values = new byte[valueCount];

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Packed32.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Packed32.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Packed32.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Packed32.java Sat Oct  1 03:04:53 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.DataInput;
 import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.IOException;
@@ -122,14 +122,14 @@ class Packed32 extends PackedInts.Reader
   }
 
   /**
-   * Creates an array with content retrieved from the given IndexInput.
-   * @param in       an IndexInput, positioned at the start of Packed64-content.
+   * Creates an array with content retrieved from the given DataInput.
+   * @param in       a DataInput, positioned at the start of Packed64-content.
    * @param valueCount  the number of elements.
    * @param bitsPerValue the number of bits available for any given value.
    * @throws java.io.IOException if the values for the backing array could not
    *                             be retrieved.
    */
-  public Packed32(IndexInput in, int valueCount, int bitsPerValue)
+  public Packed32(DataInput in, int valueCount, int bitsPerValue)
                                                             throws IOException {
     super(valueCount, bitsPerValue);
     int size = size(bitsPerValue, valueCount);

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Packed64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Packed64.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Packed64.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/Packed64.java Sat Oct  1 03:04:53 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.DataInput;
 import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.IOException;
@@ -141,14 +141,14 @@ class Packed64 extends PackedInts.Reader
   }
 
   /**
-   * Creates an array with content retrieved from the given IndexInput.
-   * @param in       an IndexInput, positioned at the start of Packed64-content.
+   * Creates an array with content retrieved from the given DataInput.
+   * @param in       a DataInput, positioned at the start of Packed64-content.
    * @param valueCount  the number of elements.
    * @param bitsPerValue the number of bits available for any given value.
    * @throws java.io.IOException if the values for the backing array could not
    *                             be retrieved.
    */
-  public Packed64(IndexInput in, int valueCount, int bitsPerValue)
+  public Packed64(DataInput in, int valueCount, int bitsPerValue)
                                                             throws IOException {
     super(valueCount, bitsPerValue);
     int size = size(valueCount, bitsPerValue);

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/PackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/PackedInts.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/PackedInts.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/PackedInts.java Sat Oct  1 03:04:53 2011
@@ -19,7 +19,8 @@ package org.apache.lucene.util.packed;
 
 import java.io.Closeable;
 
-import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.util.Constants;
@@ -134,11 +135,11 @@ public class PackedInts {
    * @lucene.internal
    */
   public static abstract class Writer {
-    protected final IndexOutput out;
+    protected final DataOutput out;
     protected final int bitsPerValue;
     protected final int valueCount;
 
-    protected Writer(IndexOutput out, int valueCount, int bitsPerValue)
+    protected Writer(DataOutput out, int valueCount, int bitsPerValue)
       throws IOException {
       assert bitsPerValue <= 64;
 
@@ -155,14 +156,14 @@ public class PackedInts {
   }
 
   /**
-   * Retrieve PackedInt data from the IndexInput and return a packed int
+   * Retrieve PackedInt data from the DataInput and return a packed int
    * structure based on it.
    * @param in positioned at the beginning of a stored packed int structure.
    * @return a read only random access capable array of positive integers.
    * @throws IOException if the structure could not be retrieved.
    * @lucene.internal
    */
-  public static Reader getReader(IndexInput in) throws IOException {
+  public static Reader getReader(DataInput in) throws IOException {
     CodecUtil.checkHeader(in, CODEC_NAME, VERSION_START, VERSION_START);
     final int bitsPerValue = in.readVInt();
     assert bitsPerValue > 0 && bitsPerValue <= 64: "bitsPerValue=" + bitsPerValue;
@@ -244,7 +245,7 @@ public class PackedInts {
    * @throws IOException if bits could not be written to out.
    * @lucene.internal
    */
-  public static Writer getWriter(IndexOutput out, int valueCount, int bitsPerValue)
+  public static Writer getWriter(DataOutput out, int valueCount, int bitsPerValue)
     throws IOException {
     return new PackedWriter(out, valueCount, bitsPerValue);
   }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/PackedWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/PackedWriter.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/PackedWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/util/packed/PackedWriter.java Sat Oct  1 03:04:53 2011
@@ -17,7 +17,7 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.DataOutput;
 
 import java.io.IOException;
 
@@ -38,7 +38,7 @@ class PackedWriter extends PackedInts.Wr
   private final long[] masks;
   private int written = 0;
 
-  public PackedWriter(IndexOutput out, int valueCount, int bitsPerValue)
+  public PackedWriter(DataOutput out, int valueCount, int bitsPerValue)
                                                             throws IOException {
     super(out, valueCount, bitsPerValue);
 

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Sat Oct  1 03:04:53 2011
@@ -208,7 +208,7 @@ public abstract class BaseTokenStreamTes
   
 
   public static void assertAnalyzesToReuse(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[]) throws IOException {
-    assertTokenStreamContents(a.reusableTokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, input.length());
+    assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, input.length());
   }
   
   public static void assertAnalyzesToReuse(Analyzer a, String input, String[] output) throws IOException {
@@ -265,7 +265,7 @@ public abstract class BaseTokenStreamTes
         System.out.println("NOTE: BaseTokenStreamTestCase: get first token stream now text=" + text);
       }
 
-      TokenStream ts = a.reusableTokenStream("dummy", new StringReader(text));
+      TokenStream ts = a.tokenStream("dummy", new StringReader(text));
       assertTrue("has no CharTermAttribute", ts.hasAttribute(CharTermAttribute.class));
       CharTermAttribute termAtt = ts.getAttribute(CharTermAttribute.class);
       OffsetAttribute offsetAtt = ts.hasAttribute(OffsetAttribute.class) ? ts.getAttribute(OffsetAttribute.class) : null;

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/CollationTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/CollationTestBase.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/CollationTestBase.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/CollationTestBase.java Sat Oct  1 03:04:53 2011
@@ -81,8 +81,8 @@ public abstract class CollationTestBase 
     IndexWriter writer = new IndexWriter(ramDir, new IndexWriterConfig(
         TEST_VERSION_CURRENT, analyzer));
     Document doc = new Document();
-    doc.add(new Field("content", TextField.TYPE_STORED, "\u0633\u0627\u0628"));
-    doc.add(new Field("body", StringField.TYPE_STORED, "body"));
+    doc.add(new Field("content", "\u0633\u0627\u0628", TextField.TYPE_STORED));
+    doc.add(new Field("body", "body", StringField.TYPE_STORED));
     writer.addDocument(doc);
     writer.close();
     IndexSearcher searcher = new IndexSearcher(ramDir, true);
@@ -116,7 +116,7 @@ public abstract class CollationTestBase 
     // orders the U+0698 character before the U+0633 character, so the single
     // index Term below should NOT be returned by a TermRangeQuery with a Farsi
     // Collator (or an Arabic one for the case when Farsi is not supported).
-    doc.add(new Field("content", TextField.TYPE_STORED, "\u0633\u0627\u0628"));
+    doc.add(new Field("content", "\u0633\u0627\u0628", TextField.TYPE_STORED));
     writer.addDocument(doc);
     writer.close();
     IndexSearcher searcher = new IndexSearcher(ramDir, true);
@@ -138,8 +138,8 @@ public abstract class CollationTestBase 
     IndexWriter writer = new IndexWriter(farsiIndex, new IndexWriterConfig(
         TEST_VERSION_CURRENT, analyzer));
     Document doc = new Document();
-    doc.add(new Field("content", TextField.TYPE_STORED, "\u0633\u0627\u0628"));
-    doc.add(new Field("body", StringField.TYPE_STORED, "body"));
+    doc.add(new Field("content", "\u0633\u0627\u0628", TextField.TYPE_STORED));
+    doc.add(new Field("body", "body", StringField.TYPE_STORED));
     writer.addDocument(doc);
     writer.close();
 
@@ -204,16 +204,16 @@ public abstract class CollationTestBase 
     
     for (int i = 0 ; i < sortData.length ; ++i) {
       Document doc = new Document();
-      doc.add(new Field("tracer", customType, sortData[i][0]));
+      doc.add(new Field("tracer", sortData[i][0], customType));
       doc.add(new TextField("contents", sortData[i][1]));
       if (sortData[i][2] != null) 
-        doc.add(new TextField("US", usAnalyzer.reusableTokenStream("US", new StringReader(sortData[i][2]))));
+        doc.add(new TextField("US", usAnalyzer.tokenStream("US", new StringReader(sortData[i][2]))));
       if (sortData[i][3] != null) 
-        doc.add(new TextField("France", franceAnalyzer.reusableTokenStream("France", new StringReader(sortData[i][3]))));
+        doc.add(new TextField("France", franceAnalyzer.tokenStream("France", new StringReader(sortData[i][3]))));
       if (sortData[i][4] != null)
-        doc.add(new TextField("Sweden", swedenAnalyzer.reusableTokenStream("Sweden", new StringReader(sortData[i][4]))));
+        doc.add(new TextField("Sweden", swedenAnalyzer.tokenStream("Sweden", new StringReader(sortData[i][4]))));
       if (sortData[i][5] != null) 
-        doc.add(new TextField("Denmark", denmarkAnalyzer.reusableTokenStream("Denmark", new StringReader(sortData[i][5]))));
+        doc.add(new TextField("Denmark", denmarkAnalyzer.tokenStream("Denmark", new StringReader(sortData[i][5]))));
       writer.addDocument(doc);
     }
     writer.optimize();
@@ -265,7 +265,7 @@ public abstract class CollationTestBase 
 
     for (int i = 0; i < numTestPoints; i++) {
       String term = _TestUtil.randomSimpleString(random);
-      TokenStream ts = analyzer.reusableTokenStream("fake", new StringReader(term));
+      TokenStream ts = analyzer.tokenStream("fake", new StringReader(term));
       TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
       BytesRef bytes = termAtt.getBytesRef();
       ts.reset();
@@ -284,7 +284,7 @@ public abstract class CollationTestBase 
             for (Map.Entry<String,BytesRef> mapping : map.entrySet()) {
               String term = mapping.getKey();
               BytesRef expected = mapping.getValue();
-              TokenStream ts = analyzer.reusableTokenStream("fake", new StringReader(term));
+              TokenStream ts = analyzer.tokenStream("fake", new StringReader(term));
               TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
               BytesRef bytes = termAtt.getBytesRef();
               ts.reset();

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java Sat Oct  1 03:04:53 2011
@@ -42,7 +42,7 @@ import org.apache.lucene.util.automaton.
  * </ul>
  * @see MockTokenizer
  */
-public final class MockAnalyzer extends ReusableAnalyzerBase {
+public final class MockAnalyzer extends Analyzer {
   private final CharacterRunAutomaton runAutomaton;
   private final boolean lowerCase;
   private final CharacterRunAutomaton filter;

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/MockPayloadAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/MockPayloadAnalyzer.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/MockPayloadAnalyzer.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/analysis/MockPayloadAnalyzer.java Sat Oct  1 03:04:53 2011
@@ -30,7 +30,7 @@ import java.io.Reader;
  *
  *
  **/
-public final class MockPayloadAnalyzer extends ReusableAnalyzerBase {
+public final class MockPayloadAnalyzer extends Analyzer {
 
   @Override
   public TokenStreamComponents createComponents(String fieldName, Reader reader) {

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java Sat Oct  1 03:04:53 2011
@@ -46,7 +46,7 @@ class DocHelper {
   public static Field textField1;
   static {
     customType = new FieldType(TextField.TYPE_STORED);
-    textField1 = new Field(TEXT_FIELD_1_KEY, customType, FIELD_1_TEXT);
+    textField1 = new Field(TEXT_FIELD_1_KEY, FIELD_1_TEXT, customType);
   }
 
   public static final FieldType customType2;
@@ -60,7 +60,7 @@ class DocHelper {
     customType2.setStoreTermVectors(true);
     customType2.setStoreTermVectorPositions(true);
     customType2.setStoreTermVectorOffsets(true);
-    textField2 = new Field(TEXT_FIELD_2_KEY, customType2, FIELD_2_TEXT);
+    textField2 = new Field(TEXT_FIELD_2_KEY, FIELD_2_TEXT, customType2);
   }
   
   public static final FieldType customType3;
@@ -71,14 +71,14 @@ class DocHelper {
   static {
     customType3 = new FieldType(TextField.TYPE_STORED);
     customType3.setOmitNorms(true);
-    textField3 = new Field(TEXT_FIELD_3_KEY, customType3, FIELD_3_TEXT);
+    textField3 = new Field(TEXT_FIELD_3_KEY, FIELD_3_TEXT, customType3);
   }
 
   public static final String KEYWORD_TEXT = "Keyword";
   public static final String KEYWORD_FIELD_KEY = "keyField";
   public static Field keyField;
   static {
-    keyField = new Field(KEYWORD_FIELD_KEY, StringField.TYPE_STORED,  KEYWORD_TEXT);
+    keyField = new Field(KEYWORD_FIELD_KEY, KEYWORD_TEXT, StringField.TYPE_STORED);
   }
 
   public static final FieldType customType5;
@@ -89,7 +89,7 @@ class DocHelper {
     customType5 = new FieldType(TextField.TYPE_STORED);
     customType5.setOmitNorms(true);
     customType5.setTokenized(false);
-    noNormsField = new Field(NO_NORMS_KEY, customType5, NO_NORMS_TEXT);
+    noNormsField = new Field(NO_NORMS_KEY, NO_NORMS_TEXT, customType5);
   }
 
   public static final FieldType customType6;
@@ -99,7 +99,7 @@ class DocHelper {
   static {
     customType6 = new FieldType(TextField.TYPE_STORED);
     customType6.setIndexOptions(IndexOptions.DOCS_ONLY);
-    noTFField = new Field(NO_TF_KEY, customType6, NO_TF_TEXT);
+    noTFField = new Field(NO_TF_KEY, NO_TF_TEXT, customType6);
   }
 
   public static final FieldType customType7;
@@ -109,13 +109,13 @@ class DocHelper {
   static {
     customType7 = new FieldType();
     customType7.setStored(true);
-    unIndField = new Field(UNINDEXED_FIELD_KEY, customType7, UNINDEXED_FIELD_TEXT);
+    unIndField = new Field(UNINDEXED_FIELD_KEY, UNINDEXED_FIELD_TEXT, customType7);
   }
 
 
   public static final String UNSTORED_1_FIELD_TEXT = "unstored field text";
   public static final String UNSTORED_FIELD_1_KEY = "unStoredField1";
-  public static Field unStoredField1 = new Field(UNSTORED_FIELD_1_KEY, TextField.TYPE_UNSTORED, UNSTORED_1_FIELD_TEXT);
+  public static Field unStoredField1 = new Field(UNSTORED_FIELD_1_KEY, UNSTORED_1_FIELD_TEXT, TextField.TYPE_UNSTORED);
 
   public static final FieldType customType8;
   public static final String UNSTORED_2_FIELD_TEXT = "unstored field text";
@@ -124,7 +124,7 @@ class DocHelper {
   static {
     customType8 = new FieldType(TextField.TYPE_UNSTORED);
     customType8.setStoreTermVectors(true);
-    unStoredField2 = new Field(UNSTORED_FIELD_2_KEY, customType8, UNSTORED_2_FIELD_TEXT);
+    unStoredField2 = new Field(UNSTORED_FIELD_2_KEY, UNSTORED_2_FIELD_TEXT, customType8);
   }
 
   public static final String LAZY_FIELD_BINARY_KEY = "lazyFieldBinary";
@@ -133,7 +133,7 @@ class DocHelper {
 
   public static final String LAZY_FIELD_KEY = "lazyField";
   public static final String LAZY_FIELD_TEXT = "These are some field bytes";
-  public static Field lazyField = new Field(LAZY_FIELD_KEY, customType, LAZY_FIELD_TEXT);
+  public static Field lazyField = new Field(LAZY_FIELD_KEY, LAZY_FIELD_TEXT, customType);
   
   public static final String LARGE_LAZY_FIELD_KEY = "largeLazyField";
   public static String LARGE_LAZY_FIELD_TEXT;
@@ -142,13 +142,13 @@ class DocHelper {
   //From Issue 509
   public static final String FIELD_UTF1_TEXT = "field one \u4e00text";
   public static final String TEXT_FIELD_UTF1_KEY = "textField1Utf8";
-  public static Field textUtfField1 = new Field(TEXT_FIELD_UTF1_KEY, customType, FIELD_UTF1_TEXT);
+  public static Field textUtfField1 = new Field(TEXT_FIELD_UTF1_KEY, FIELD_UTF1_TEXT, customType);
 
   public static final String FIELD_UTF2_TEXT = "field field field \u4e00two text";
   //Fields will be lexicographically sorted.  So, the order is: field, text, two
   public static final int [] FIELD_UTF2_FREQS = {3, 1, 1};
   public static final String TEXT_FIELD_UTF2_KEY = "textField2Utf8";
-  public static Field textUtfField2 = new Field(TEXT_FIELD_UTF2_KEY, customType2, FIELD_UTF2_TEXT);
+  public static Field textUtfField2 = new Field(TEXT_FIELD_UTF2_KEY, FIELD_UTF2_TEXT, customType2);
  
   
   
@@ -200,7 +200,7 @@ class DocHelper {
     lazyFieldBinary = new BinaryField(LAZY_FIELD_BINARY_KEY, LAZY_FIELD_BINARY_BYTES);
     fields[fields.length - 2] = lazyFieldBinary;
     LARGE_LAZY_FIELD_TEXT = buffer.toString();
-    largeLazyField = new Field(LARGE_LAZY_FIELD_KEY, customType, LARGE_LAZY_FIELD_TEXT);
+    largeLazyField = new Field(LARGE_LAZY_FIELD_KEY, LARGE_LAZY_FIELD_TEXT, customType);
     fields[fields.length - 1] = largeLazyField;
     for (int i=0; i<fields.length; i++) {
       IndexableField f = fields[i];
@@ -304,15 +304,15 @@ class DocHelper {
     customType1.setStoreTermVectorOffsets(true);
 
     final Document doc = new Document();
-    doc.add(new Field("id", customType1, Integer.toString(n)));
-    doc.add(new Field("indexname", customType1, indexName));
+    doc.add(new Field("id", Integer.toString(n), customType1));
+    doc.add(new Field("indexname", indexName, customType1));
     sb.append("a");
     sb.append(n);
-    doc.add(new Field("field1", customType, sb.toString()));
+    doc.add(new Field("field1", sb.toString(), customType));
     sb.append(" b");
     sb.append(n);
     for (int i = 1; i < numFields; i++) {
-      doc.add(new Field("field" + (i + 1), customType, sb.toString()));
+      doc.add(new Field("field" + (i + 1), sb.toString(), customType));
     }
     return doc;
   }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java Sat Oct  1 03:04:53 2011
@@ -30,16 +30,16 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.sep.IntStreamFactory;
 import org.apache.lucene.index.codecs.sep.IntIndexInput;
 import org.apache.lucene.index.codecs.sep.IntIndexOutput;
+import org.apache.lucene.index.codecs.sep.SepDocValuesConsumer;
+import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
 import org.apache.lucene.index.codecs.sep.SepPostingsReader;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexOutput;
-import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.index.codecs.PerDocConsumer;
-import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
@@ -207,7 +207,7 @@ public class MockFixedIntBlockCodec exte
     SepPostingsReader.files(segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
-    DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files, getDocValuesUseCFS());
+    SepDocValuesConsumer.files(dir, segmentInfo, codecId, files);
   }
 
   @Override
@@ -215,16 +215,16 @@ public class MockFixedIntBlockCodec exte
     SepPostingsWriter.getExtensions(extensions);
     BlockTermsReader.getExtensions(extensions);
     FixedGapTermsIndexReader.getIndexExtensions(extensions);
-    DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());
+    SepDocValuesConsumer.getExtensions(extensions);
   }
   
   @Override
   public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
-    return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS());
+    return new SepDocValuesConsumer(state);
   }
 
   @Override
   public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
+    return new SepDocValuesProducer(state);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java Sat Oct  1 03:04:53 2011
@@ -30,16 +30,16 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.sep.IntStreamFactory;
 import org.apache.lucene.index.codecs.sep.IntIndexInput;
 import org.apache.lucene.index.codecs.sep.IntIndexOutput;
+import org.apache.lucene.index.codecs.sep.SepDocValuesConsumer;
+import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
 import org.apache.lucene.index.codecs.sep.SepPostingsReader;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
 import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexInput;
 import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexOutput;
-import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.index.codecs.PerDocConsumer;
-import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
@@ -230,7 +230,7 @@ public class MockVariableIntBlockCodec e
     SepPostingsReader.files(segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
-    DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files, getDocValuesUseCFS());
+    SepDocValuesConsumer.files(dir, segmentInfo, codecId, files);
   }
 
   @Override
@@ -238,16 +238,16 @@ public class MockVariableIntBlockCodec e
     SepPostingsWriter.getExtensions(extensions);
     BlockTermsReader.getExtensions(extensions);
     FixedGapTermsIndexReader.getIndexExtensions(extensions);
-    DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());
+    SepDocValuesConsumer.getExtensions(extensions);
   }
   
   @Override
   public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
-    return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS());
+    return new SepDocValuesConsumer(state);
   }
 
   @Override
   public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
+    return new SepDocValuesProducer(state);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java Sat Oct  1 03:04:53 2011
@@ -58,6 +58,8 @@ import org.apache.lucene.index.codecs.pu
 import org.apache.lucene.index.codecs.sep.IntIndexInput;
 import org.apache.lucene.index.codecs.sep.IntIndexOutput;
 import org.apache.lucene.index.codecs.sep.IntStreamFactory;
+import org.apache.lucene.index.codecs.sep.SepDocValuesConsumer;
+import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
 import org.apache.lucene.index.codecs.sep.SepPostingsReader;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
 import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
@@ -75,17 +77,13 @@ import org.apache.lucene.util._TestUtil;
  */
 
 public class MockRandomCodec extends Codec {
-
+  private final boolean useSepDocValues;
   private final Random seedRandom;
   private final String SEED_EXT = "sd";
   
   public MockRandomCodec(Random random) {
-    this(random, "MockRandom", true);
-    
-  }
-  
-  protected MockRandomCodec(Random random, String name, boolean docValuesUseCompoundFile) {
-    super(name, docValuesUseCompoundFile);
+    super("MockRandom");
+    this.useSepDocValues = random.nextBoolean();
     this.seedRandom = new Random(random.nextLong());
   }
 
@@ -428,7 +426,11 @@ public class MockRandomCodec extends Cod
     BlockTreeTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
     VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
-    DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files, getDocValuesUseCFS());
+    if (useSepDocValues) {
+      SepDocValuesConsumer.files(dir, segmentInfo, codecId, files);
+    } else {
+      DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files);
+    }
     // hackish!
     Iterator<String> it = files.iterator();
     while(it.hasNext()) {
@@ -447,7 +449,11 @@ public class MockRandomCodec extends Cod
     BlockTreeTermsReader.getExtensions(extensions);
     FixedGapTermsIndexReader.getIndexExtensions(extensions);
     VariableGapTermsIndexReader.getIndexExtensions(extensions);
-    DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());
+    if (useSepDocValues) {
+      SepDocValuesConsumer.getExtensions(extensions);
+    } else {
+      DefaultDocValuesConsumer.getExtensions(extensions);      
+    }
     extensions.add(SEED_EXT);
     //System.out.println("MockRandom.getExtensions return " + extensions);
   }
@@ -455,11 +461,19 @@ public class MockRandomCodec extends Cod
   // can we make this more evil?
   @Override
   public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
-    return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS());
+    if (useSepDocValues) {
+      return new SepDocValuesConsumer(state);
+    } else {
+      return new DefaultDocValuesConsumer(state);
+    }
   }
 
   @Override
   public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
+    if (useSepDocValues) {
+      return new SepDocValuesProducer(state);
+    } else {
+      return new DefaultDocValuesProducer(state);
+    }
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSepCodec.java Sat Oct  1 03:04:53 2011
@@ -25,13 +25,11 @@ import org.apache.lucene.index.SegmentIn
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.DefaultDocValuesProducer;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
 import org.apache.lucene.index.codecs.PerDocConsumer;
-import org.apache.lucene.index.codecs.DefaultDocValuesConsumer;
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
@@ -40,6 +38,8 @@ import org.apache.lucene.index.codecs.Bl
 import org.apache.lucene.index.codecs.TermsIndexReaderBase;
 import org.apache.lucene.index.codecs.TermsIndexWriterBase;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
+import org.apache.lucene.index.codecs.sep.SepDocValuesConsumer;
+import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
 import org.apache.lucene.index.codecs.sep.SepPostingsReader;
 import org.apache.lucene.store.Directory;
@@ -139,28 +139,28 @@ public class MockSepCodec extends Codec 
     SepPostingsReader.files(segmentInfo, codecId, files);
     BlockTermsReader.files(dir, segmentInfo, codecId, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
-    DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files, getDocValuesUseCFS());
+    SepDocValuesConsumer.files(dir, segmentInfo, codecId, files);
   }
 
   @Override
   public void getExtensions(Set<String> extensions) {
     getSepExtensions(extensions);
-    DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS());
   }
 
   public static void getSepExtensions(Set<String> extensions) {
     SepPostingsWriter.getExtensions(extensions);
     BlockTermsReader.getExtensions(extensions);
     FixedGapTermsIndexReader.getIndexExtensions(extensions);
+    SepDocValuesConsumer.getExtensions(extensions);
   }
   
   @Override
   public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
-    return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS());
+    return new SepDocValuesConsumer(state);
   }
 
   @Override
   public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
+    return new SepDocValuesProducer(state);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/search/AssertingIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/search/AssertingIndexSearcher.java?rev=1177888&r1=1177887&r2=1177888&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/search/AssertingIndexSearcher.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/org/apache/lucene/search/AssertingIndexSearcher.java Sat Oct  1 03:04:53 2011
@@ -23,6 +23,7 @@ import java.io.IOException;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader.ReaderContext;
+import org.apache.lucene.util.Bits;
 
 /** 
  * Helper class that adds some extra checks to ensure correct
@@ -67,8 +68,9 @@ public class AssertingIndexSearcher exte
       }
 
       @Override
-      public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
-        return w.scorer(context, scorerContext);
+      public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+          boolean topScorer, Bits acceptDocs) throws IOException {
+        return w.scorer(context, scoreDocsInOrder, topScorer, acceptDocs);
       }
 
       @Override