You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2015/02/16 20:29:34 UTC

svn commit: r1660184 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/search/ lucene/core/src/test/org/apache/lucene/search/ lucene/test-framework/ lucene/test-framework/src/java/org/apache/lucene/search/

Author: jpountz
Date: Mon Feb 16 19:29:34 2015
New Revision: 1660184

URL: http://svn.apache.org/r1660184
Log:
LUCENE-6244: DisjunctionScorer propagates two-phase iterators of its sub scorers.

Added:
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestApproximationSearchEquivalence.java
      - copied unchanged from r1660180, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestApproximationSearchEquivalence.java
    lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/RandomApproximationQuery.java
      - copied unchanged from r1660180, lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/RandomApproximationQuery.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/core/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ScorerPriorityQueue.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java
    lucene/dev/branches/branch_5x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java

Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1660184&r1=1660183&r2=1660184&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Mon Feb 16 19:29:34 2015
@@ -54,6 +54,9 @@ Optimizations
   positions lazily if the phrase query is in a conjunction with other queries.
   (Robert Muir, Adrien Grand)
 
+* LUCENE-6244: Pure disjunctions now propagate two-phase iterators of the
+  wrapped scorers (see LUCENE-6198). (Adrien Grand, Robert Muir)
+
 * LUCENE-6241: FSDirectory.listAll() doesnt filter out subdirectories anymore,
   for faster performance. Subdirectories don't matter to Lucene. If you need to
   filter out non-index files with some custom usage, you may want to look at 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java?rev=1660184&r1=1660183&r2=1660184&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java Mon Feb 16 19:29:34 2015
@@ -376,10 +376,7 @@ public class BooleanWeight extends Weigh
     } else {
       float coords[] = new float[prohibited.size()+1];
       Arrays.fill(coords, 1F);
-      return new ReqExclScorer(main, 
-                               new DisjunctionSumScorer(this, 
-                                                        prohibited.toArray(new Scorer[prohibited.size()]), 
-                                                        coords));
+      return new ReqExclScorer(main, new DisjunctionSumScorer(this, prohibited, coords, false));
     }
   }
   
@@ -402,9 +399,7 @@ public class BooleanWeight extends Weigh
       if (minShouldMatch > 1) {
         return new MinShouldMatchSumScorer(this, optional, minShouldMatch, coords);
       } else {
-        return new DisjunctionSumScorer(this, 
-                                        optional.toArray(new Scorer[optional.size()]), 
-                                        coords);
+        return new DisjunctionSumScorer(this, optional, coords, needsScores);
       }
     }
   }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java?rev=1660184&r1=1660183&r2=1660184&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java Mon Feb 16 19:29:34 2015
@@ -115,7 +115,8 @@ public class DisjunctionMaxQuery extends
   protected class DisjunctionMaxWeight extends Weight {
 
     /** The Weights for our subqueries, in 1-1 correspondence with disjuncts */
-    protected ArrayList<Weight> weights = new ArrayList<>();  // The Weight's for our subqueries, in 1-1 correspondence with disjuncts
+    protected final ArrayList<Weight> weights = new ArrayList<>();  // The Weight's for our subqueries, in 1-1 correspondence with disjuncts
+    private final boolean needsScores;
 
     /** Construct the Weight for this Query searched by searcher.  Recursively construct subquery weights. */
     public DisjunctionMaxWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
@@ -123,6 +124,7 @@ public class DisjunctionMaxQuery extends
       for (Query disjunctQuery : disjuncts) {
         weights.add(disjunctQuery.createWeight(searcher, needsScores));
       }
+      this.needsScores = needsScores;
     }
 
     /** Compute the sub of squared weights of us applied to our subqueries.  Used for normalization. */
@@ -166,7 +168,7 @@ public class DisjunctionMaxQuery extends
         // only one sub-scorer in this segment
         return scorers.get(0);
       } else {
-        return new DisjunctionMaxScorer(this, tieBreakerMultiplier, scorers.toArray(new Scorer[scorers.size()]));
+        return new DisjunctionMaxScorer(this, tieBreakerMultiplier, scorers, needsScores);
       }
     }
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java?rev=1660184&r1=1660183&r2=1660184&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java Mon Feb 16 19:29:34 2015
@@ -17,6 +17,7 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
+import java.util.List;
 
 import org.apache.lucene.search.ScorerPriorityQueue.ScorerWrapper;
 
@@ -41,13 +42,13 @@ final class DisjunctionMaxScorer extends
    * @param subScorers
    *          The sub scorers this Scorer should iterate on
    */
-  DisjunctionMaxScorer(Weight weight, float tieBreakerMultiplier, Scorer[] subScorers) {
-    super(weight, subScorers);
+  DisjunctionMaxScorer(Weight weight, float tieBreakerMultiplier, List<Scorer> subScorers, boolean needsScores) {
+    super(weight, subScorers, needsScores);
     this.tieBreakerMultiplier = tieBreakerMultiplier;
   }
 
   @Override
-  protected float score(ScorerWrapper topList, int freq) throws IOException {
+  protected float score(ScorerWrapper topList) throws IOException {
     float scoreSum = 0;
     float scoreMax = 0;
     for (ScorerWrapper w = topList; w != null; w = w.next) {

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java?rev=1660184&r1=1660183&r2=1660184&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java Mon Feb 16 19:29:34 2015
@@ -20,6 +20,7 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 
 import org.apache.lucene.search.ScorerPriorityQueue.ScorerWrapper;
 import org.apache.lucene.util.BytesRef;
@@ -29,135 +30,225 @@ import org.apache.lucene.util.BytesRef;
  */
 abstract class DisjunctionScorer extends Scorer {
 
+  private final boolean needsScores;
   private final ScorerPriorityQueue subScorers;
+  private final long cost;
 
-  /** The document number of the current match. */
-  protected int doc = -1;
-  protected int numScorers;
-  /** Number of matching scorers for the current match. */
-  private int freq = -1;
   /** Linked list of scorers which are on the current doc */
   private ScorerWrapper topScorers;
 
-  protected DisjunctionScorer(Weight weight, Scorer subScorers[]) {
+  protected DisjunctionScorer(Weight weight, List<Scorer> subScorers, boolean needsScores) {
     super(weight);
-    if (subScorers.length <= 1) {
+    if (subScorers.size() <= 1) {
       throw new IllegalArgumentException("There must be at least 2 subScorers");
     }
-    this.subScorers = new ScorerPriorityQueue(subScorers.length);
+    this.subScorers = new ScorerPriorityQueue(subScorers.size());
+    long cost = 0;
     for (Scorer scorer : subScorers) {
-      this.subScorers.add(new ScorerWrapper(scorer));
+      final ScorerWrapper w = new ScorerWrapper(scorer);
+      cost += w.cost;
+      this.subScorers.add(w);
+    }
+    this.cost = cost;
+    this.needsScores = needsScores;
+  }
+
+  /**
+   * A {@link DocIdSetIterator} which is a disjunction of the approximations of
+   * the provided iterators.
+   */
+  private static class DisjunctionDISIApproximation extends DocIdSetIterator {
+
+    final ScorerPriorityQueue subScorers;
+    final long cost;
+
+    DisjunctionDISIApproximation(ScorerPriorityQueue subScorers) {
+      this.subScorers = subScorers;
+      long cost = 0;
+      for (ScorerWrapper w : subScorers) {
+        cost += w.cost;
+      }
+      this.cost = cost;
     }
-  }
-  
-  @Override
-  public final Collection<ChildScorer> getChildren() {
-    ArrayList<ChildScorer> children = new ArrayList<>();
-    for (ScorerWrapper scorer : subScorers) {
-      children.add(new ChildScorer(scorer.scorer, "SHOULD"));
+
+    @Override
+    public long cost() {
+      return cost;
     }
-    return children;
-  }
 
-  @Override
-  public int nextPosition() throws IOException {
-    return -1;
-  }
+    @Override
+    public int docID() {
+     return subScorers.top().doc;
+    }
 
-  @Override
-  public int startOffset() throws IOException {
-    return -1;
-  }
+    @Override
+    public int nextDoc() throws IOException {
+      ScorerWrapper top = subScorers.top();
+      final int doc = top.doc;
+      do {
+        top.doc = top.approximation.nextDoc();
+        top = subScorers.updateTop();
+      } while (top.doc == doc);
 
-  @Override
-  public int endOffset() throws IOException {
-    return -1;
+      return top.doc;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      ScorerWrapper top = subScorers.top();
+      do {
+        top.doc = top.approximation.advance(target);
+        top = subScorers.updateTop();
+      } while (top.doc < target);
+
+      return top.doc;
+    }
   }
 
   @Override
-  public BytesRef getPayload() throws IOException {
-    return null;
+  public TwoPhaseDocIdSetIterator asTwoPhaseIterator() {
+    boolean hasApproximation = false;
+    for (ScorerWrapper w : subScorers) {
+      if (w.twoPhaseView != null) {
+        hasApproximation = true;
+        break;
+      }
+    }
+
+    if (hasApproximation == false) {
+      // none of the sub scorers supports approximations
+      return null;
+    }
+
+    return new TwoPhaseDocIdSetIterator() {
+
+      @Override
+      public DocIdSetIterator approximation() {
+        // note it is important to share the same pq as this scorer so that
+        // rebalancing the pq through the approximation will also rebalance
+        // the pq in this scorer.
+        return new DisjunctionDISIApproximation(subScorers);
+      }
+
+      @Override
+      public boolean matches() throws IOException {
+        ScorerWrapper topScorers = subScorers.topList();
+        // remove the head of the list as long as it does not match
+        while (topScorers.twoPhaseView != null && topScorers.twoPhaseView.matches() == false) {
+          topScorers = topScorers.next;
+          if (topScorers == null) {
+            return false;
+          }
+        }
+        // now we know we have at least one match since the first element of 'matchList' matches
+        if (needsScores) {
+          // if scores or freqs are needed, we also need to remove scorers
+          // from the top list that do not actually match
+          ScorerWrapper previous = topScorers;
+          for (ScorerWrapper w = topScorers.next; w != null; w = w.next) {
+            if (w.twoPhaseView != null && w.twoPhaseView.matches() == false) {
+              // w does not match, remove it
+              previous.next = w.next;
+            } else {
+              previous = w;
+            }
+          }
+
+          // We need to explicitely set the list of top scorers to avoid the
+          // laziness of DisjunctionScorer.score() that would take all scorers
+          // positioned on the same doc as the top of the pq, including
+          // non-matching scorers
+          DisjunctionScorer.this.topScorers = topScorers;
+        }
+        return true;
+      }
+    };
   }
 
   @Override
   public final long cost() {
-    long sum = 0;
-    for (ScorerWrapper scorer : subScorers) {
-      sum += scorer.cost;
-    }
-    return sum;
-  } 
-  
+    return cost;
+  }
+
   @Override
   public final int docID() {
-   return doc;
+   return subScorers.top().doc;
   }
- 
+
   @Override
   public final int nextDoc() throws IOException {
-    assert doc != NO_MORE_DOCS;
-
+    topScorers = null;
     ScorerWrapper top = subScorers.top();
-    final int doc = this.doc;
-    while (top.doc == doc) {
+    final int doc = top.doc;
+    do {
       top.doc = top.scorer.nextDoc();
-      if (top.doc == NO_MORE_DOCS) {
-        subScorers.pop();
-        if (subScorers.size() == 0) {
-          return this.doc = NO_MORE_DOCS;
-        }
-        top = subScorers.top();
-      } else {
-        top = subScorers.updateTop();
-      }
-    }
+      top = subScorers.updateTop();
+    } while (top.doc == doc);
 
-    freq = -1;
-    return this.doc = top.doc;
+    return top.doc;
   }
-  
+
   @Override
   public final int advance(int target) throws IOException {
-    assert doc != NO_MORE_DOCS;
-
+    topScorers = null;
     ScorerWrapper top = subScorers.top();
-    while (top.doc < target) {
+    do {
       top.doc = top.scorer.advance(target);
-      if (top.doc == NO_MORE_DOCS) {
-        subScorers.pop();
-        if (subScorers.size() == 0) {
-          return this.doc = NO_MORE_DOCS;
-        }
-        top = subScorers.top();
-      } else {
-        top = subScorers.updateTop();
-      }
-    }
+      top = subScorers.updateTop();
+    } while (top.doc < target);
 
-    freq = -1;
-    return this.doc = top.doc;
+    return top.doc;
   }
 
   @Override
   public final int freq() throws IOException {
-    if (freq < 0) {
+    if (topScorers == null) {
       topScorers = subScorers.topList();
-      int freq = 1;
-      for (ScorerWrapper w = topScorers.next; w != null; w = w.next) {
-        freq += 1;
-      }
-      this.freq = freq;
+    }
+    int freq = 1;
+    for (ScorerWrapper w = topScorers.next; w != null; w = w.next) {
+      freq += 1;
     }
     return freq;
   }
 
   @Override
   public final float score() throws IOException {
-    final int freq = freq(); // compute the top scorers if necessary
-    return score(topScorers, freq);
+    if (topScorers == null) {
+      topScorers = subScorers.topList();
+    }
+    return score(topScorers);
   }
 
   /** Compute the score for the given linked list of scorers. */
-  protected abstract float score(ScorerWrapper topList, int freq) throws IOException;
+  protected abstract float score(ScorerWrapper topList) throws IOException;
+
+  @Override
+  public final Collection<ChildScorer> getChildren() {
+    ArrayList<ChildScorer> children = new ArrayList<>();
+    for (ScorerWrapper scorer : subScorers) {
+      children.add(new ChildScorer(scorer.scorer, "SHOULD"));
+    }
+    return children;
+  }
+
+  @Override
+  public int nextPosition() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return -1;
+  }
 
+  @Override
+  public int endOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public BytesRef getPayload() throws IOException {
+    return null;
+  }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java?rev=1660184&r1=1660183&r2=1660184&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java Mon Feb 16 19:29:34 2015
@@ -18,6 +18,7 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
+import java.util.List;
 
 import org.apache.lucene.search.ScorerPriorityQueue.ScorerWrapper;
 
@@ -32,16 +33,18 @@ final class DisjunctionSumScorer extends
    * @param subScorers Array of at least two subscorers.
    * @param coord Table of coordination factors
    */
-  DisjunctionSumScorer(Weight weight, Scorer[] subScorers, float[] coord) {
-    super(weight, subScorers);
+  DisjunctionSumScorer(Weight weight, List<Scorer> subScorers, float[] coord, boolean needsScores) {
+    super(weight, subScorers, needsScores);
     this.coord = coord;
   }
 
   @Override
-  protected float score(ScorerWrapper topList, int freq) throws IOException {
+  protected float score(ScorerWrapper topList) throws IOException {
     double score = 0;
+    int freq = 0;
     for (ScorerWrapper w = topList; w != null; w = w.next) {
       score += w.scorer.score();
+      freq += 1;
     }
     return (float)score * coord[freq];
   }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ScorerPriorityQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ScorerPriorityQueue.java?rev=1660184&r1=1660183&r2=1660184&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ScorerPriorityQueue.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ScorerPriorityQueue.java Mon Feb 16 19:29:34 2015
@@ -35,10 +35,23 @@ final class ScorerPriorityQueue implemen
     int doc; // the current doc, used for comparison
     ScorerWrapper next; // reference to a next element, see #topList
 
+    // An approximation of the scorer, or the scorer itself if it does not
+    // support two-phase iteration
+    final DocIdSetIterator approximation;
+    // A two-phase view of the scorer, or null if the scorer does not support
+    // two-phase iteration
+    final TwoPhaseDocIdSetIterator twoPhaseView;
+
     ScorerWrapper(Scorer scorer) {
       this.scorer = scorer;
       this.cost = scorer.cost();
       this.doc = -1;
+      this.twoPhaseView = scorer.asTwoPhaseIterator();
+      if (twoPhaseView != null) {
+        approximation = twoPhaseView.approximation();
+      } else {
+        approximation = scorer;
+      }
     }
   }
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java?rev=1660184&r1=1660183&r2=1660184&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java Mon Feb 16 19:29:34 2015
@@ -591,7 +591,7 @@ public class TestBooleanQuery extends Lu
     dir.close();
   }
 
-  public void testConjunctionSupportsApproximations() throws IOException {
+  public void testConjunctionPropagatesApproximations() throws IOException {
     Directory dir = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
@@ -613,6 +613,35 @@ public class TestBooleanQuery extends Lu
 
     final Weight weight = searcher.createNormalizedWeight(q, random().nextBoolean());
     final Scorer scorer = weight.scorer(reader.leaves().get(0), null);
+    assertNotNull(scorer.asTwoPhaseIterator());
+
+    reader.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testDisjunctionPropagatesApproximations() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    Field f = newTextField("field", "a b c", Field.Store.NO);
+    doc.add(f);
+    w.addDocument(doc);
+    w.commit();
+
+    DirectoryReader reader = w.getReader();
+    final IndexSearcher searcher = new IndexSearcher(reader);
+
+    PhraseQuery pq = new PhraseQuery();
+    pq.add(new Term("field", "a"));
+    pq.add(new Term("field", "b"));
+
+    BooleanQuery q = new BooleanQuery();
+    q.add(pq, Occur.SHOULD);
+    q.add(new TermQuery(new Term("field", "c")), Occur.SHOULD);
+
+    final Weight weight = searcher.createNormalizedWeight(q, random().nextBoolean());
+    final Scorer scorer = weight.scorer(reader.leaves().get(0), null);
     assertNotNull(scorer.asTwoPhaseIterator());
 
     reader.close();

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java?rev=1660184&r1=1660183&r2=1660184&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java Mon Feb 16 19:29:34 2015
@@ -17,6 +17,7 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.BitSet;
 import java.util.Random;
 
@@ -193,4 +194,32 @@ public abstract class SearchEquivalenceT
       assertTrue(bitset.get(td1.scoreDocs[i].doc));
     }
   }
+
+  /**
+   * Assert that two queries return the same documents and with the same scores.
+   */
+  protected void assertSameScores(Query q1, Query q2) throws Exception {
+    assertSameSet(q1, q2);
+
+    assertSameScores(q1, q2, null);
+    // also test with a filter to test advancing
+    assertSameScores(q1, q2, randomFilter());
+  }
+
+  protected void assertSameScores(Query q1, Query q2, Filter filter) throws Exception {
+    if (filter != null && random().nextBoolean()) {
+      q1 = new FilteredQuery(q1, filter, TestUtil.randomFilterStrategy(random()));
+      q2 = new FilteredQuery(q2, filter,  TestUtil.randomFilterStrategy(random()));
+      filter = null;
+    }
+    
+    // not efficient, but simple!
+    TopDocs td1 = s1.search(q1, filter, reader.maxDoc());
+    TopDocs td2 = s2.search(q2, filter, reader.maxDoc());
+    assertEquals(td1.totalHits, td2.totalHits);
+    for (int i = 0; i < td1.scoreDocs.length; ++i) {
+      assertEquals(td1.scoreDocs[i].doc, td2.scoreDocs[i].doc);
+      assertEquals(td1.scoreDocs[i].score, td2.scoreDocs[i].score, 10e-5);
+    }
+  }
 }