You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by go...@apache.org on 2005/01/24 20:21:02 UTC

cvs commit: jakarta-lucene/src/test/org/apache/lucene/search TestBoolean2.java CheckHits.java

goller      2005/01/24 11:21:01

  Modified:    src/java/org/apache/lucene/search ConjunctionScorer.java
                        BooleanQuery.java
               src/test/org/apache/lucene/search CheckHits.java
  Added:       src/java/org/apache/lucene/search ReqOptSumScorer.java
                        BooleanScorer2.java DisjunctionSumScorer.java
                        NonMatchingScorer.java ReqExclScorer.java
               src/test/org/apache/lucene/search TestBoolean2.java
  Log:
  New BooleanScorer implemented by Paul Elschot
  (Patch 31785) that implements skipTo and delivers
  documents in correct order. Furthermore a small bug
  in ConjunctionScorer skipTo (if called without a
  preceeding next) is eliminated.
  
  Revision  Changes    Path
  1.7       +25 -14    jakarta-lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java
  
  Index: ConjunctionScorer.java
  ===================================================================
  RCS file: /home/cvs/jakarta-lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java,v
  retrieving revision 1.6
  retrieving revision 1.7
  diff -u -r1.6 -r1.7
  --- ConjunctionScorer.java	6 Sep 2004 12:07:04 -0000	1.6
  +++ ConjunctionScorer.java	24 Jan 2005 19:21:01 -0000	1.7
  @@ -17,10 +17,13 @@
    */
   
   import java.io.IOException;
  -import java.util.*;
  +import java.util.Arrays;
  +import java.util.Comparator;
  +import java.util.Iterator;
  +import java.util.LinkedList;
   
   /** Scorer for conjunctions, sets of queries, all of which are required. */
  -final class ConjunctionScorer extends Scorer {
  +class ConjunctionScorer extends Scorer {
     private LinkedList scorers = new LinkedList();
     private boolean firstTime = true;
     private boolean more = true;
  @@ -41,7 +44,7 @@
   
     public boolean next() throws IOException {
       if (firstTime) {
  -      init();
  +      init(true);
       } else if (more) {
         more = last().next();                       // trigger further scanning
       }
  @@ -57,12 +60,18 @@
     }
   
     public boolean skipTo(int target) throws IOException {
  +    if(firstTime) {
  +      init(false);
  +    }
  +    
       Iterator i = scorers.iterator();
       while (more && i.hasNext()) {
         more = ((Scorer)i.next()).skipTo(target);
       }
  +    
       if (more)
         sortScorers();                              // re-sort scorers
  +    
       return doNext();
     }
   
  @@ -74,20 +83,22 @@
       score *= coord;
       return score;
     }
  -
  -  private void init() throws IOException {
  -    more = scorers.size() > 0;
  -
  -    // compute coord factor
  +  
  +  private void init(boolean initScorers) throws IOException {
  +    //  compute coord factor
       coord = getSimilarity().coord(scorers.size(), scorers.size());
  +   
  +    more = scorers.size() > 0;
   
  -    // move each scorer to its first entry
  -    Iterator i = scorers.iterator();
  -    while (more && i.hasNext()) {
  -      more = ((Scorer)i.next()).next();
  +    if(initScorers){
  +      // move each scorer to its first entry
  +      Iterator i = scorers.iterator();
  +      while (more && i.hasNext()) {
  +        more = ((Scorer)i.next()).next();
  +      }
  +      if (more)
  +        sortScorers(); // initial sort of list
       }
  -    if (more)
  -      sortScorers();                              // initial sort of list
   
       firstTime = false;
     }
  
  
  
  1.29      +43 -5     jakarta-lucene/src/java/org/apache/lucene/search/BooleanQuery.java
  
  Index: BooleanQuery.java
  ===================================================================
  RCS file: /home/cvs/jakarta-lucene/src/java/org/apache/lucene/search/BooleanQuery.java,v
  retrieving revision 1.28
  retrieving revision 1.29
  diff -u -r1.28 -r1.29
  --- BooleanQuery.java	14 Dec 2004 19:28:44 -0000	1.28
  +++ BooleanQuery.java	24 Jan 2005 19:21:01 -0000	1.29
  @@ -1,7 +1,7 @@
   package org.apache.lucene.search;
   
   /**
  - * Copyright 2004 The Apache Software Foundation
  + * Copyright 2004-2005 The Apache Software Foundation
    *
    * Licensed under the Apache License, Version 2.0 (the "License");
    * you may not use this file except in compliance with the License.
  @@ -21,7 +21,7 @@
   import org.apache.lucene.index.IndexReader;
   
   /** A Query that matches documents matching boolean combinations of other
  -  queries, typically {@link TermQuery}s or {@link PhraseQuery}s.
  +  * queries, typically {@link TermQuery}s or {@link PhraseQuery}s.
     */
   public class BooleanQuery extends Query {
     
  @@ -117,8 +117,8 @@
     }
   
     private class BooleanWeight implements Weight {
  -    private Searcher searcher;
  -    private Vector weights = new Vector();
  +    protected Searcher searcher;
  +    protected Vector weights = new Vector();
   
       public BooleanWeight(Searcher searcher) {
         this.searcher = searcher;
  @@ -126,6 +126,7 @@
           BooleanClause c = (BooleanClause)clauses.elementAt(i);
           weights.add(c.getQuery().createWeight(searcher));
         }
  +  //System.out.println("Creating " + getClass().getName());
       }
   
       public Query getQuery() { return BooleanQuery.this; }
  @@ -156,6 +157,7 @@
         }
       }
   
  +    /** @return A good old 1.4 Scorer */
       public Scorer scorer(IndexReader reader) throws IOException {
         // First see if the (faster) ConjunctionScorer will work.  This can be
         // used when all clauses are required.  Also, at this point a
  @@ -246,8 +248,44 @@
       }
     }
   
  +  private class BooleanWeight2 extends BooleanWeight {
  +    /* Merge into BooleanWeight in case the 1.4 BooleanScorer is dropped */
  +    public BooleanWeight2(Searcher searcher) {  super(searcher); }
  +
  +    /** @return An alternative Scorer that uses and provides skipTo(),
  +     *          and scores documents in document number order.
  +     */
  +    public Scorer scorer(IndexReader reader) throws IOException {
  +      BooleanScorer2 result = new BooleanScorer2(getSimilarity(searcher));
  +
  +      for (int i = 0 ; i < weights.size(); i++) {
  +        BooleanClause c = (BooleanClause)clauses.elementAt(i);
  +        Weight w = (Weight)weights.elementAt(i);
  +        Scorer subScorer = w.scorer(reader);
  +        if (subScorer != null)
  +          result.add(subScorer, c.isRequired(), c.isProhibited());
  +        else if (c.isRequired())
  +          return null;
  +      }
  +
  +      return result;
  +    }
  +  }
  +
  +  /** Indicates whether to use good old 1.4 BooleanScorer. */
  +  private static boolean useScorer14 = false;
  +  
  +  public static void setUseScorer14(boolean use14) {
  +    useScorer14 = use14;
  +  }
  +  
  +  public static boolean getUseScorer14() {
  +    return useScorer14;
  +  }
  +  
     protected Weight createWeight(Searcher searcher) {
  -    return new BooleanWeight(searcher);
  +    return getUseScorer14() ? (Weight) new BooleanWeight(searcher)
  +                            : (Weight) new BooleanWeight2(searcher);
     }
   
     public Query rewrite(IndexReader reader) throws IOException {
  
  
  
  1.1                  jakarta-lucene/src/java/org/apache/lucene/search/ReqOptSumScorer.java
  
  Index: ReqOptSumScorer.java
  ===================================================================
  package org.apache.lucene.search;
  /**
   * Copyright 2005 Apache Software Foundation.
   *
   * Licensed under the Apache License, Version 2.0 (the "License");
   * you may not use this file except in compliance with the License.
   * You may obtain a copy of the License at
   *
   *     http://www.apache.org/licenses/LICENSE-2.0
   *
   * Unless required by applicable law or agreed to in writing, software
   * distributed under the License is distributed on an "AS IS" BASIS,
   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
   * See the License for the specific language governing permissions and
   * limitations under the License.
   */
  
  import java.io.IOException;
  
  /** A Scorer for queries with a required part and an optional part.
   * Delays skipTo() on the optional part until a score() is needed.
   * <br>
   * This <code>Scorer</code> implements {@link Scorer#skipTo(int)}.
   */
  public class ReqOptSumScorer extends Scorer {
    /** The scorers passed from the constructor.
     * These are set to null as soon as their next() or skipTo() returns false.
     */
    private Scorer reqScorer;
    private Scorer optScorer;
  
    /** Construct a <code>ReqOptScorer</code>.
     * @param reqScorer The required scorer. This must match.
     * @param optScorer The optional scorer. This is used for scoring only.
     */
    public ReqOptSumScorer(
        Scorer reqScorer,
        Scorer optScorer)
    {
      super(null); // No similarity used.
      this.reqScorer = reqScorer;
      this.optScorer = optScorer;
    }
  
    private boolean firstTimeOptScorer = true;
  
    public boolean next() throws IOException {
      return reqScorer.next();
    }
  
    public boolean skipTo(int target) throws IOException {
      return reqScorer.skipTo(target);
    }
  
    public int doc() {
      return reqScorer.doc();
    }
  
    /** 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();
      float reqScore = reqScorer.score();
      if (firstTimeOptScorer) {
        firstTimeOptScorer = false;
        if (! optScorer.skipTo(curDoc)) {
          optScorer = null;
          return reqScore;
        }
      } else if (optScorer == null) {
        return reqScore;
      } else if ((optScorer.doc() < curDoc) && (! optScorer.skipTo(curDoc))) {
        optScorer = null;
        return reqScore;
      }
      // assert (optScorer != null) && (optScorer.doc() >= curDoc);
      return (optScorer.doc() == curDoc)
         ? reqScore + optScorer.score()
         : reqScore;
    }
  
    /** Explain the score of a document.
     * @todo Also show the total score.
     * See BooleanScorer.explain() on how to do this.
     */
    public Explanation explain(int doc) throws IOException {
      Explanation res = new Explanation();
      res.setDescription("required, optional");
      res.addDetail(reqScorer.explain(doc));
      res.addDetail(optScorer.explain(doc));
      return res;
    }
  }
  
  
  
  
  1.1                  jakarta-lucene/src/java/org/apache/lucene/search/BooleanScorer2.java
  
  Index: BooleanScorer2.java
  ===================================================================
  package org.apache.lucene.search;
  
  /**
   * Copyright 2005 The Apache Software Foundation
   *
   * Licensed under the Apache License, Version 2.0 (the "License");
   * you may not use this file except in compliance with the License.
   * You may obtain a copy of the License at
   *
   *     http://www.apache.org/licenses/LICENSE-2.0
   *
   * Unless required by applicable law or agreed to in writing, software
   * distributed under the License is distributed on an "AS IS" BASIS,
   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
   * See the License for the specific language governing permissions and
   * limitations under the License.
   */
  
  import java.io.IOException;
  import java.util.ArrayList;
  import java.util.List;
  import java.util.Iterator;
  
  /** An alternative to BooleanScorer.
   * <br>Uses ConjunctionScorer, DisjunctionScorer, ReqOptScorer and ReqExclScorer.
   * <br>Implements skipTo(), and has no limitations on the numbers of added scorers.
   */
  public class BooleanScorer2 extends Scorer {
    private ArrayList requiredScorers = new ArrayList();
    private ArrayList optionalScorers = new ArrayList();
    private ArrayList prohibitedScorers = new ArrayList();
  
  
    private class Coordinator {
      int maxCoord = 0; // to be increased for each non prohibited scorer
      
      private float[] coordFactors = null;
      
      void init() { // use after all scorers have been added.
        coordFactors = new float[maxCoord + 1];
        Similarity sim = getSimilarity();
        for (int i = 0; i <= maxCoord; i++) {
          coordFactors[i] = sim.coord(i, maxCoord);
        }
      }
      
      int nrMatchers; // to be increased by score() of match counting scorers.
  
      void initDoc() {
        nrMatchers = 0;
      }
      
      float coordFactor() {
        return coordFactors[nrMatchers];
      }
    }
  
    private final Coordinator coordinator;
  
    /** The scorer to which all scoring will be delegated,
     * except for computing and using the coordination factor.
     */
    private Scorer countingSumScorer = null;
  
    public BooleanScorer2(Similarity similarity) {
      super(similarity);
      coordinator = new Coordinator();
    }
  
    public void add(final Scorer scorer, boolean required, boolean prohibited) {
      if (!prohibited) {
        coordinator.maxCoord++;
      }
  
      if (required) {
        if (prohibited) {
          throw new IllegalArgumentException("scorer cannot be required and prohibited");
        }
        requiredScorers.add(scorer);
      } else if (prohibited) {
        prohibitedScorers.add(scorer);
      } else {
        optionalScorers.add(scorer);
      }
    }
  
    /** Initialize the match counting scorer that sums all the
     * scores. <p>
     * When "counting" is used in a name it means counting the number
     * of matching scorers.<br>
     * When "sum" is used in a name it means score value summing
     * over the matching scorers
     */
    private void initCountingSumScorer() {
      coordinator.init();
      countingSumScorer = makeCountingSumScorer();
    }
  
    /** Count a scorer as a single match. */
    private class SingleMatchScorer extends Scorer {
      private Scorer scorer;
      SingleMatchScorer(Scorer scorer) {
        super(scorer.getSimilarity());
        this.scorer = scorer;
      }
      public float score() throws IOException {
        coordinator.nrMatchers++;
        return scorer.score();
      }
      public int doc() {
        return scorer.doc();
      }
      public boolean next() throws IOException {
        return scorer.next();
      }
      public boolean skipTo(int docNr) throws IOException {
        return scorer.skipTo(docNr);
      }
      public Explanation explain(int docNr) throws IOException {
        return scorer.explain(docNr);
      }
    }
  
    private Scorer countingDisjunctionSumScorer(List scorers)
    // each scorer from the list counted as a single matcher
    {
      return new DisjunctionSumScorer(scorers) {
        public float score() throws IOException {
          coordinator.nrMatchers += nrMatchers;
          return super.score();
        }
      };
    }
  
    private static Similarity defaultSimilarity = new DefaultSimilarity();
  
    private Scorer countingConjunctionSumScorer(List requiredScorers)
    // each scorer from the list counted as a single matcher
    {
      final int requiredNrMatchers = requiredScorers.size();
      ConjunctionScorer cs = new ConjunctionScorer(defaultSimilarity) {
        public float score() throws IOException {
          coordinator.nrMatchers += requiredNrMatchers;
          // All scorers match, so defaultSimilarity super.score() always has 1 as
          // the coordination factor.
          // Therefore the sum of the scores of the requiredScorers
          // is used as score.
          return super.score();
        }
      };
      Iterator rsi = requiredScorers.iterator();
      while (rsi.hasNext()) {
        cs.add((Scorer) rsi.next());
      }
      return cs;
    }
  
    /** Returns the scorer to be used for match counting and score summing.
     * Uses requiredScorers, optionalScorers and prohibitedScorers.
     */
    private Scorer makeCountingSumScorer()
    // each scorer counted as a single matcher
    {
      if (requiredScorers.size() == 0) {
        if (optionalScorers.size() == 0) {
          return new NonMatchingScorer();  // only prohibited scorers
        } else if (optionalScorers.size() == 1) {
          return makeCountingSumScorer2( // the only optional scorer is required
                    new SingleMatchScorer((Scorer) optionalScorers.get(0)),
                    new ArrayList()); // no optional scorers left
        } else { // more than 1 optionalScorers, no required scorers
          return makeCountingSumScorer2( // at least one optional scorer is required
                    countingDisjunctionSumScorer(optionalScorers), 
                    new ArrayList()); // no optional scorers left
        }
      } else if (requiredScorers.size() == 1) { // 1 required
        return makeCountingSumScorer2(
                    new SingleMatchScorer((Scorer) requiredScorers.get(0)),
                    optionalScorers);
      } else { // more required scorers
        return makeCountingSumScorer2(
                    countingConjunctionSumScorer(requiredScorers),
                    optionalScorers);
      }
    }
  
    /** Returns the scorer to be used for match counting and score summing.
     * Uses the arguments and prohibitedScorers.
     * @param requiredCountingSumScorer A required scorer already built.
     * @param optionalScorers A list of optional scorers, possibly empty.
     */
    private Scorer makeCountingSumScorer2(
        Scorer requiredCountingSumScorer,
        List optionalScorers) // not match counting
    {
      if (optionalScorers.size() == 0) { // no optional
        if (prohibitedScorers.size() == 0) { // no prohibited
          return requiredCountingSumScorer;
        } else if (prohibitedScorers.size() == 1) { // no optional, 1 prohibited
          return new ReqExclScorer(
                        requiredCountingSumScorer,
                        (Scorer) prohibitedScorers.get(0)); // not match counting
        } else { // no optional, more prohibited
          return new ReqExclScorer(
                        requiredCountingSumScorer,
                        new DisjunctionSumScorer(prohibitedScorers)); // score unused. not match counting
        }
      } else if (optionalScorers.size() == 1) { // 1 optional
        return makeCountingSumScorer3(
                        requiredCountingSumScorer,
                        new SingleMatchScorer((Scorer) optionalScorers.get(0)));
     } else { // more optional
        return makeCountingSumScorer3(
                        requiredCountingSumScorer,
                        countingDisjunctionSumScorer(optionalScorers));
      }
    }
  
    /** Returns the scorer to be used for match counting and score summing.
     * Uses the arguments and prohibitedScorers.
     * @param requiredCountingSumScorer A required scorer already built.
     * @param optionalCountingSumScorer An optional scorer already built.
     */
    private Scorer makeCountingSumScorer3(
        Scorer requiredCountingSumScorer,
        Scorer optionalCountingSumScorer)
    {
      if (prohibitedScorers.size() == 0) { // no prohibited
        return new ReqOptSumScorer(requiredCountingSumScorer,
                                   optionalCountingSumScorer);
      } else if (prohibitedScorers.size() == 1) { // 1 prohibited
        return new ReqOptSumScorer(
                      new ReqExclScorer(requiredCountingSumScorer,
                                        (Scorer) prohibitedScorers.get(0)),  // not match counting
                      optionalCountingSumScorer);
      } else { // more prohibited
        return new ReqOptSumScorer(
                      new ReqExclScorer(
                            requiredCountingSumScorer,
                            new DisjunctionSumScorer(prohibitedScorers)), // score unused. not match counting
                      optionalCountingSumScorer);
      }
    }
  
    /** Scores and collects all matching documents.
     * @param hc The collector to which all matching documents are passed through
     * {@link HitCollector#collect(int, float)}.
     * <br>When this method is used the {@link #explain(int)} method should not be used.
     */
    public void score(HitCollector hc) throws IOException {
      if (countingSumScorer == null) {
        initCountingSumScorer();
      }
      while (countingSumScorer.next()) {
        hc.collect(countingSumScorer.doc(), score());
      }
    }
  
    /** Expert: Collects matching documents in a range.
     * <br>Note that {@link #next()} must be called once before this method is
     * called for the first time.
     * @param hc The collector to which all matching documents are passed through
     * {@link HitCollector#collect(int, float)}.
     * @param max Do not score documents past this.
     * @return true if more matching documents may remain.
     */
    protected boolean score(HitCollector hc, int max) throws IOException {
      // null pointer exception when next() was not called before:
      int docNr = countingSumScorer.doc();
      while (docNr < max) {
        hc.collect(docNr, score());
        if (! countingSumScorer.next()) {
          return false;
        }
        docNr = countingSumScorer.doc();
      }
      return true;
    }
  
    public int doc() { return countingSumScorer.doc(); }
  
    public boolean next() throws IOException {
      if (countingSumScorer == null) {
        initCountingSumScorer();
      }
      return countingSumScorer.next();
    }
  
    public float score() throws IOException {
      coordinator.initDoc();
      float sum = countingSumScorer.score();
      return sum * coordinator.coordFactor();
    }
  
    public boolean skipTo(int target) throws IOException {
      if (countingSumScorer == null) {
        initCountingSumScorer();
      }
      return countingSumScorer.skipTo(target);
    }
  
    public Explanation explain(int doc) throws IOException {
      throw new UnsupportedOperationException();
   /* How to explain the coordination factor?
      initCountingSumScorer();
      return countingSumScorer.explain(doc); // misses coord factor. 
    */
    }
  }
  
  
  
  
  1.1                  jakarta-lucene/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
  
  Index: DisjunctionSumScorer.java
  ===================================================================
  package org.apache.lucene.search;
  
  /**
   * Copyright 2005 The Apache Software Foundation
   *
   * Licensed under the Apache License, Version 2.0 (the "License");
   * you may not use this file except in compliance with the License.
   * You may obtain a copy of the License at
   *
   *     http://www.apache.org/licenses/LICENSE-2.0
   *
   * Unless required by applicable law or agreed to in writing, software
   * distributed under the License is distributed on an "AS IS" BASIS,
   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
   * See the License for the specific language governing permissions and
   * limitations under the License.
   */
  
  import java.util.List;
  import java.util.Iterator;
  import java.io.IOException;
  
  import org.apache.lucene.util.PriorityQueue;
  
  /** A Scorer for OR like queries, counterpart of Lucene's <code>ConjunctionScorer</code>.
   * This Scorer implements {@link Scorer#skipTo(int)} and uses skipTo() on the given Scorers. 
   */
  public class DisjunctionSumScorer extends Scorer {
    /** The number of subscorers. */ 
    private final int nrScorers;
    
    /** The subscorers. */
    protected final List subScorers;
    
    /** The minimum number of scorers that should match. */
    private final int minimumNrMatchers;
    
    /** The scorerQueue contains all subscorers ordered by their current doc(),
     * with the minimum at the top.
     * <br>The scorerQueue is initialized the first time next() or skipTo() is called.
     * <br>An exhausted scorer is immediately removed from the scorerQueue.
     * <br>If less than the minimumNrMatchers scorers
     * remain in the scorerQueue next() and skipTo() return false.
     * <p>
     * After each to call to next() or skipTo()
     * <code>currentSumScore</code> is the total score of the current matching doc,
     * <code>nrMatchers</code> is the number of matching scorers,
     * and all scorers are after the matching doc, or are exhausted.
     */
    private ScorerQueue scorerQueue = null;
    
    /** The document number of the current match. */
    private int currentDoc = -1;
  
    /** The number of subscorers that provide the current match. */
    protected int nrMatchers = -1;
  
    private float currentScore = Float.NaN;
    
    /** Construct a <code>DisjunctionScorer</code>.
     * @param subScorers A collection of at least two subscorers.
     * @param minimumNrMatchers The positive minimum number of subscorers that should
     * match to match this query.
     * <br>When <code>minimumNrMatchers</code> is bigger than
     * the number of <code>subScorers</code>,
     * no matches will be produced.
     * <br>When minimumNrMatchers equals the number of subScorers,
     * it more efficient to use <code>ConjunctionScorer</code>.
     */
    public DisjunctionSumScorer( List subScorers, int minimumNrMatchers) {
      super(null);
      
      nrScorers = subScorers.size();
  
      if (minimumNrMatchers <= 0) {
        throw new IllegalArgumentException("Minimum nr of matchers must be positive");
      }
      if (nrScorers <= 1) {
        throw new IllegalArgumentException("There must be at least 2 subScorers");
      }
  
      this.minimumNrMatchers = minimumNrMatchers;
      this.subScorers = subScorers;
    }
    
    /** Construct a <code>DisjunctionScorer</code>, using one as the minimum number
     * of matching subscorers.
     */
    public DisjunctionSumScorer(List subScorers) {
      this(subScorers, 1);
    }
  
    /** Called the first time next() or skipTo() is called to
     * initialize <code>scorerQueue</code>.
     */
    private void initScorerQueue() throws IOException {
      Iterator si = subScorers.iterator();
      scorerQueue = new ScorerQueue(nrScorers);
      while (si.hasNext()) {
        Scorer se = (Scorer) si.next();
        if (se.next()) { // doc() method will be used in scorerQueue.
          scorerQueue.insert(se);
        }
      }
    }
  
    /** A <code>PriorityQueue</code> that orders by {@link Scorer#doc()}. */
    private class ScorerQueue extends PriorityQueue {
      ScorerQueue(int size) {
        initialize(size);
      }
  
      protected boolean lessThan(Object o1, Object o2) {
        return ((Scorer)o1).doc() < ((Scorer)o2).doc();
      }
    }
    
    public boolean next() throws IOException {
      if (scorerQueue == null) {
        initScorerQueue();
      }
      if (scorerQueue.size() < minimumNrMatchers) {
        return false;
      } else {
        return advanceAfterCurrent();
      }
    }
  
  
    /** Advance all subscorers after the current document determined by the
     * top of the <code>scorerQueue</code>.
     * Repeat until at least the minimum number of subscorers match on the same
     * document and all subscorers are after that document or are exhausted.
     * <br>On entry the <code>scorerQueue</code> has at least <code>minimumNrMatchers</code>
     * available. At least the scorer with the minimum document number will be advanced.
     * @return true iff there is a match.
     * <br>In case there is a match, </code>currentDoc</code>, </code>currentSumScore</code>,
     * and </code>nrMatchers</code> describe the match.
     *
     * @todo Investigate whether it is possible to use skipTo() when
     * the minimum number of matchers is bigger than one, ie. try and use the
     * character of ConjunctionScorer for the minimum number of matchers.
     */
    protected boolean advanceAfterCurrent() throws IOException {
      do { // repeat until minimum nr of matchers
        Scorer top = (Scorer) scorerQueue.top();
        currentDoc = top.doc();
        currentScore = top.score();
        nrMatchers = 1;
        do { // Until all subscorers are after currentDoc
          if (top.next()) {
            scorerQueue.adjustTop();
          } else {
            scorerQueue.pop();
            if (scorerQueue.size() < (minimumNrMatchers - nrMatchers)) {
              // Not enough subscorers left for a match on this document,
              // and also no more chance of any further match.
              return false;
            }
            if (scorerQueue.size() == 0) {
              break; // nothing more to advance, check for last match.
            }
          }
          top = (Scorer) scorerQueue.top();
          if (top.doc() != currentDoc) {
            break; // All remaining subscorers are after currentDoc.
          } else {
            currentScore += top.score();
            nrMatchers++;
          }
        } while (true);
        
        if (nrMatchers >= minimumNrMatchers) {
          return true;
        } else if (scorerQueue.size() < minimumNrMatchers) {
          return false;
        }
      } while (true);
    }
    
    /** Returns the score of the current document matching the query.
     * Initially invalid, until {@link #next()} is called the first time.
     */
    public float score() throws IOException { return currentScore; }
     
    public int doc() { return currentDoc; }
  
    /** Returns the number of subscorers matching the current document.
     * Initially invalid, until {@link #next()} is called the first time.
     */
    public int nrMatchers() {
      return nrMatchers;
    }
  
    /** 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.
     * <br>The implementation uses the skipTo() method on the subscorers.
     * @param target The target document number.
     * @return true iff there is such a match.
     */
    public boolean skipTo(int target) throws IOException {
      if (scorerQueue == null) {
        initScorerQueue();
      }
      if (scorerQueue.size() < minimumNrMatchers) {
        return false;
      }
      if (target <= currentDoc) {
        target = currentDoc + 1;
      }
      do {
        Scorer top = (Scorer) scorerQueue.top();
        if (top.doc() >= target) {
          return advanceAfterCurrent();
        } else if (top.skipTo(target)) {
          scorerQueue.adjustTop();
        } else {
          scorerQueue.pop();
          if (scorerQueue.size() < minimumNrMatchers) {
            return false;
          }
        }
      } while (true);
    }
  
   /** Gives and explanation for the score of a given document.
    * @todo Show the resulting score. See BooleanScorer.explain() on how to do this.
    */
    public Explanation explain(int doc) throws IOException {
      Explanation res = new Explanation();
      res.setDescription("At least " + minimumNrMatchers + " of");
      Iterator ssi = subScorers.iterator();
      while (ssi.hasNext()) {
        res.addDetail( ((Scorer) ssi.next()).explain(doc));
      }
      return res;
    }
  }
  
  
  
  1.1                  jakarta-lucene/src/java/org/apache/lucene/search/NonMatchingScorer.java
  
  Index: NonMatchingScorer.java
  ===================================================================
  package org.apache.lucene.search;
  
  /**
   * Copyright 2005 Apache Software Foundation
   *
   * Licensed under the Apache License, Version 2.0 (the "License");
   * you may not use this file except in compliance with the License.
   * You may obtain a copy of the License at
   *
   *     http://www.apache.org/licenses/LICENSE-2.0
   *
   * Unless required by applicable law or agreed to in writing, software
   * distributed under the License is distributed on an "AS IS" BASIS,
   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
   * See the License for the specific language governing permissions and
   * limitations under the License.
   */
   
  import java.io.IOException;
  
  /** A scorer that matches no document at all. */
  class NonMatchingScorer extends Scorer {
    public NonMatchingScorer() { super(null); } // no similarity used
    
    public int doc() { throw new UnsupportedOperationException(); }
  
    public boolean next() throws IOException { return false; }
  
    public float score() { throw new UnsupportedOperationException(); }
  
    public boolean skipTo(int target) { return false; }
  
    public Explanation explain(int doc) {
      Explanation e = new Explanation();
      e.setDescription("No document matches.");
      return e;
    }
  }
   
  
  
  
  
  1.1                  jakarta-lucene/src/java/org/apache/lucene/search/ReqExclScorer.java
  
  Index: ReqExclScorer.java
  ===================================================================
  package org.apache.lucene.search;
  
  /**
   * Copyright 2005 The Apache Software Foundation
   *
   * Licensed under the Apache License, Version 2.0 (the "License");
   * you may not use this file except in compliance with the License.
   * You may obtain a copy of the License at
   *
   *     http://www.apache.org/licenses/LICENSE-2.0
   *
   * Unless required by applicable law or agreed to in writing, software
   * distributed under the License is distributed on an "AS IS" BASIS,
   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
   * See the License for the specific language governing permissions and
   * limitations under the License.
   */
  
  import java.io.IOException;
  
  
  /** A Scorer for queries with a required subscorer and an excluding (prohibited) subscorer.
   * <br>
   * This <code>Scorer</code> implements {@link Scorer#skipTo(int)},
   * and it uses the skipTo() on the given scorers.
   */
  public class ReqExclScorer extends Scorer {
    private Scorer reqScorer, exclScorer;
  
    /** Construct a <code>ReqExclScorer</code>.
     * @param reqScorer The scorer that must match, except where
     * @param exclScorer indicates exclusion.
     */
    public ReqExclScorer(
        Scorer reqScorer,
        Scorer exclScorer) {
      super(null); // No similarity used.
      this.reqScorer = reqScorer;
      this.exclScorer = exclScorer;
    }
  
    private boolean firstTime = true;
    
    public boolean next() throws IOException {
      if (firstTime) {
        if (! exclScorer.next()) {
          exclScorer = null; // exhausted at start
        }
        firstTime = false;
      }
      if (reqScorer == null) {
        return false;
      }
      if (! reqScorer.next()) {
        reqScorer = null; // exhausted, nothing left
        return false;
      }
      if (exclScorer == null) {
        return true; // reqScorer.next() already returned true
      }
      return toNonExcluded();
    }
    
    /** Advance to non excluded doc.
     * <br>On entry:
     * <ul>
     * <li>reqScorer != 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 = exclScorer.doc();
      do {  
        int reqDoc = reqScorer.doc(); // may be excluded
        if (reqDoc < exclDoc) {
          return true; // reqScorer advanced to before exclScorer, ie. not excluded
        } else if (reqDoc > exclDoc) {
          if (! exclScorer.skipTo(reqDoc)) {
            exclScorer = null; // exhausted, no more exclusions
            return true;
          }
          exclDoc = exclScorer.doc();
          if (exclDoc > reqDoc) {
            return true; // not excluded
          }
        }
      } while (reqScorer.next());
      reqScorer = null; // exhausted, nothing left
      return false;
    }
  
    public int doc() {
      return reqScorer.doc(); // reqScorer may be null when next() or skipTo() already return false
    }
  
    /** 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.
     */
    public float score() throws IOException {
      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.
     */
    public boolean skipTo(int target) throws IOException {
      if (firstTime) {
        firstTime = false;
        if (! exclScorer.skipTo(target)) {
          exclScorer = null; // exhausted
        }
      }
      if (reqScorer == null) {
        return false;
      }
      if (exclScorer == null) {
        return reqScorer.skipTo(target);
      }
      if (! reqScorer.skipTo(target)) {
        reqScorer = null;
        return false;
      }
      return toNonExcluded();
    }
  
    public Explanation explain(int doc) throws IOException {
      Explanation res = new Explanation();
      if (exclScorer.skipTo(doc) && (exclScorer.doc() == doc)) {
        res.setDescription("excluded");
      } else {
        res.setDescription("not excluded");
        res.addDetail(reqScorer.explain(doc));
      }
      return res;
    }
  }
  
  
  
  1.3       +38 -9     jakarta-lucene/src/test/org/apache/lucene/search/CheckHits.java
  
  Index: CheckHits.java
  ===================================================================
  RCS file: /home/cvs/jakarta-lucene/src/test/org/apache/lucene/search/CheckHits.java,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- CheckHits.java	10 Oct 2004 15:44:09 -0000	1.2
  +++ CheckHits.java	24 Jan 2005 19:21:01 -0000	1.3
  @@ -1,7 +1,7 @@
   package org.apache.lucene.search;
   
   /**
  - * Copyright 2004 The Apache Software Foundation
  + * Copyright 2004-2005 The Apache Software Foundation
    *
    * Licensed under the Apache License, Version 2.0 (the "License");
    * you may not use this file except in compliance with the License.
  @@ -16,13 +16,6 @@
    * limitations under the License.
    */
   
  -/* 20 May 2004:   Factored out of spans tests. Please leave this comment
  -                  until this class is evt. also used by tests in search package.
  - */
  -
  -import org.apache.lucene.search.Searcher;
  -import org.apache.lucene.search.Query;
  -import org.apache.lucene.search.Hits;
   import junit.framework.TestCase;
   
   import java.io.IOException;
  @@ -30,6 +23,8 @@
   import java.util.TreeSet;
   
   public class CheckHits {
  +  /** Tests that a query has expected document number results.
  +   */
     public static void checkHits(
           Query query,
           String defaultFieldName,
  @@ -49,7 +44,41 @@
         actual.add(new Integer(hits.id(i)));
       }
   
  -    TestCase.assertEquals(query.toString(defaultFieldName), correct, actual);
  +    testCase.assertEquals(query.toString(defaultFieldName), correct, actual);
  +  }
  +
  +  /** Tests that a Hits has an expected order of documents */
  +  public static void checkDocIds(String mes, int[] results, Hits hits, TestCase testCase)
  +  throws IOException {
  +    testCase.assertEquals(mes + " nr of hits", results.length, hits.length());
  +    for (int i = 0; i < results.length; i++) {
  +      testCase.assertEquals(mes + " doc nrs for hit " + i, results[i], hits.id(i));
  +    }
  +  }
  +
  +  /** Tests that two queries have an expected order of documents,
  +   * and that the two queries have the same score values.
  +   */
  +  public static void checkHitsQuery(
  +        Query query,
  +        Hits hits1,
  +        Hits hits2,
  +        int[] results,
  +        TestCase testCase)
  +          throws IOException {
  +
  +    checkDocIds("hits1", results, hits1, testCase);
  +    checkDocIds("hits2", results, hits2, testCase);
  +    
  +    final float scoreTolerance = 1.0e-7f;
  +    for (int i = 0; i < results.length; i++) {
  +      if (Math.abs(hits1.score(i) -  hits2.score(i)) > scoreTolerance) {
  +        testCase.fail("Hit " + i + ", doc nrs " + hits1.id(i) + " and " + hits2.id(i)
  +                      + "\nunequal scores: " + hits1.score(i)
  +                      + "\n           and: " + hits2.score(i)
  +                      + "\nfor query:" + query.toString());
  +      }
  +    }
     }
   
     public static void printDocNrs(Hits hits) throws IOException {
  
  
  
  1.1                  jakarta-lucene/src/test/org/apache/lucene/search/TestBoolean2.java
  
  Index: TestBoolean2.java
  ===================================================================
  package org.apache.lucene.search;
  
  /**
   * Copyright 2005 Apache Software Foundation
   *
   * Licensed under the Apache License, Version 2.0 (the "License");
   * you may not use this file except in compliance with the License.
   * You may obtain a copy of the License at
   *
   *     http://www.apache.org/licenses/LICENSE-2.0
   *
   * Unless required by applicable law or agreed to in writing, software
   * distributed under the License is distributed on an "AS IS" BASIS,
   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
   * See the License for the specific language governing permissions and
   * limitations under the License.
   */
  
  
  import org.apache.lucene.store.RAMDirectory;
  
  import org.apache.lucene.index.IndexWriter;
  
  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.queryParser.ParseException;
  
  import junit.framework.TestCase;
  
  /** Test BooleanQuery2 against BooleanQuery by overriding the standard query parser.
   * This also tests the scoring order of BooleanQuery.
   */
  public class TestBoolean2 extends TestCase {
    private IndexSearcher searcher;
  
    public static final String field = "field";
  
    public void setUp() throws Exception {
      RAMDirectory directory = new RAMDirectory();
      IndexWriter writer= new IndexWriter(directory, new WhitespaceAnalyzer(), true);
      for (int i = 0; i < docFields.length; i++) {
        Document doc = new Document();
        doc.add(new Field(field, docFields[i], Field.Store.NO, Field.Index.TOKENIZED));
        writer.addDocument(doc);
      }
      writer.close();
      searcher = new IndexSearcher(directory);
    }
  
    private String[] docFields = {
      "w1 w2 w3 w4 w5",
      "w1 w3 w2 w3",
      "w1 xx w2 yy w3",
      "w1 w3 xx w2 yy w3"
    };
  
    public Query makeQuery(String queryText) throws ParseException {
      return (new QueryParser(field, new WhitespaceAnalyzer())).parse(queryText);
    }
    
    public void queriesTest(String queryText, int[] expDocNrs) throws Exception {
  //System.out.println();
  //System.out.println("Query: " + queryText);
      Query query1 = makeQuery(queryText);
      BooleanQuery.setUseScorer14(true);
      Hits hits1 = searcher.search(query1);
  
      Query query2 = makeQuery(queryText); // there should be no need to parse again...
      BooleanQuery.setUseScorer14(false);
      Hits hits2 = searcher.search(query2);
  
      CheckHits.checkHitsQuery(query2, hits1, hits2, expDocNrs, this);
    }
  
    public void testQueries01() throws Exception {
      String queryText = "+w3 +xx";
      int[] expDocNrs = {2,3};
      queriesTest(queryText, expDocNrs);
    }
    
    public void testQueries02() throws Exception {
      String queryText = "+w3 xx";
      int[] expDocNrs = {2,3,1,0};
      queriesTest(queryText, expDocNrs);
    }
    
    public void testQueries03() throws Exception {
      String queryText = "w3 xx";
      int[] expDocNrs = {2,3,1,0};
      queriesTest(queryText, expDocNrs);
    }
    
    public void testQueries04() throws Exception {
      String queryText = "w3 -xx";
      int[] expDocNrs = {1,0};
      queriesTest(queryText, expDocNrs);
    }
    
    public void testQueries05() throws Exception {
      String queryText = "+w3 -xx";
      int[] expDocNrs = {1,0};
      queriesTest(queryText, expDocNrs);
    }
    
    public void testQueries06() throws Exception {
      String queryText = "+w3 -xx -w5";
      int[] expDocNrs = {1};
      queriesTest(queryText, expDocNrs);
    }
    
    public void testQueries07() throws Exception {
      String queryText = "-w3 -xx -w5";
      int[] expDocNrs = {};
      queriesTest(queryText, expDocNrs);
    }
    
    public void testQueries08() throws Exception {
      String queryText = "+w3 xx -w5";
      int[] expDocNrs = {2,3,1};
      queriesTest(queryText, expDocNrs);
    }
    
    public void testQueries09() throws Exception {
      String queryText = "+w3 +xx +w2 zz";
      int[] expDocNrs = {2, 3};
      queriesTest(queryText, expDocNrs);
    }
    
      public void testQueries10() throws Exception {
      String queryText = "+w3 +xx +w2 zz";
      int[] expDocNrs = {2, 3};
      searcher.setSimilarity(new DefaultSimilarity(){
        public float coord(int overlap, int maxOverlap) {
          return overlap / ((float)maxOverlap - 1);
        }
      });
      queriesTest(queryText, expDocNrs);
    }
  }
  
  
  

---------------------------------------------------------------------
To unsubscribe, e-mail: lucene-dev-unsubscribe@jakarta.apache.org
For additional commands, e-mail: lucene-dev-help@jakarta.apache.org