You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2009/06/07 18:58:43 UTC

svn commit: r782410 [2/2] - in /lucene/java/trunk: ./ contrib/miscellaneous/src/test/org/apache/lucene/misc/ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/search/ src/java/org/apache/lucene/search/function/ src/java/org/apache/lucene/sea...

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/NonMatchingScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/NonMatchingScorer.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/NonMatchingScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/NonMatchingScorer.java Sun Jun  7 16:58:41 2009
@@ -23,13 +23,22 @@
 class NonMatchingScorer extends Scorer {
   public NonMatchingScorer() { super(null); } // no similarity used
   
+  /** @deprecated use {@link #docID()} instead. */
   public int doc() { throw new UnsupportedOperationException(); }
+  
+  public int docID() { return NO_MORE_DOCS; }
 
+  /** @deprecated use {@link #nextDoc()} instead. */
   public boolean next() throws IOException { return false; }
+  
+  public int nextDoc() throws IOException { return NO_MORE_DOCS; }
 
   public float score() { throw new UnsupportedOperationException(); }
 
+  /** @deprecated use {@link #advance(int)} instead. */
   public boolean skipTo(int target) { return false; }
+  
+  public int advance(int target) { return NO_MORE_DOCS; }
 
   public Explanation explain(int doc) {
     Explanation e = new Explanation();

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/PhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/PhraseScorer.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/PhraseScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/PhraseScorer.java Sun Jun  7 16:58:41 2009
@@ -43,9 +43,8 @@
 
   private float freq; //prhase frequency in current doc as computed by phraseFreq().
 
-
-  PhraseScorer(Weight weight, TermPositions[] tps, int[] offsets, Similarity similarity,
-               byte[] norms) {
+  PhraseScorer(Weight weight, TermPositions[] tps, int[] offsets,
+      Similarity similarity, byte[] norms) {
     super(similarity);
     this.norms = norms;
     this.weight = weight;
@@ -60,25 +59,37 @@
       PhrasePositions pp = new PhrasePositions(tps[i], offsets[i]);
       if (last != null) {			  // add next to end of list
         last.next = pp;
-      } else
+      } else {
         first = pp;
+      }
       last = pp;
     }
 
     pq = new PhraseQueue(tps.length);             // construct empty pq
-
+    first.doc = -1;
   }
 
+  /** @deprecated use {@link #docID()} instead. */
   public int doc() { return first.doc; }
+  
+  public int docID() { return first.doc; }
 
+  /** @deprecated use {@link #nextDoc()} instead. */
   public boolean next() throws IOException {
+    return nextDoc() != NO_MORE_DOCS;
+  }
+
+  public int nextDoc() throws IOException {
     if (firstTime) {
       init();
       firstTime = false;
     } else if (more) {
       more = last.next();                         // trigger further scanning
     }
-    return doNext();
+    if (!doNext()) {
+      first.doc = NO_MORE_DOCS;
+    }
+    return first.doc;
   }
   
   // next without initial increment
@@ -107,16 +118,25 @@
     return norms == null ? raw : raw * Similarity.decodeNorm(norms[first.doc]); // normalize
   }
 
+  /** @deprecated use {@link #advance(int)} instead. */
   public boolean skipTo(int target) throws IOException {
+    return advance(target) != NO_MORE_DOCS;
+  }
+
+  public int advance(int target) throws IOException {
     firstTime = false;
     for (PhrasePositions pp = first; more && pp != null; pp = pp.next) {
       more = pp.skipTo(target);
     }
-    if (more)
+    if (more) {
       sort();                                     // re-sort
-    return doNext();
+    }
+    if (!doNext()) {
+      first.doc = NO_MORE_DOCS;
+    }
+    return first.doc;
   }
-
+  
   /**
    * For a document containing all the phrase query terms, compute the
    * frequency of the phrase in that document. 
@@ -127,16 +147,19 @@
   protected abstract float phraseFreq() throws IOException;
 
   private void init() throws IOException {
-    for (PhrasePositions pp = first; more && pp != null; pp = pp.next) 
+    for (PhrasePositions pp = first; more && pp != null; pp = pp.next) {
       more = pp.next();
-    if(more)
+    }
+    if (more) {
       sort();
+    }
   }
   
   private void sort() {
     pq.clear();
-    for (PhrasePositions pp = first; pp != null; pp = pp.next)
-      pq.put(pp);
+    for (PhrasePositions pp = first; pp != null; pp = pp.next) {
+      pq.add(pp);
+    }
     pqToList();
   }
 
@@ -163,9 +186,8 @@
   public Explanation explain(final int doc) throws IOException {
     Explanation tfExplanation = new Explanation();
 
-    while (next() && doc() < doc) {}
-
-    float phraseFreq = (doc() == doc) ? freq : 0.0f;
+    int d = advance(doc);
+    float phraseFreq = (d == doc) ? freq : 0.0f;
     tfExplanation.setValue(getSimilarity().tf(phraseFreq));
     tfExplanation.setDescription("tf(phraseFreq=" + phraseFreq + ")");
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/ReqExclScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/ReqExclScorer.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/ReqExclScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/ReqExclScorer.java Sun Jun  7 16:58:41 2009
@@ -29,76 +29,78 @@
 class ReqExclScorer extends Scorer {
   private Scorer reqScorer;
   private DocIdSetIterator exclDisi;
+  private int doc = -1;
 
   /** Construct a <code>ReqExclScorer</code>.
    * @param reqScorer The scorer that must match, except where
    * @param exclDisi indicates exclusion.
    */
-  public ReqExclScorer(
-      Scorer reqScorer,
-      DocIdSetIterator exclDisi) {
+  public ReqExclScorer(Scorer reqScorer, DocIdSetIterator exclDisi) {
     super(null); // No similarity used.
     this.reqScorer = reqScorer;
     this.exclDisi = exclDisi;
   }
 
-  private boolean firstTime = true;
-  
+  /** @deprecated use {@link #nextDoc()} instead. */
   public boolean next() throws IOException {
-    if (firstTime) {
-      if (! exclDisi.next()) {
-        exclDisi = null; // exhausted at start
-      }
-      firstTime = false;
-    }
+    return nextDoc() != NO_MORE_DOCS;
+  }
+
+  public int nextDoc() throws IOException {
     if (reqScorer == null) {
-      return false;
+      return doc;
     }
-    if (! reqScorer.next()) {
+    doc = reqScorer.nextDoc();
+    if (doc == NO_MORE_DOCS) {
       reqScorer = null; // exhausted, nothing left
-      return false;
+      return doc;
     }
     if (exclDisi == null) {
-      return true; // reqScorer.next() already returned true
+      return doc;
     }
-    return toNonExcluded();
+    return doc = toNonExcluded();
   }
   
   /** Advance to non excluded doc.
    * <br>On entry:
    * <ul>
    * <li>reqScorer != null,
-   * <li>exclDisi != null,
+   * <li>exclScorer != null,
    * <li>reqScorer was advanced once via next() or skipTo()
    *      and reqScorer.doc() may still be excluded.
    * </ul>
    * Advances reqScorer a non excluded required doc, if any.
    * @return true iff there is a non excluded required doc.
    */
-  private boolean toNonExcluded() throws IOException {
-    int exclDoc = exclDisi.doc();
+  private int toNonExcluded() throws IOException {
+    int exclDoc = exclDisi.docID();
+    int reqDoc = reqScorer.docID(); // may be excluded
     do {  
-      int reqDoc = reqScorer.doc(); // may be excluded
       if (reqDoc < exclDoc) {
-        return true; // reqScorer advanced to before exclScorer, ie. not excluded
+        return reqDoc; // reqScorer advanced to before exclScorer, ie. not excluded
       } else if (reqDoc > exclDoc) {
-        if (! exclDisi.skipTo(reqDoc)) {
+        exclDoc = exclDisi.advance(reqDoc);
+        if (exclDoc == NO_MORE_DOCS) {
           exclDisi = null; // exhausted, no more exclusions
-          return true;
+          return reqDoc;
         }
-        exclDoc = exclDisi.doc();
         if (exclDoc > reqDoc) {
-          return true; // not excluded
+          return reqDoc; // not excluded
         }
       }
-    } while (reqScorer.next());
+    } while ((reqDoc = reqScorer.nextDoc()) != NO_MORE_DOCS);
     reqScorer = null; // exhausted, nothing left
-    return false;
+    return NO_MORE_DOCS;
   }
 
+  /** @deprecated use {@link #docID()} instead. */
   public int doc() {
     return reqScorer.doc(); // reqScorer may be null when next() or skipTo() already return false
   }
+  
+  public int docID() {
+    return doc;
+  }
 
   /** Returns the score of the current document matching the query.
    * Initially invalid, until {@link #next()} is called the first time.
@@ -108,35 +110,28 @@
     return reqScorer.score(); // reqScorer may be null when next() or skipTo() already return false
   }
   
-  /** Skips to the first match beyond the current whose document number is
-   * greater than or equal to a given target.
-   * <br>When this method is used the {@link #explain(int)} method should not be used.
-   * @param target The target document number.
-   * @return true iff there is such a match.
-   */
+  /** @deprecated use {@link #advance(int)} instead. */
   public boolean skipTo(int target) throws IOException {
-    if (firstTime) {
-      firstTime = false;
-      if (! exclDisi.skipTo(target)) {
-        exclDisi = null; // exhausted
-      }
-    }
+    return advance(target) != NO_MORE_DOCS;
+  }
+
+  public int advance(int target) throws IOException {
     if (reqScorer == null) {
-      return false;
+      return doc = NO_MORE_DOCS;
     }
     if (exclDisi == null) {
-      return reqScorer.skipTo(target);
+      return doc = reqScorer.advance(target);
     }
-    if (! reqScorer.skipTo(target)) {
+    if (reqScorer.advance(target) == NO_MORE_DOCS) {
       reqScorer = null;
-      return false;
+      return doc = NO_MORE_DOCS;
     }
-    return toNonExcluded();
+    return doc = toNonExcluded();
   }
-
+  
   public Explanation explain(int doc) throws IOException {
     Explanation res = new Explanation();
-    if (exclDisi.skipTo(doc) && (exclDisi.doc() == doc)) {
+    if (exclDisi.advance(doc) == doc) {
       res.setDescription("excluded");
     } else {
       res.setDescription("not excluded");

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/ReqOptSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/ReqOptSumScorer.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/ReqOptSumScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/ReqOptSumScorer.java Sun Jun  7 16:58:41 2009
@@ -43,44 +43,52 @@
     this.optScorer = optScorer;
   }
 
-  private boolean firstTimeOptScorer = true;
-
+  /** @deprecated use {@link #nextDoc()} instead. */
   public boolean next() throws IOException {
     return reqScorer.next();
   }
 
+  public int nextDoc() throws IOException {
+    return reqScorer.nextDoc();
+  }
+  
+  /** @deprecated use {@link #advance(int)} instead. */
   public boolean skipTo(int target) throws IOException {
     return reqScorer.skipTo(target);
   }
 
+  public int advance(int target) throws IOException {
+    return reqScorer.advance(target);
+  }
+  
+  /** @deprecated use {@link #docID()} instead. */
   public int doc() {
     return reqScorer.doc();
   }
 
+  public int docID() {
+    return reqScorer.docID();
+  }
+  
   /** Returns the score of the current document matching the query.
    * Initially invalid, until {@link #next()} is called the first time.
    * @return The score of the required scorer, eventually increased by the score
    * of the optional scorer when it also matches the current document.
    */
   public float score() throws IOException {
-    int curDoc = reqScorer.doc();
+    int curDoc = reqScorer.docID();
     float reqScore = reqScorer.score();
-    if (firstTimeOptScorer) {
-      firstTimeOptScorer = false;
-      if (! optScorer.skipTo(curDoc)) {
-        optScorer = null;
-        return reqScore;
-      }
-    } else if (optScorer == null) {
+    if (optScorer == null) {
       return reqScore;
-    } else if ((optScorer.doc() < curDoc) && (! optScorer.skipTo(curDoc))) {
+    }
+    
+    int optScorerDoc = optScorer.docID();
+    if (optScorerDoc < curDoc && (optScorerDoc = optScorer.advance(curDoc)) == NO_MORE_DOCS) {
       optScorer = null;
       return reqScore;
     }
-    // assert (optScorer != null) && (optScorer.doc() >= curDoc);
-    return (optScorer.doc() == curDoc)
-       ? reqScore + optScorer.score()
-       : reqScore;
+    
+    return optScorerDoc == curDoc ? reqScore + optScorer.score() : reqScore;
   }
 
   /** Explain the score of a document.

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java Sun Jun  7 16:58:41 2009
@@ -42,8 +42,8 @@
     this.scorer = scorer;
   }
 
-  protected boolean score(Collector collector, int max) throws IOException {
-    return scorer.score(collector, max);
+  protected boolean score(Collector collector, int max, int firstDocID) throws IOException {
+    return scorer.score(collector, max, firstDocID);
   }
 
   public Similarity getSimilarity() {
@@ -55,7 +55,7 @@
   }
 
   public float score() throws IOException {
-    int doc = scorer.doc();
+    int doc = scorer.docID();
     if (doc != curDoc) {
       curScore = scorer.score();
       curDoc = doc;
@@ -64,20 +64,35 @@
     return curScore;
   }
 
+  /** @deprecated use {@link #docID()} instead. */
   public int doc() {
     return scorer.doc();
   }
+  
+  public int docID() {
+    return scorer.docID();
+  }
 
+  /** @deprecated use {@link #nextDoc()} instead. */
   public boolean next() throws IOException {
     return scorer.next();
   }
 
+  public int nextDoc() throws IOException {
+    return scorer.nextDoc();
+  }
+  
   public void score(Collector collector) throws IOException {
     scorer.score(collector);
   }
   
+  /** @deprecated use {@link #advance(int)} instead. */
   public boolean skipTo(int target) throws IOException {
     return scorer.skipTo(target);
   }
 
+  public int advance(int target) throws IOException {
+    return scorer.advance(target);
+  }
+  
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/Scorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/Scorer.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/Scorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/Scorer.java Sun Jun  7 16:58:41 2009
@@ -71,8 +71,9 @@
    */
   public void score(Collector collector) throws IOException {
     collector.setScorer(this);
-    while (next()) {
-      collector.collect(doc());
+    int doc;
+    while ((doc = nextDoc()) != NO_MORE_DOCS) {
+      collector.collect(doc);
     }
   }
 
@@ -86,26 +87,33 @@
    * @deprecated use {@link #score(Collector, int)} instead.
    */
   protected boolean score(HitCollector hc, int max) throws IOException {
-    return score(new HitCollectorWrapper(hc), max);
+    return score(new HitCollectorWrapper(hc), max, docID());
   }
   
-  /** Expert: Collects matching documents in a range.  Hook for optimization.
-   * Note that {@link #next()} must be called once before this method is called
-   * for the first time.
-   * @param collector The collector to which all matching documents are passed.
-   * @param max Do not score documents past this.
+  /**
+   * Expert: Collects matching documents in a range. Hook for optimization.
+   * Note, <code>firstDocID</code> is added to ensure that {@link #nextDoc()}
+   * was called before this method.
+   * 
+   * @param collector
+   *          The collector to which all matching documents are passed.
+   * @param max
+   *          Do not score documents past this.
+   * @param firstDocID
+   *          The first document ID (ensures {@link #nextDoc()} is called before
+   *          this method.
    * @return true if more matching documents may remain.
    */
-  protected boolean score(Collector collector, int max) throws IOException {
+  protected boolean score(Collector collector, int max, int firstDocID) throws IOException {
     collector.setScorer(this);
-    while (doc() < max) {
-      collector.collect(doc());
-      if (!next())
-        return false;
+    int doc = firstDocID;
+    while (doc < max) {
+      collector.collect(doc);
+      doc = nextDoc();
     }
-    return true;
+    return doc == NO_MORE_DOCS;
   }
-
+  
   /** Returns the score of the current document matching the query.
    * Initially invalid, until {@link #next()} or {@link #skipTo(int)}
    * is called the first time, or when called from within

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/TermScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/TermScorer.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/TermScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/TermScorer.java Sun Jun  7 16:58:41 2009
@@ -31,7 +31,7 @@
   private TermDocs termDocs;
   private byte[] norms;
   private float weightValue;
-  private int doc;
+  private int doc = -1;
 
   private final int[] docs = new int[32];         // buffered doc numbers
   private final int[] freqs = new int[32];        // buffered term freqs
@@ -65,16 +65,16 @@
   }
 
   public void score(Collector c) throws IOException {
-    next();
-    score(c, Integer.MAX_VALUE);
+    score(c, Integer.MAX_VALUE, nextDoc());
   }
 
   /** @deprecated use {@link #score(Collector, int)} instead. */
   protected boolean score(HitCollector c, int end) throws IOException {
-    return score(new HitCollectorWrapper(c), end);
+    return score(new HitCollectorWrapper(c), end, doc);
   }
   
-  protected boolean score(Collector c, int end) throws IOException {
+  // firstDocID is ignored since nextDoc() sets 'doc'
+  protected boolean score(Collector c, int end, int firstDocID) throws IOException {
     c.setScorer(this);
     while (doc < end) {                           // for docs in window
       c.collect(doc);                      // collect score
@@ -94,17 +94,31 @@
     return true;
   }
 
-  /** Returns the current document number matching the query.
-   * Initially invalid, until {@link #next()} is called the first time.
-   */
+  /** @deprecated use {@link #docID()} instead. */
   public int doc() { return doc; }
+  
+  public int docID() { return doc; }
 
-  /** Advances to the next document matching the query.
-   * <br>The iterator over the matching documents is buffered using
+  /**
+   * Advances to the next document matching the query. <br>
+   * The iterator over the matching documents is buffered using
    * {@link TermDocs#read(int[],int[])}.
+   * 
    * @return true iff there is another document matching the query.
+   * @deprecated use {@link #nextDoc()} instead.
    */
   public boolean next() throws IOException {
+    return nextDoc() != NO_MORE_DOCS;
+  }
+
+  /**
+   * Advances to the next document matching the query. <br>
+   * The iterator over the matching documents is buffered using
+   * {@link TermDocs#read(int[],int[])}.
+   * 
+   * @return the document matching the query or -1 if there are no more documents.
+   */
+  public int nextDoc() throws IOException {
     pointer++;
     if (pointer >= pointerMax) {
       pointerMax = termDocs.read(docs, freqs);    // refill buffer
@@ -112,15 +126,15 @@
         pointer = 0;
       } else {
         termDocs.close();                         // close stream
-        doc = Integer.MAX_VALUE;                  // set to sentinel value
-        return false;
+        return doc = NO_MORE_DOCS;
       }
     } 
     doc = docs[pointer];
-    return true;
+    return doc;
   }
-
+  
   public float score() {
+    assert doc != -1;
     int f = freqs[pointer];
     float raw =                                   // compute tf(f)*weight
       f < SCORE_CACHE_SIZE                        // check cache
@@ -130,18 +144,34 @@
     return norms == null ? raw : raw * SIM_NORM_DECODER[norms[doc] & 0xFF]; // normalize for field
   }
 
-  /** Skips to the first match beyond the current whose document number is
-   * greater than or equal to a given target. 
-   * <br>The implementation uses {@link TermDocs#skipTo(int)}.
-   * @param target The target document number.
+  /**
+   * Skips to the first match beyond the current whose document number is
+   * greater than or equal to a given target. <br>
+   * The implementation uses {@link TermDocs#skipTo(int)}.
+   * 
+   * @param target
+   *          The target document number.
    * @return true iff there is such a match.
+   * @deprecated use {@link #advance(int)} instead.
    */
   public boolean skipTo(int target) throws IOException {
+    return advance(target) != NO_MORE_DOCS;
+  }
+
+  /**
+   * Advances to the first match beyond the current whose document number is
+   * greater than or equal to a given target. <br>
+   * The implementation uses {@link TermDocs#skipTo(int)}.
+   * 
+   * @param target
+   *          The target document number.
+   * @return the matching document or -1 if none exist.
+   */
+  public int advance(int target) throws IOException {
     // first scan in cache
     for (pointer++; pointer < pointerMax; pointer++) {
       if (docs[pointer] >= target) {
-        doc = docs[pointer];
-        return true;
+        return doc = docs[pointer];
       }
     }
 
@@ -153,11 +183,11 @@
       docs[pointer] = doc = termDocs.doc();
       freqs[pointer] = termDocs.freq();
     } else {
-      doc = Integer.MAX_VALUE;
+      doc = NO_MORE_DOCS;
     }
-    return result;
+    return doc;
   }
-
+  
   /** Returns an explanation of the score for a document.
    * <br>When this method is used, the {@link #next()} method
    * and the {@link #score(HitCollector)} method should not be used.

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/function/CustomScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/function/CustomScoreQuery.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/function/CustomScoreQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/function/CustomScoreQuery.java Sun Jun  7 16:58:41 2009
@@ -367,41 +367,53 @@
       this.vScores = new float[valSrcScorers.length];
     }
 
-    /*(non-Javadoc) @see org.apache.lucene.search.Scorer#next() */
+    /** @deprecated use {@link #nextDoc()} instead. */
     public boolean next() throws IOException {
-      boolean hasNext = subQueryScorer.next();
-      if (hasNext) {
-        for(int i = 0; i < valSrcScorers.length; i++) {
-          valSrcScorers[i].skipTo(subQueryScorer.doc());  
+      return nextDoc() != NO_MORE_DOCS;
+    }
+
+    public int nextDoc() throws IOException {
+      int doc = subQueryScorer.nextDoc();
+      if (doc != NO_MORE_DOCS) {
+        for (int i = 0; i < valSrcScorers.length; i++) {
+          valSrcScorers[i].advance(doc);
         }
       }
-      return hasNext;
+      return doc;
     }
 
-    /*(non-Javadoc) @see org.apache.lucene.search.Scorer#doc() */
+    /** @deprecated use {@link #docID()} instead. */
     public int doc() {
       return subQueryScorer.doc();
     }
 
+    public int docID() {
+      return subQueryScorer.docID();
+    }
+    
     /*(non-Javadoc) @see org.apache.lucene.search.Scorer#score() */
     public float score() throws IOException {
-      for(int i = 0; i < valSrcScorers.length; i++) {
+      for (int i = 0; i < valSrcScorers.length; i++) {
         vScores[i] = valSrcScorers[i].score();
       }
-      return qWeight * customScore(subQueryScorer.doc(), subQueryScorer.score(), vScores);
+      return qWeight * customScore(subQueryScorer.docID(), subQueryScorer.score(), vScores);
     }
 
-    /*(non-Javadoc) @see org.apache.lucene.search.Scorer#skipTo(int) */
+    /** @deprecated use {@link #advance(int)} instead. */
     public boolean skipTo(int target) throws IOException {
-      boolean hasNext = subQueryScorer.skipTo(target);
-      if (hasNext) {
-      	for (int i = 0; i < valSrcScorers.length; i++) {
-          valSrcScorers[i].skipTo(subQueryScorer.doc());
+      return advance(target) != NO_MORE_DOCS;
+    }
+
+    public int advance(int target) throws IOException {
+      int doc = subQueryScorer.advance(target);
+      if (doc != NO_MORE_DOCS) {
+        for (int i = 0; i < valSrcScorers.length; i++) {
+          valSrcScorers[i].advance(doc);
         }
       }
-      return hasNext;
+      return doc;
     }
-
+    
     /*(non-Javadoc) @see org.apache.lucene.search.Scorer#explain(int) */
     public Explanation explain(int doc) throws IOException {
       Explanation subQueryExpl = weight.subQueryWeight.explain(reader,doc);

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/function/ValueSourceQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/function/ValueSourceQuery.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/function/ValueSourceQuery.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/function/ValueSourceQuery.java Sun Jun  7 16:58:41 2009
@@ -115,6 +115,7 @@
     private final float qWeight;
     private final DocValues vals;
     private final TermDocs termDocs;
+    private int doc = -1;
 
     // constructor
     private ValueSourceScorer(Similarity similarity, IndexReader reader, ValueSourceWeight w) throws IOException {
@@ -126,26 +127,37 @@
       termDocs = reader.termDocs(null);
     }
 
-    /*(non-Javadoc) @see org.apache.lucene.search.Scorer#next() */
+    /** @deprecated use {@link #nextDoc()} instead. */
     public boolean next() throws IOException {
       return termDocs.next();
     }
 
-    /*(non-Javadoc) @see org.apache.lucene.search.Scorer#doc()
-     */
+    public int nextDoc() throws IOException {
+      return doc = termDocs.next() ? termDocs.doc() : NO_MORE_DOCS;
+    }
+    
+    /** @deprecated use {@link #docID()} instead. */
     public int doc() {
       return termDocs.doc();
     }
 
+    public int docID() {
+      return doc;
+    }
+    
     /*(non-Javadoc) @see org.apache.lucene.search.Scorer#score() */
     public float score() throws IOException {
       return qWeight * vals.floatVal(termDocs.doc());
     }
 
-    /*(non-Javadoc) @see org.apache.lucene.search.Scorer#skipTo(int) */
+    /** @deprecated use {@link #advance(int)} instead. */
     public boolean skipTo(int target) throws IOException {
       return termDocs.skipTo(target);
     }
+    
+    public int advance(int target) throws IOException {
+      return doc = termDocs.skipTo(target) ? termDocs.doc() : NO_MORE_DOCS;
+    }
 
     /*(non-Javadoc) @see org.apache.lucene.search.Scorer#explain(int) */
     public Explanation explain(int doc) throws IOException {

Modified: lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanScorer.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanScorer.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/search/spans/SpanScorer.java Sun Jun  7 16:58:41 2009
@@ -33,6 +33,7 @@
   protected byte[] norms;
   protected float value;
 
+  /** @deprecated not needed anymore */
   protected boolean firstTime = true;
   protected boolean more = true;
 
@@ -46,33 +47,46 @@
     this.norms = norms;
     this.weight = weight;
     this.value = weight.getValue();
-    doc = -1;
+    if (this.spans.next()) {
+      doc = -1;
+    } else {
+      doc = NO_MORE_DOCS;
+      more = false;
+    }
   }
 
+  /** @deprecated use {@link #nextDoc()} instead. */
   public boolean next() throws IOException {
-    if (firstTime) {
-      more = spans.next();
-      firstTime = false;
+    return nextDoc() != NO_MORE_DOCS;
+  }
+
+  public int nextDoc() throws IOException {
+    if (!setFreqCurrentDoc()) {
+      doc = NO_MORE_DOCS;
     }
-    return setFreqCurrentDoc();
+    return doc;
   }
 
+  /** @deprecated use {@link #advance(int)} instead. */
   public boolean skipTo(int target) throws IOException {
-    if (firstTime) {
-      more = spans.skipTo(target);
-      firstTime = false;
-    }
-    if (! more) {
-      return false;
+    return advance(target) != NO_MORE_DOCS;
+  }
+
+  public int advance(int target) throws IOException {
+    if (!more) {
+      return doc = NO_MORE_DOCS;
     }
     if (spans.doc() < target) { // setFreqCurrentDoc() leaves spans.doc() ahead
       more = spans.skipTo(target);
     }
-    return setFreqCurrentDoc();
+    if (!setFreqCurrentDoc()) {
+      doc = NO_MORE_DOCS;
+    }
+    return doc;
   }
-
+  
   protected boolean setFreqCurrentDoc() throws IOException {
-    if (! more) {
+    if (!more) {
       return false;
     }
     doc = spans.doc();
@@ -85,7 +99,10 @@
     return true;
   }
 
+  /** @deprecated use {@link #docID()} instead. */
   public int doc() { return doc; }
+  
+  public int docID() { return doc; }
 
   public float score() throws IOException {
     float raw = getSimilarity().tf(freq) * value; // raw score
@@ -95,9 +112,9 @@
   public Explanation explain(final int doc) throws IOException {
     Explanation tfExplanation = new Explanation();
 
-    skipTo(doc);
+    int expDoc = advance(doc);
 
-    float phraseFreq = (doc() == doc) ? freq : 0.0f;
+    float phraseFreq = (expDoc == doc) ? freq : 0.0f;
     tfExplanation.setValue(getSimilarity().tf(phraseFreq));
     tfExplanation.setDescription("tf(phraseFreq=" + phraseFreq + ")");
 

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/DocIdBitSet.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/DocIdBitSet.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/DocIdBitSet.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/DocIdBitSet.java Sun Jun  7 16:58:41 2009
@@ -18,6 +18,7 @@
  */
 
 import java.util.BitSet;
+
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 
@@ -50,28 +51,40 @@
       this.docId = -1;
     }
     
+    /** @deprecated use {@link #docID()} instead. */
     public int doc() {
       assert docId != -1;
       return docId;
     }
     
+    public int docID() {
+      return docId;
+    }
+    
+    /** @deprecated use {@link #nextDoc()} instead. */
     public boolean next() {
       // (docId + 1) on next line requires -1 initial value for docNr:
-      return checkNextDocId(bitSet.nextSetBit(docId + 1));
+      return nextDoc() != NO_MORE_DOCS;
+    }
+    
+    public int nextDoc() {
+      // (docId + 1) on next line requires -1 initial value for docNr:
+      int d = bitSet.nextSetBit(docId + 1);
+      // -1 returned by BitSet.nextSetBit() when exhausted
+      docId = d == -1 ? NO_MORE_DOCS : d;
+      return docId;
     }
   
+    /** @deprecated use {@link #advance(int)} instead. */
     public boolean skipTo(int skipDocNr) {
-      return checkNextDocId( bitSet.nextSetBit(skipDocNr));
+      return advance(skipDocNr) != NO_MORE_DOCS;
     }
   
-    private boolean checkNextDocId(int d) {
-      if (d == -1) { // -1 returned by BitSet.nextSetBit() when exhausted
-        docId = Integer.MAX_VALUE;
-        return false;
-      } else {
-        docId = d;
-        return true;
-      }
+    public int advance(int target) {
+      int d = bitSet.nextSetBit(target);
+      // -1 returned by BitSet.nextSetBit() when exhausted
+      docId = d == -1 ? NO_MORE_DOCS : d;
+      return docId;
     }
   }
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/OpenBitSetDISI.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/OpenBitSetDISI.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/OpenBitSetDISI.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/OpenBitSetDISI.java Sun Jun  7 16:58:41 2009
@@ -47,8 +47,10 @@
    * constructor.
    */   
   public void inPlaceOr(DocIdSetIterator disi) throws IOException {
-    while (disi.next() && (disi.doc() < size())) {
-      fastSet(disi.doc());
+    int doc;
+    long size = size();
+    while ((doc = disi.nextDoc()) < size) {
+      fastSet(doc);
     }
   }
 
@@ -60,8 +62,8 @@
    */   
   public void inPlaceAnd(DocIdSetIterator disi) throws IOException {
     int bitSetDoc = nextSetBit(0);
-    while ((bitSetDoc != -1) && disi.skipTo(bitSetDoc)) {
-      int disiDoc = disi.doc();
+    int disiDoc;
+    while (bitSetDoc != -1 && (disiDoc = disi.advance(bitSetDoc)) != DocIdSetIterator.NO_MORE_DOCS) {
       clear(bitSetDoc, disiDoc);
       bitSetDoc = nextSetBit(disiDoc + 1);
     }
@@ -77,8 +79,10 @@
    * constructor.
    */   
   public void inPlaceNot(DocIdSetIterator disi) throws IOException {
-    while (disi.next() && (disi.doc() < size())) {
-      fastClear(disi.doc());
+    int doc;
+    long size = size();
+    while ((doc = disi.nextDoc()) < size) {
+      fastClear(doc);
     }
   }
 
@@ -89,8 +93,10 @@
    * constructor.
    */   
   public void inPlaceXor(DocIdSetIterator disi) throws IOException {
-    while (disi.next() && (disi.doc() < size())) {
-      fastFlip(disi.doc());
+    int doc;
+    long size = size();
+    while ((doc = disi.nextDoc()) < size) {
+      fastFlip(doc);
     }
   }
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/OpenBitSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/OpenBitSetIterator.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/OpenBitSetIterator.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/OpenBitSetIterator.java Sun Jun  7 16:58:41 2009
@@ -17,8 +17,6 @@
 
 package org.apache.lucene.util;
 
-import java.io.IOException;
-
 import org.apache.lucene.search.DocIdSetIterator;
 
 /** An iterator to iterate over set bits in an OpenBitSet.
@@ -35,7 +33,34 @@
   // should be faster than accessing an array for each index, and
   // the total array size is kept smaller (256*sizeof(int))=1K
   protected final static int[] bitlist={
-    0x0,0x1,0x2,0x21,0x3,0x31,0x32,0x321,0x4,0x41,0x42,0x421,0x43,0x431,0x432,0x4321,0x5,0x51,0x52,0x521,0x53,0x531,0x532,0x5321,0x54,0x541,0x542,0x5421,0x543,0x5431,0x5432,0x54321,0x6,0x61,0x62,0x621,0x63,0x631,0x632,0x6321,0x64,0x641,0x642,0x6421,0x643,0x6431,0x6432,0x64321,0x65,0x651,0x652,0x6521,0x653,0x6531,0x6532,0x65321,0x654,0x6541,0x6542,0x65421,0x6543,0x65431,0x65432,0x654321,0x7,0x71,0x72,0x721,0x73,0x731,0x732,0x7321,0x74,0x741,0x742,0x7421,0x743,0x7431,0x7432,0x74321,0x75,0x751,0x752,0x7521,0x753,0x7531,0x7532,0x75321,0x754,0x7541,0x7542,0x75421,0x7543,0x75431,0x75432,0x754321,0x76,0x761,0x762,0x7621,0x763,0x7631,0x7632,0x76321,0x764,0x7641,0x7642,0x76421,0x7643,0x76431,0x76432,0x764321,0x765,0x7651,0x7652,0x76521,0x7653,0x76531,0x76532,0x765321,0x7654,0x76541,0x76542,0x765421,0x76543,0x765431,0x765432,0x7654321,0x8,0x81,0x82,0x821,0x83,0x831,0x832,0x8321,0x84,0x841,0x842,0x8421,0x843,0x8431,0x8432,0x84321,0x85,0x851,0x852,0x8521,0x853,0x8531,0x8532,0x85321,0x85
 4,0x8541,0x8542,0x85421,0x8543,0x85431,0x85432,0x854321,0x86,0x861,0x862,0x8621,0x863,0x8631,0x8632,0x86321,0x864,0x8641,0x8642,0x86421,0x8643,0x86431,0x86432,0x864321,0x865,0x8651,0x8652,0x86521,0x8653,0x86531,0x86532,0x865321,0x8654,0x86541,0x86542,0x865421,0x86543,0x865431,0x865432,0x8654321,0x87,0x871,0x872,0x8721,0x873,0x8731,0x8732,0x87321,0x874,0x8741,0x8742,0x87421,0x8743,0x87431,0x87432,0x874321,0x875,0x8751,0x8752,0x87521,0x8753,0x87531,0x87532,0x875321,0x8754,0x87541,0x87542,0x875421,0x87543,0x875431,0x875432,0x8754321,0x876,0x8761,0x8762,0x87621,0x8763,0x87631,0x87632,0x876321,0x8764,0x87641,0x87642,0x876421,0x87643,0x876431,0x876432,0x8764321,0x8765,0x87651,0x87652,0x876521,0x87653,0x876531,0x876532,0x8765321,0x87654,0x876541,0x876542,0x8765421,0x876543,0x8765431,0x8765432,0x87654321
+    0x0, 0x1, 0x2, 0x21, 0x3, 0x31, 0x32, 0x321, 0x4, 0x41, 0x42, 0x421, 0x43, 
+    0x431, 0x432, 0x4321, 0x5, 0x51, 0x52, 0x521, 0x53, 0x531, 0x532, 0x5321, 
+    0x54, 0x541, 0x542, 0x5421, 0x543, 0x5431, 0x5432, 0x54321, 0x6, 0x61, 0x62, 
+    0x621, 0x63, 0x631, 0x632, 0x6321, 0x64, 0x641, 0x642, 0x6421, 0x643, 
+    0x6431, 0x6432, 0x64321, 0x65, 0x651, 0x652, 0x6521, 0x653, 0x6531, 0x6532, 
+    0x65321, 0x654, 0x6541, 0x6542, 0x65421, 0x6543, 0x65431, 0x65432, 0x654321, 
+    0x7, 0x71, 0x72, 0x721, 0x73, 0x731, 0x732, 0x7321, 0x74, 0x741, 0x742,
+    0x7421, 0x743, 0x7431, 0x7432, 0x74321, 0x75, 0x751, 0x752, 0x7521, 0x753, 
+    0x7531, 0x7532, 0x75321, 0x754, 0x7541, 0x7542, 0x75421, 0x7543, 0x75431, 
+    0x75432, 0x754321, 0x76, 0x761, 0x762, 0x7621, 0x763, 0x7631, 0x7632, 
+    0x76321, 0x764, 0x7641, 0x7642, 0x76421, 0x7643, 0x76431, 0x76432, 0x764321, 
+    0x765, 0x7651, 0x7652, 0x76521, 0x7653, 0x76531, 0x76532, 0x765321, 0x7654, 
+    0x76541, 0x76542, 0x765421, 0x76543, 0x765431, 0x765432, 0x7654321, 0x8, 
+    0x81, 0x82, 0x821, 0x83, 0x831, 0x832, 0x8321, 0x84, 0x841, 0x842, 0x8421, 
+    0x843, 0x8431, 0x8432, 0x84321, 0x85, 0x851, 0x852, 0x8521, 0x853, 0x8531, 
+    0x8532, 0x85321, 0x854, 0x8541, 0x8542, 0x85421, 0x8543, 0x85431, 0x85432, 
+    0x854321, 0x86, 0x861, 0x862, 0x8621, 0x863, 0x8631, 0x8632, 0x86321, 0x864, 
+    0x8641, 0x8642, 0x86421, 0x8643, 0x86431, 0x86432, 0x864321, 0x865, 0x8651, 
+    0x8652, 0x86521, 0x8653, 0x86531, 0x86532, 0x865321, 0x8654, 0x86541, 
+    0x86542, 0x865421, 0x86543, 0x865431, 0x865432, 0x8654321, 0x87, 0x871, 
+    0x872, 0x8721, 0x873, 0x8731, 0x8732, 0x87321, 0x874, 0x8741, 0x8742, 
+    0x87421, 0x8743, 0x87431, 0x87432, 0x874321, 0x875, 0x8751, 0x8752, 0x87521, 
+    0x8753, 0x87531, 0x87532, 0x875321, 0x8754, 0x87541, 0x87542, 0x875421, 
+    0x87543, 0x875431, 0x875432, 0x8754321, 0x876, 0x8761, 0x8762, 0x87621, 
+    0x8763, 0x87631, 0x87632, 0x876321, 0x8764, 0x87641, 0x87642, 0x876421, 
+    0x87643, 0x876431, 0x876432, 0x8764321, 0x8765, 0x87651, 0x87652, 0x876521, 
+    0x87653, 0x876531, 0x876532, 0x8765321, 0x87654, 0x876541, 0x876542, 
+    0x8765421, 0x876543, 0x8765431, 0x8765432, 0x87654321
   };
   /***** the python code that generated bitlist
   def bits2int(val):
@@ -56,14 +81,13 @@
   // for efficiency, or have a common root interface?  (or
   // maybe both?  could ask for a SetBitsIterator, etc...
 
-
   private final long[] arr;
   private final int words;
   private int i=-1;
   private long word;
   private int wordShift;
   private int indexArray;
-  private int curDocId;
+  private int curDocId = -1;
 
   public OpenBitSetIterator(OpenBitSet obs) {
     this(obs.getBits(), obs.getNumWords());
@@ -104,50 +128,24 @@
   }
   ******/
 
+  /** @deprecated use {@link #nextDoc()} instead. */
   public boolean next() {
-    if (indexArray==0) {
-      if (word!=0) {
-        word >>>= 8;
-        wordShift += 8;
-      }
-
-      while (word==0) {
-        if (++i >= words) {
-          curDocId = -1;
-          return false;
-        }
-        word = arr[i];
-        wordShift =-1;  // loop invariant code motion should move this
-      }
-
-      // after the first time, should I go with a linear search, or
-      // stick with the binary search in shift?
-      shift();
-    }
-
-    int bitIndex = (indexArray & 0x0f) + wordShift;
-    indexArray >>>= 4;
-    // should i<<6 be cached as a separate variable?
-    // it would only save one cycle in the best circumstances.
-    curDocId = (i<<6) + bitIndex;
-    return true;
+    return nextDoc() != NO_MORE_DOCS;
   }
 
-  /** Moves iterator to the next doc and returns its id;
-      returns -1 when this iterator is exhausted. */
   public int nextDoc() {
-    if (indexArray==0) {
-      if (word!=0) {
+    if (indexArray == 0) {
+      if (word != 0) {
         word >>>= 8;
         wordShift += 8;
       }
 
-      while (word==0) {
+      while (word == 0) {
         if (++i >= words) {
-          return curDocId = -1;
+          return curDocId = NO_MORE_DOCS;
         }
         word = arr[i];
-        wordShift =-1;  // loop invariant code motion should move this
+        wordShift = -1; // loop invariant code motion should move this
       }
 
       // after the first time, should I go with a linear search, or
@@ -162,60 +160,30 @@
     return curDocId = (i<<6) + bitIndex;
   }
   
+  /** @deprecated use {@link #advance(int)} instead. */
   public boolean skipTo(int target) {
-    indexArray=0;
+    return advance(target) != NO_MORE_DOCS;
+  }
+
+  public int advance(int target) {
+    indexArray = 0;
     i = target >> 6;
-    if (i>=words) {
-      word =0; // setup so next() will also return -1
-      curDocId = -1;
-      return false;
+    if (i >= words) {
+      word = 0; // setup so next() will also return -1
+      return curDocId = NO_MORE_DOCS;
     }
     wordShift = target & 0x3f;
     word = arr[i] >>> wordShift;
-    if (word !=0) {
+    if (word != 0) {
       wordShift--; // compensate for 1 based arrIndex
     } else {
-      while (word ==0) {
+      while (word == 0) {
         if (++i >= words) {
-          curDocId = -1;
-          return false;
+          return curDocId = NO_MORE_DOCS;
         }
         word = arr[i];
       }
-      wordShift =-1;
-    }
-
-    shift();
-
-    int bitIndex = (indexArray & 0x0f) + wordShift;
-    indexArray >>>= 4;
-    // should i<<6 be cached as a separate variable?
-    // it would only save one cycle in the best circumstances.
-    curDocId = (i<<6) + bitIndex;
-    return true;
-  }
-  
-  /** Behaves like {@link #skipTo(int)} and returns the docId the iterator
-   *  skipped to; returns -1 if no valid document could be skipped to. */
-  public int next(int fromIndex) {
-    indexArray=0;
-    i = fromIndex >> 6;
-    if (i>=words) {
-      word =0; // setup so next() will also return -1
-      return curDocId = -1;
-    }
-    wordShift = fromIndex & 0x3f;
-    word = arr[i] >>> wordShift;
-    if (word !=0) {
-      wordShift--; // compensate for 1 based arrIndex
-    } else {
-      while (word ==0) {
-        if (++i >= words) {
-          return curDocId = -1;
-        }
-        word = arr[i];
-      }
-      wordShift =-1;
+      wordShift = -1;
     }
 
     shift();
@@ -226,9 +194,14 @@
     // it would only save one cycle in the best circumstances.
     return curDocId = (i<<6) + bitIndex;
   }
-  
+
+  /** @deprecated use {@link #docID()} instead. */
   public int doc() {
-    return this.curDocId;
+    return curDocId;
   }
 
+  public int docID() {
+    return curDocId;
+  }
+  
 }

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/ScorerDocQueue.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/ScorerDocQueue.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/ScorerDocQueue.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/ScorerDocQueue.java Sun Jun  7 16:58:41 2009
@@ -20,6 +20,8 @@
 /* Derived from org.apache.lucene.util.PriorityQueue of March 2005 */
 
 import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Scorer;
 
 /** A ScorerDocQueue maintains a partial ordering of its Scorers such that the
@@ -35,14 +37,14 @@
     Scorer scorer;
     int doc;
     
-    HeapedScorerDoc(Scorer s) { this(s, s.doc()); }
+    HeapedScorerDoc(Scorer s) { this(s, s.docID()); }
     
     HeapedScorerDoc(Scorer scorer, int doc) {
       this.scorer = scorer;
       this.doc = doc;
     }
     
-    void adjust() { doc = scorer.doc(); }
+    void adjust() { doc = scorer.docID(); }
   }
   
   private HeapedScorerDoc topHSD; // same as heap[1], only for speed
@@ -79,7 +81,7 @@
       put(scorer);
       return true;
     } else {
-      int docNr = scorer.doc();
+      int docNr = scorer.docID();
       if ((size > 0) && (! (docNr < topHSD.doc))) { // heap[1] is top()
         heap[1] = new HeapedScorerDoc(scorer, docNr);
         downHeap();
@@ -113,16 +115,16 @@
   }
 
   public final boolean topNextAndAdjustElsePop() throws IOException {
-    return checkAdjustElsePop( topHSD.scorer.next());
+    return checkAdjustElsePop(topHSD.scorer.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
   }
 
   public final boolean topSkipToAndAdjustElsePop(int target) throws IOException {
-    return checkAdjustElsePop( topHSD.scorer.skipTo(target));
+    return checkAdjustElsePop(topHSD.scorer.advance(target) != DocIdSetIterator.NO_MORE_DOCS);
   }
   
   private boolean checkAdjustElsePop(boolean cond) {
     if (cond) { // see also adjustTop
-      topHSD.doc = topHSD.scorer.doc();
+      topHSD.doc = topHSD.scorer.docID();
     } else { // see also popNoResult
       heap[1] = heap[size]; // move last to first
       heap[size] = null;

Modified: lucene/java/trunk/src/java/org/apache/lucene/util/SortedVIntList.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/util/SortedVIntList.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/util/SortedVIntList.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/util/SortedVIntList.java Sun Jun  7 16:58:41 2009
@@ -24,10 +24,15 @@
 import org.apache.lucene.search.DocIdSetIterator;
 
 /**
- *  Store and iterate sorted integers in compressed form in RAM.
- *  <br>The code for compressing the differences between ascending integers was
- *  borrowed from {@link org.apache.lucene.store.IndexInput} and
- *  {@link org.apache.lucene.store.IndexOutput}.
+ * Stores and iterate on sorted integers in compressed form in RAM. <br>
+ * The code for compressing the differences between ascending integers was
+ * borrowed from {@link org.apache.lucene.store.IndexInput} and
+ * {@link org.apache.lucene.store.IndexOutput}.
+ * <p>
+ * <b>NOTE:</b> this class assumes the stored integers are doc Ids (hence why it
+ * extends {@link DocIdSet}). Therefore its {@link #iterator()} assumes {@value
+ * DocIdSetIterator#NO_MORE_DOCS} can be used as sentinel. If you intent to use
+ * this value, then make sure it's not used during search flow.
  */
 public class SortedVIntList extends DocIdSet {
   /** When a BitSet has fewer than 1 in BITS2VINTLIST_SIZE bits set,
@@ -99,8 +104,9 @@
    */
   public SortedVIntList(DocIdSetIterator docIdSetIterator) throws IOException {
     SortedVIntListBuilder builder = new SortedVIntListBuilder();
-    while (docIdSetIterator.next()) {
-      builder.addInt(docIdSetIterator.doc());
+    int doc;
+    while ((doc = docIdSetIterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+      builder.addInt(doc);
     }
     builder.done();
   }
@@ -181,6 +187,7 @@
     return new DocIdSetIterator() {
       int bytePos = 0;
       int lastInt = 0;
+      int doc = -1;
       
       private void advance() {
         // See org.apache.lucene.store.IndexInput.readVInt()
@@ -192,26 +199,43 @@
         }
       }
       
+      /** @deprecated use {@link #docID()} instead. */
       public int doc() {return lastInt;}
       
+      public int docID() {
+        return doc;
+      }
+      
+      /** @deprecated use {@link #nextDoc()} instead. */
       public boolean next() {
+        return nextDoc() != NO_MORE_DOCS;
+      }
+
+      public int nextDoc() {
         if (bytePos >= lastBytePos) {
-          return false;
+          doc = NO_MORE_DOCS;
         } else {
           advance();
-          return true;
+          doc = lastInt;
         }
+        return doc;
       }
-
+      
+      /** @deprecated use {@link #advance(int)} instead. */
       public boolean skipTo(int docNr) {
+        return advance(docNr) != NO_MORE_DOCS;
+      }
+      
+      public int advance(int target) {
         while (bytePos < lastBytePos) {
           advance();
-          if (lastInt >= docNr) { // No skipping to docNr available.
-            return true;
+          if (lastInt >= target) {
+            return doc = lastInt;
           }
         }
-        return false;
+        return doc = NO_MORE_DOCS;
       }
+      
     };
   }
 }

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/JustCompileSearch.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/JustCompileSearch.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/JustCompileSearch.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/JustCompileSearch.java Sun Jun  7 16:58:41 2009
@@ -175,18 +175,32 @@
 
   static final class JustCompileDocIdSetIterator extends DocIdSetIterator {
 
+    /** @deprecated delete in 3.0 */
     public int doc() {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
+    
+    public int docID() {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+    }
 
+    /** @deprecated delete in 3.0 */
     public boolean next() throws IOException {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 
+    /** @deprecated delete in 3.0 */
     public boolean skipTo(int target) throws IOException {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
     
+    public int nextDoc() throws IOException {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+    }
+    
+    public int advance(int target) throws IOException {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+    }
   }
   
   static final class JustCompileFieldCache implements FieldCache {
@@ -470,6 +484,11 @@
       super(similarity);
     }
 
+    protected boolean score(Collector collector, int max, int firstDocID)
+        throws IOException {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+    }
+    
     public Explanation explain(int doc) throws IOException {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
@@ -478,18 +497,32 @@
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 
+    /** @deprecated delete in 3.0 */
     public int doc() {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
+    
+    public int docID() {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+    }
 
+    /** @deprecated delete in 3.0. */
     public boolean next() throws IOException {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 
+    /** @deprecated delete in 3.0. */
     public boolean skipTo(int target) throws IOException {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
+
+    public int nextDoc() throws IOException {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+    }
     
+    public int advance(int target) throws IOException {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+    }
   }
   
   static final class JustCompileSimilarity extends Similarity {

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/QueryUtils.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/QueryUtils.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/QueryUtils.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/QueryUtils.java Sun Jun  7 16:58:41 2009
@@ -1,13 +1,13 @@
 package org.apache.lucene.search;
 
-import junit.framework.TestCase;
-
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 
+import junit.framework.Assert;
+
 import org.apache.lucene.index.IndexReader;
 
 /**
@@ -56,17 +56,17 @@
   }
 
   public static void checkEqual(Query q1, Query q2) {
-    TestCase.assertEquals(q1, q2);
-    TestCase.assertEquals(q1.hashCode(), q2.hashCode());
+    Assert.assertEquals(q1, q2);
+    Assert.assertEquals(q1.hashCode(), q2.hashCode());
   }
 
   public static void checkUnequal(Query q1, Query q2) {
-    TestCase.assertTrue(!q1.equals(q2));
-    TestCase.assertTrue(!q2.equals(q1));
+    Assert.assertTrue(!q1.equals(q2));
+    Assert.assertTrue(!q2.equals(q1));
 
     // possible this test can fail on a hash collision... if that
     // happens, please change test to use a different example.
-    TestCase.assertTrue(q1.hashCode() != q2.hashCode());
+    Assert.assertTrue(q1.hashCode() != q2.hashCode());
   }
   
   /** deep check that explanations of a query 'score' correctly */
@@ -169,8 +169,9 @@
           try {
             int op = order[(opidx[0]++)%order.length];
             //System.out.println(op==skip_op ? "skip("+(sdoc[0]+1)+")":"next()");
-            boolean more = op==skip_op ? scorer.skipTo(sdoc[0]+1) : scorer.next();
-            sdoc[0] = scorer.doc();
+            boolean more = op == skip_op ? scorer.advance(sdoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS
+                : scorer.nextDoc() != DocIdSetIterator.NO_MORE_DOCS;
+            sdoc[0] = scorer.docID();
             float scorerScore = scorer.score();
             float scorerScore2 = scorer.score();
             float scoreDiff = Math.abs(score-scorerScore);
@@ -204,8 +205,9 @@
       // make sure next call to scorer is false.
       int op = order[(opidx[0]++)%order.length];
       //System.out.println(op==skip_op ? "last: skip()":"last: next()");
-      boolean more = op==skip_op ? scorer.skipTo(sdoc[0]+1) : scorer.next();
-      TestCase.assertFalse(more);
+      boolean more = (op == skip_op ? scorer.advance(sdoc[0] + 1) : scorer
+          .nextDoc()) != DocIdSetIterator.NO_MORE_DOCS;
+      Assert.assertFalse(more);
     }
   }
     
@@ -228,11 +230,11 @@
           for (int i=lastDoc[0]+1; i<=doc; i++) {
             Weight w = q.weight(s);
             Scorer scorer = w.scorer(s.getIndexReader());
-            TestCase.assertTrue("query collected "+doc+" but skipTo("+i+") says no more docs!",scorer.skipTo(i));
-            TestCase.assertEquals("query collected "+doc+" but skipTo("+i+") got to "+scorer.doc(),doc,scorer.doc());
+            Assert.assertTrue("query collected "+doc+" but skipTo("+i+") says no more docs!",scorer.advance(i) != DocIdSetIterator.NO_MORE_DOCS);
+            Assert.assertEquals("query collected "+doc+" but skipTo("+i+") got to "+scorer.docID(),doc,scorer.docID());
             float skipToScore = scorer.score();
-            TestCase.assertEquals("unstable skipTo("+i+") score!",skipToScore,scorer.score(),maxDiff); 
-            TestCase.assertEquals("query assigned doc "+doc+" a score of <"+score+"> but skipTo("+i+") has <"+skipToScore+">!",score,skipToScore,maxDiff);
+            Assert.assertEquals("unstable skipTo("+i+") score!",skipToScore,scorer.score(),maxDiff); 
+            Assert.assertEquals("query assigned doc "+doc+" a score of <"+score+"> but skipTo("+i+") has <"+skipToScore+">!",score,skipToScore,maxDiff);
           }
           lastDoc[0] = doc;
         } catch (IOException e) {
@@ -245,8 +247,8 @@
     });
     Weight w = q.weight(s);
     Scorer scorer = w.scorer(s.getIndexReader());
-    boolean more = scorer.skipTo(lastDoc[0]+1);
+    boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS;
     if (more) 
-      TestCase.assertFalse("query's last doc was "+lastDoc[0]+" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.doc(),more);
+      Assert.assertFalse("query's last doc was "+lastDoc[0]+" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more);
   }
 }

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/TestBoolean2.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/TestBoolean2.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/TestBoolean2.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/TestBoolean2.java Sun Jun  7 16:58:41 2009
@@ -18,23 +18,18 @@
  */
 
 
-import org.apache.lucene.store.RAMDirectory;
-
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.Term;
+import java.util.Random;
 
 import org.apache.lucene.analysis.WhitespaceAnalyzer;
-
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-
-import org.apache.lucene.queryParser.QueryParser;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.queryParser.ParseException;
-
+import org.apache.lucene.queryParser.QueryParser;
+import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
 
-import java.util.Random;
-
 /** Test BooleanQuery2 against BooleanQuery by overriding the standard query parser.
  * This also tests the scoring order of BooleanQuery.
  */
@@ -158,13 +153,14 @@
 
     int tot=0;
 
+    BooleanQuery q1 = null;
     try {
 
       // increase number of iterations for more complete testing
       for (int i=0; i<1000; i++) {
         int level = rnd.nextInt(3);
-        BooleanQuery q1 = randBoolQuery(new Random(rnd.nextLong()), level, field, vals, null);
-
+        q1 = randBoolQuery(new Random(rnd.nextLong()), level, field, vals, null);
+        
         // Can't sort by relevance since floating point numbers may not quite
         // match up.
         Sort sort = Sort.INDEXORDER;
@@ -181,6 +177,10 @@
         CheckHits.checkEqual(q1, hits1, hits2);
       }
 
+    } catch (Exception e) {
+      // For easier debugging
+      System.out.println("failed query: " + q1);
+      throw e;
     } finally { // even when a test fails.
       BooleanQuery.setAllowDocsOutOfOrder(false);
     }

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/TestBooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/TestBooleanScorer.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/TestBooleanScorer.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/TestBooleanScorer.java Sun Jun  7 16:58:41 2009
@@ -18,6 +18,7 @@
  */
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.lucene.analysis.WhitespaceAnalyzer;
 import org.apache.lucene.document.Document;
@@ -74,5 +75,41 @@
 
   }
   
+  public void testEmptyBucketWithMoreDocs() throws Exception {
+    // This test checks the logic of nextDoc() when all sub scorers have docs
+    // beyond the first bucket (for example). Currently, the code relies on the
+    // 'more' variable to work properly, and this test ensures that if the logic
+    // changes, we have a test to back it up.
+    
+    Similarity sim = Similarity.getDefault();
+    Scorer[] scorers = new Scorer[] {new Scorer(sim) {
+      private int doc = -1;
+      public Explanation explain(int doc) throws IOException { return null; }
+      public float score() throws IOException { return 0; }
+      /** @deprecated delete in 3.0. */
+      public int doc() { return 3000; }
+      public int docID() { return doc; }
+      /** @deprecated delete in 3.0 */
+      public boolean next() throws IOException { return nextDoc() != NO_MORE_DOCS; }
+      
+      public int nextDoc() throws IOException {
+        return doc = doc == -1 ? 3000 : NO_MORE_DOCS;
+      }
+
+      /** @deprecated delete in 3.0 */
+      public boolean skipTo(int target) throws IOException {
+        return advance(target) != NO_MORE_DOCS;
+      }
+      
+      public int advance(int target) throws IOException {
+        return doc = target <= 3000 ? 3000 : NO_MORE_DOCS;
+      }
+      
+    }};
+    BooleanScorer bs = new BooleanScorer(sim, 1, Arrays.asList(scorers), null);
+    
+    assertEquals("should have received 3000", 3000, bs.nextDoc());
+    assertEquals("should have received NO_MORE_DOCS", DocIdSetIterator.NO_MORE_DOCS, bs.nextDoc());
+  }
 
 }

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java Sun Jun  7 16:58:41 2009
@@ -136,10 +136,9 @@
 
     final Weight dw = dq.weight(s);
     final Scorer ds = dw.scorer(r);
-    final boolean skipOk = ds.skipTo(3);
+    final boolean skipOk = ds.advance(3) != DocIdSetIterator.NO_MORE_DOCS;
     if (skipOk) {
-      fail("firsttime skipTo found a match? ... " + 
-            r.document(ds.doc()).get("id"));
+      fail("firsttime skipTo found a match? ... " + r.document(ds.docID()).get("id"));
     }
   }
 
@@ -152,8 +151,8 @@
 
     final Weight dw = dq.weight(s);
     final Scorer ds = dw.scorer(r);
-    assertTrue("firsttime skipTo found no match", ds.skipTo(3));
-    assertEquals("found wrong docid", "d4", r.document(ds.doc()).get("id"));
+    assertTrue("firsttime skipTo found no match", ds.advance(3) != DocIdSetIterator.NO_MORE_DOCS);
+    assertEquals("found wrong docid", "d4", r.document(ds.docID()).get("id"));
   }
 
 

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/TestDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/TestDocIdSet.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/TestDocIdSet.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/TestDocIdSet.java Sun Jun  7 16:58:41 2009
@@ -34,27 +34,37 @@
         public DocIdSetIterator iterator() {
           return new DocIdSetIterator() {
 
-            int docid=-1;
-            //@Override
+            int docid = -1;
+            
+            /** @deprecated use {@link #docID()} instead. */
             public int doc() {
               return docid;
             }
 
-            //@Override
+            public int docID() {
+              return docid;
+            }
+            
+            /** @deprecated use {@link #nextDoc()} instead. */
             public boolean next() throws IOException {
-              docid++;
-              return (docid<maxdoc);
+              return nextDoc() != NO_MORE_DOCS;
             }
 
             //@Override
-            public boolean skipTo(int target) throws IOException {
-              do {
-                if (!next()) {
-                  return false;
-                }
-              } while (target > doc());
+            public int nextDoc() throws IOException {
+              docid++;
+              return docid < maxdoc ? docid : (docid = NO_MORE_DOCS);
+            }
 
-              return true;
+            /** @deprecated use {@link #advance(int)} instead. */
+            public boolean skipTo(int target) throws IOException {
+              return advance(target) != NO_MORE_DOCS;
+            }
+            
+            //@Override
+            public int advance(int target) throws IOException {
+              while (nextDoc() < target) {}
+              return docid;
             }
           };
         } 
@@ -70,10 +80,11 @@
 	  
     DocIdSetIterator iter = filteredSet.iterator();
     ArrayList/*<Integer>*/ list = new ArrayList/*<Integer>*/();
-    if (iter.skipTo(3)) {
-      list.add(new Integer(iter.doc()));
-      while(iter.next()) {
-        list.add(new Integer(iter.doc()));
+    int doc = iter.advance(3);
+    if (doc != DocIdSetIterator.NO_MORE_DOCS) {
+      list.add(Integer.valueOf(doc));
+      while((doc = iter.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+        list.add(Integer.valueOf(doc));
       }
     }
 	  

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java Sun Jun  7 16:58:41 2009
@@ -36,15 +36,28 @@
       return idx == scores.length ? Float.NaN : scores[idx];
     }
 
+    /** @deprecated use {@link #docID()} instead. */
     public int doc() { return idx; }
+    
+    public int docID() { return idx; }
 
-    public boolean next() throws IOException { 
-      return ++idx == scores.length;
+    /** @deprecated use {@link #nextDoc()} instead. */
+    public boolean next() throws IOException {
+      return nextDoc() != NO_MORE_DOCS;
     }
 
+    public int nextDoc() throws IOException {
+      return ++idx != scores.length ? idx : NO_MORE_DOCS;
+    }
+    
+    /** @deprecated use {@link #advance(int)} instead. */
     public boolean skipTo(int target) throws IOException {
+      return advance(target) != NO_MORE_DOCS;
+    }
+    
+    public int advance(int target) throws IOException {
       idx = target;
-      return idx >= scores.length;
+      return idx < scores.length ? idx : NO_MORE_DOCS;
     }
   }
 
@@ -71,7 +84,7 @@
     TopDocsCollector tdc = TopScoreDocCollector.create(scores.length, true);
     Collector c = new PositiveScoresOnlyCollector(tdc);
     c.setScorer(s);
-    while (!s.next()) {
+    while (s.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
       c.collect(0);
     }
     TopDocs td = tdc.topDocs();

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java Sun Jun  7 16:58:41 2009
@@ -42,16 +42,30 @@
       return idx == scores.length ? Float.NaN : scores[idx++];
     }
 
+    /** @deprecated use {@link #docID()} instead. */
     public int doc() { return doc; }
+    
+    public int docID() { return doc; }
 
+    /** @deprecated use {@link #nextDoc()} instead. */
     public boolean next() throws IOException { 
-      return ++doc == scores.length;
+      return nextDoc() != NO_MORE_DOCS;
     }
 
+    public int nextDoc() throws IOException {
+      return ++doc < scores.length ? doc : NO_MORE_DOCS;
+    }
+    
+    /** @deprecated use {@link #advance(int)} instead. */
     public boolean skipTo(int target) throws IOException {
+      return advance(target) != NO_MORE_DOCS;
+    }
+    
+    public int advance(int target) throws IOException {
       doc = target;
-      return doc >= scores.length;
+      return doc < scores.length ? doc : NO_MORE_DOCS;
     }
+    
   }
   
   private static final class ScoreCachingCollector extends Collector {
@@ -98,8 +112,9 @@
     scc.setScorer(s);
     
     // We need to iterate on the scorer so that its doc() advances.
-    while (!s.next()) {
-      scc.collect(s.doc());
+    int doc;
+    while ((doc = s.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+      scc.collect(doc);
     }
     
     for (int i = 0; i < scores.length; i++) {

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/TestSpanQueryFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/TestSpanQueryFilter.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/TestSpanQueryFilter.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/TestSpanQueryFilter.java Sun Jun  7 16:58:41 2009
@@ -16,7 +16,9 @@
  * limitations under the License.
  */
 
-import org.apache.lucene.util.LuceneTestCase;
+import java.util.Iterator;
+import java.util.List;
+
 import org.apache.lucene.analysis.SimpleAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -27,10 +29,7 @@
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.English;
-
-import java.util.BitSet;
-import java.util.Iterator;
-import java.util.List;
+import org.apache.lucene.util.LuceneTestCase;
 
 public class TestSpanQueryFilter extends LuceneTestCase {
 
@@ -77,7 +76,7 @@
   int getDocIdSetSize(DocIdSet docIdSet) throws Exception {
     int size = 0;
     DocIdSetIterator it = docIdSet.iterator();
-    while (it.next()) {
+    while (it.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
       size++;
     }
     return size;
@@ -85,7 +84,7 @@
   
   public void assertContainsDocId(String msg, DocIdSet docIdSet, int docId) throws Exception {
     DocIdSetIterator it = docIdSet.iterator();
-    assertTrue(msg, it.skipTo(docId));
-    assertTrue(msg, it.doc() == docId);
+    assertTrue(msg, it.advance(docId) != DocIdSetIterator.NO_MORE_DOCS);
+    assertTrue(msg, it.docID() == docId);
   }
 }

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/TestTermScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/TestTermScorer.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/TestTermScorer.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/TestTermScorer.java Sun Jun  7 16:58:41 2009
@@ -134,11 +134,11 @@
         TermScorer ts = new TermScorer(weight,
                                        indexReader.termDocs(allTerm), indexSearcher.getSimilarity(),
                                        indexReader.norms(FIELD));
-        assertTrue("next did not return a doc", ts.next() == true);
+        assertTrue("next did not return a doc", ts.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
         assertTrue("score is not correct", ts.score() == 1.6931472f);
-        assertTrue("next did not return a doc", ts.next() == true);
+        assertTrue("next did not return a doc", ts.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
         assertTrue("score is not correct", ts.score() == 1.6931472f);
-        assertTrue("next returned a doc and it should not have", ts.next() == false);
+        assertTrue("next returned a doc and it should not have", ts.nextDoc() == DocIdSetIterator.NO_MORE_DOCS);
     }
 
     public void testSkipTo() throws Exception {
@@ -151,7 +151,7 @@
         TermScorer ts = new TermScorer(weight,
                                        indexReader.termDocs(allTerm), indexSearcher.getSimilarity(),
                                        indexReader.norms(FIELD));
-        assertTrue("Didn't skip", ts.skipTo(3) == true);
+        assertTrue("Didn't skip", ts.advance(3) != DocIdSetIterator.NO_MORE_DOCS);
         //The next doc should be doc 5
         assertTrue("doc should be number 5", ts.doc() == 5);
     }

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java Sun Jun  7 16:58:41 2009
@@ -21,6 +21,8 @@
 import java.util.Collection;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Similarity;
+import org.apache.lucene.search.Weight;
 
 /**
  * Holds all implementations of classes in the o.a.l.s.spans package as a
@@ -109,4 +111,15 @@
     
   }
   
+  static final class JustCompileSpanScorer extends SpanScorer {
+
+    protected JustCompileSpanScorer(Spans spans, Weight weight,
+        Similarity similarity, byte[] norms) throws IOException {
+      super(spans, weight, similarity, norms);
+    }
+
+    protected boolean setFreqCurrentDoc() throws IOException {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+    }
+  }
 }

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java Sun Jun  7 16:58:41 2009
@@ -165,8 +165,7 @@
     SpanNearQuery q = makeQuery();
     Weight w = q.createWeight(searcher);
     Scorer s = w.scorer(searcher.getIndexReader());
-    assertEquals(true, s.skipTo(1));
-    assertEquals(1, s.doc());
+    assertEquals(1, s.advance(1));
   }
   /**
    * not a direct test of NearSpans, but a demonstration of how/when

Modified: lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestSpans.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestSpans.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestSpans.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/search/spans/TestSpans.java Sun Jun  7 16:58:41 2009
@@ -17,6 +17,7 @@
  * limitations under the License.
  */
 
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.CheckHits;
@@ -392,11 +393,11 @@
 
     Scorer spanScorer = snq.weight(searcher).scorer(searcher.getIndexReader());
 
-    assertTrue("first doc", spanScorer.next());
-    assertEquals("first doc number", spanScorer.doc(), 11);
+    assertTrue("first doc", spanScorer.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
+    assertEquals("first doc number", spanScorer.docID(), 11);
     float score = spanScorer.score();
     assertTrue("first doc score should be zero, " + score, score == 0.0f);
-    assertTrue("no second doc", ! spanScorer.next());
+    assertTrue("no second doc", spanScorer.nextDoc() == DocIdSetIterator.NO_MORE_DOCS);
   }
 
   // LUCENE-1404

Modified: lucene/java/trunk/src/test/org/apache/lucene/util/TestOpenBitSet.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/util/TestOpenBitSet.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/util/TestOpenBitSet.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/util/TestOpenBitSet.java Sun Jun  7 16:58:41 2009
@@ -20,6 +20,8 @@
 import java.util.Random;
 import java.util.BitSet;
 
+import org.apache.lucene.search.DocIdSetIterator;
+
 /**
  * @version $Id$
  */
@@ -55,12 +57,8 @@
     OpenBitSetIterator iterator = new OpenBitSetIterator(b);
     do {
       aa = a.nextSetBit(aa+1);
-      if (rand.nextBoolean())
-        iterator.next();
-      else
-        iterator.skipTo(bb+1);
-      bb = iterator.doc();
-      assertEquals(aa,bb);
+      bb = rand.nextBoolean() ? iterator.nextDoc() : iterator.advance(bb + 1);
+      assertEquals(aa == -1 ? DocIdSetIterator.NO_MORE_DOCS : aa, bb);
     } while (aa>=0);
   }
 
@@ -69,11 +67,8 @@
     OpenBitSetIterator iterator = new OpenBitSetIterator(b);
     do {
       aa = a.nextSetBit(aa+1);
-      if (rand.nextBoolean())
-        bb = iterator.nextDoc();
-      else
-        bb = iterator.next(bb+1);
-      assertEquals(aa,bb);
+      bb = rand.nextBoolean() ? iterator.nextDoc() : iterator.advance(bb + 1);
+      assertEquals(aa == -1 ? DocIdSetIterator.NO_MORE_DOCS : aa, bb);
     } while (aa>=0);
   }
 

Modified: lucene/java/trunk/src/test/org/apache/lucene/util/TestSortedVIntList.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/util/TestSortedVIntList.java?rev=782410&r1=782409&r2=782410&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/util/TestSortedVIntList.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/util/TestSortedVIntList.java Sun Jun  7 16:58:41 2009
@@ -42,10 +42,10 @@
     }
     DocIdSetIterator m = vintList.iterator();
     for (int i = 0; i < ints.length; i++) {
-      assertTrue("No end of Matcher at: " + i, m.next());
-      assertEquals(ints[i], m.doc());
+      assertTrue("No end of Matcher at: " + i, m.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
+      assertEquals(ints[i], m.docID());
     }
-    assertTrue("End of Matcher", (! m.next()));
+    assertTrue("End of Matcher", m.nextDoc() == DocIdSetIterator.NO_MORE_DOCS);
   }
 
   void tstVIntList(
@@ -143,9 +143,6 @@
   public void test02() {
     tstInts(new int[] {0});
   }
-  public void test03() {
-    tstInts(new int[] {0,Integer.MAX_VALUE});
-  }
   public void test04a() {
     tstInts(new int[] {0, VB2 - 1});
   }