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

svn commit: r1657902 - 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/queries/ lucene/queries/src/java/org/apache/lucene/queries/

Author: rmuir
Date: Fri Feb  6 17:20:32 2015
New Revision: 1657902

URL: http://svn.apache.org/r1657902
Log:
LUCENE-6223: Move BooleanWeight to its own file

Added:
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
      - copied unchanged from r1657895, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.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/BooleanQuery.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/package.html
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java
    lucene/dev/branches/branch_5x/lucene/queries/   (props changed)
    lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.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=1657902&r1=1657901&r2=1657902&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Fri Feb  6 17:20:32 2015
@@ -52,6 +52,9 @@ API Changes
   instead. This will be as efficient now that queries can opt out from
   scoring. (Adrien Grand)
 
+* LUCENE-6223: Move BooleanQuery.BooleanWeight to BooleanWeight.
+  (Robert Muir)
+
 Other
 
 * LUCENE-6193: Collapse identical catch branches in try-catch statements.

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1657902&r1=1657901&r2=1657902&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java Fri Feb  6 17:20:32 2015
@@ -162,379 +162,9 @@ public class BooleanQuery extends Query
   @Override
   public final Iterator<BooleanClause> iterator() { return clauses().iterator(); }
 
-  /**
-   * Expert: the Weight for BooleanQuery, used to
-   * normalize, score and explain these queries.
-   *
-   * @lucene.experimental
-   */
-  protected class BooleanWeight extends Weight {
-    /** The Similarity implementation. */
-    protected Similarity similarity;
-    protected ArrayList<Weight> weights;
-    protected int maxCoord;  // num optional + num required
-    private final boolean disableCoord;
-    private final boolean needsScores;
-
-    public BooleanWeight(IndexSearcher searcher, boolean needsScores, boolean disableCoord)
-      throws IOException {
-      super(BooleanQuery.this);
-      this.needsScores = needsScores;
-      this.similarity = searcher.getSimilarity();
-      this.disableCoord = disableCoord;
-      weights = new ArrayList<>(clauses.size());
-      for (int i = 0 ; i < clauses.size(); i++) {
-        BooleanClause c = clauses.get(i);
-        final boolean queryNeedsScores = needsScores && c.getOccur() != Occur.MUST_NOT;
-        Weight w = c.getQuery().createWeight(searcher, queryNeedsScores);
-        weights.add(w);
-        if (!c.isProhibited()) {
-          maxCoord++;
-        }
-      }
-    }
-
-    @Override
-    public float getValueForNormalization() throws IOException {
-      float sum = 0.0f;
-      for (int i = 0 ; i < weights.size(); i++) {
-        // call sumOfSquaredWeights for all clauses in case of side effects
-        float s = weights.get(i).getValueForNormalization();         // sum sub weights
-        if (!clauses.get(i).isProhibited()) {
-          // only add to sum for non-prohibited clauses
-          sum += s;
-        }
-      }
-
-      sum *= getBoost() * getBoost();             // boost each sub-weight
-
-      return sum ;
-    }
-
-    public float coord(int overlap, int maxOverlap) {
-      // LUCENE-4300: in most cases of maxOverlap=1, BQ rewrites itself away,
-      // so coord() is not applied. But when BQ cannot optimize itself away
-      // for a single clause (minNrShouldMatch, prohibited clauses, etc), it's
-      // important not to apply coord(1,1) for consistency, it might not be 1.0F
-      return maxOverlap == 1 ? 1F : similarity.coord(overlap, maxOverlap);
-    }
-
-    @Override
-    public void normalize(float norm, float topLevelBoost) {
-      topLevelBoost *= getBoost();                         // incorporate boost
-      for (Weight w : weights) {
-        // normalize all clauses, (even if prohibited in case of side affects)
-        w.normalize(norm, topLevelBoost);
-      }
-    }
-
-    @Override
-    public Explanation explain(LeafReaderContext context, int doc)
-      throws IOException {
-      final int minShouldMatch =
-        BooleanQuery.this.getMinimumNumberShouldMatch();
-      ComplexExplanation sumExpl = new ComplexExplanation();
-      sumExpl.setDescription("sum of:");
-      int coord = 0;
-      float sum = 0.0f;
-      boolean fail = false;
-      int shouldMatchCount = 0;
-      Iterator<BooleanClause> cIter = clauses.iterator();
-      for (Iterator<Weight> wIter = weights.iterator(); wIter.hasNext();) {
-        Weight w = wIter.next();
-        BooleanClause c = cIter.next();
-        if (w.scorer(context, context.reader().getLiveDocs()) == null) {
-          if (c.isRequired()) {
-            fail = true;
-            Explanation r = new Explanation(0.0f, "no match on required clause (" + c.getQuery().toString() + ")");
-            sumExpl.addDetail(r);
-          }
-          continue;
-        }
-        Explanation e = w.explain(context, doc);
-        if (e.isMatch()) {
-          if (!c.isProhibited()) {
-            sumExpl.addDetail(e);
-            sum += e.getValue();
-            coord++;
-          } else {
-            Explanation r =
-              new Explanation(0.0f, "match on prohibited clause (" + c.getQuery().toString() + ")");
-            r.addDetail(e);
-            sumExpl.addDetail(r);
-            fail = true;
-          }
-          if (c.getOccur() == Occur.SHOULD) {
-            shouldMatchCount++;
-          }
-        } else if (c.isRequired()) {
-          Explanation r = new Explanation(0.0f, "no match on required clause (" + c.getQuery().toString() + ")");
-          r.addDetail(e);
-          sumExpl.addDetail(r);
-          fail = true;
-        }
-      }
-      if (fail) {
-        sumExpl.setMatch(Boolean.FALSE);
-        sumExpl.setValue(0.0f);
-        sumExpl.setDescription
-          ("Failure to meet condition(s) of required/prohibited clause(s)");
-        return sumExpl;
-      } else if (shouldMatchCount < minShouldMatch) {
-        sumExpl.setMatch(Boolean.FALSE);
-        sumExpl.setValue(0.0f);
-        sumExpl.setDescription("Failure to match minimum number "+
-                               "of optional clauses: " + minShouldMatch);
-        return sumExpl;
-      }
-      
-      sumExpl.setMatch(0 < coord ? Boolean.TRUE : Boolean.FALSE);
-      sumExpl.setValue(sum);
-      
-      final float coordFactor = disableCoord ? 1.0f : coord(coord, maxCoord);
-      if (coordFactor == 1.0f) {
-        return sumExpl;                             // eliminate wrapper
-      } else {
-        ComplexExplanation result = new ComplexExplanation(sumExpl.isMatch(),
-                                                           sum*coordFactor,
-                                                           "product of:");
-        result.addDetail(sumExpl);
-        result.addDetail(new Explanation(coordFactor,
-                                         "coord("+coord+"/"+maxCoord+")"));
-        return result;
-      }
-    }
-
-    /** Try to build a boolean scorer for this weight. Returns null if {@link BooleanScorer}
-     *  cannot be used. */
-    // pkg-private for forcing use of BooleanScorer in tests
-    BooleanScorer booleanScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-      List<BulkScorer> optional = new ArrayList<BulkScorer>();
-      Iterator<BooleanClause> cIter = clauses.iterator();
-      for (Weight w  : weights) {
-        BooleanClause c =  cIter.next();
-        BulkScorer subScorer = w.bulkScorer(context, acceptDocs);
-        if (subScorer == null) {
-          if (c.isRequired()) {
-            return null;
-          }
-        } else if (c.isRequired()) {
-          // TODO: there are some cases where BooleanScorer
-          // would handle conjunctions faster than
-          // BooleanScorer2...
-          return null;
-        } else if (c.isProhibited()) {
-          // TODO: there are some cases where BooleanScorer could do this faster
-          return null;
-        } else {
-          optional.add(subScorer);
-        }
-      }
-
-      if (optional.size() == 0) {
-        return null;
-      }
-
-      if (minNrShouldMatch > optional.size()) {
-        return null;
-      }
-
-      return new BooleanScorer(this, disableCoord, maxCoord, optional, Math.max(1, minNrShouldMatch));
-    }
-
-    @Override
-    public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-      final BooleanScorer bulkScorer = booleanScorer(context, acceptDocs);
-      if (bulkScorer != null) { // BooleanScorer is applicable
-        // TODO: what is the right heuristic here?
-        final long costThreshold;
-        if (minNrShouldMatch <= 1) {
-          // when all clauses are optional, use BooleanScorer aggressively
-          // TODO: is there actually a threshold under which we should rather
-          // use the regular scorer?
-          costThreshold = -1;
-        } else {
-          // when a minimum number of clauses should match, BooleanScorer is
-          // going to score all windows that have at least minNrShouldMatch
-          // matches in the window. But there is no way to know if there is
-          // an intersection (all clauses might match a different doc ID and
-          // there will be no matches in the end) so we should only use
-          // BooleanScorer if matches are very dense
-          costThreshold = context.reader().maxDoc() / 3;
-        }
-
-        if (bulkScorer.cost() > costThreshold) {
-          return bulkScorer;
-        }
-      }
-      return super.bulkScorer(context, acceptDocs);
-    }
-
-    @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-      // initially the user provided value,
-      // but if minNrShouldMatch == optional.size(),
-      // we will optimize and move these to required, making this 0
-      int minShouldMatch = minNrShouldMatch;
-
-      List<Scorer> required = new ArrayList<>();
-      List<Scorer> prohibited = new ArrayList<>();
-      List<Scorer> optional = new ArrayList<>();
-      Iterator<BooleanClause> cIter = clauses.iterator();
-      for (Weight w  : weights) {
-        BooleanClause c =  cIter.next();
-        Scorer subScorer = w.scorer(context, acceptDocs);
-        if (subScorer == null) {
-          if (c.isRequired()) {
-            return null;
-          }
-        } else if (c.isRequired()) {
-          required.add(subScorer);
-        } else if (c.isProhibited()) {
-          prohibited.add(subScorer);
-        } else {
-          optional.add(subScorer);
-        }
-      }
-      
-      // scorer simplifications:
-      
-      if (optional.size() == minShouldMatch) {
-        // any optional clauses are in fact required
-        required.addAll(optional);
-        optional.clear();
-        minShouldMatch = 0;
-      }
-      
-      if (required.isEmpty() && optional.isEmpty()) {
-        // no required and optional clauses.
-        return null;
-      } else if (optional.size() < minShouldMatch) {
-        // either >1 req scorer, or there are 0 req scorers and at least 1
-        // optional scorer. Therefore if there are not enough optional scorers
-        // no documents will be matched by the query
-        return null;
-      }
-      
-      // we don't need scores, so if we have required clauses, drop optional clauses completely
-      if (!needsScores && minShouldMatch == 0 && required.size() > 0) {
-        optional.clear();
-      }
-      
-      // three cases: conjunction, disjunction, or mix
-      
-      // pure conjunction
-      if (optional.isEmpty()) {
-        return excl(req(required, disableCoord), prohibited);
-      }
-      
-      // pure disjunction
-      if (required.isEmpty()) {
-        return excl(opt(optional, minShouldMatch, disableCoord), prohibited);
-      }
-      
-      // conjunction-disjunction mix:
-      // we create the required and optional pieces with coord disabled, and then
-      // combine the two: if minNrShouldMatch > 0, then it's a conjunction: because the
-      // optional side must match. otherwise it's required + optional, factoring the
-      // number of optional terms into the coord calculation
-      
-      Scorer req = excl(req(required, true), prohibited);
-      Scorer opt = opt(optional, minShouldMatch, true);
-
-      // TODO: clean this up: it's horrible
-      if (disableCoord) {
-        if (minShouldMatch > 0) {
-          return new ConjunctionScorer(this, new Scorer[] { req, opt }, 1F);
-        } else {
-          return new ReqOptSumScorer(req, opt);          
-        }
-      } else if (optional.size() == 1) {
-        if (minShouldMatch > 0) {
-          return new ConjunctionScorer(this, new Scorer[] { req, opt }, coord(required.size()+1, maxCoord));
-        } else {
-          float coordReq = coord(required.size(), maxCoord);
-          float coordBoth = coord(required.size() + 1, maxCoord);
-          return new BooleanTopLevelScorers.ReqSingleOptScorer(req, opt, coordReq, coordBoth);
-        }
-      } else {
-        if (minShouldMatch > 0) {
-          return new BooleanTopLevelScorers.CoordinatingConjunctionScorer(this, coords(), req, required.size(), opt);
-        } else {
-          return new BooleanTopLevelScorers.ReqMultiOptScorer(req, opt, required.size(), coords()); 
-        }
-      }
-    }
-    
-    private Scorer req(List<Scorer> required, boolean disableCoord) {
-      if (required.size() == 1) {
-        Scorer req = required.get(0);
-        if (!disableCoord && maxCoord > 1) {
-          return new BooleanTopLevelScorers.BoostedScorer(req, coord(1, maxCoord));
-        } else {
-          return req;
-        }
-      } else {
-        return new ConjunctionScorer(this, 
-                                     required.toArray(new Scorer[required.size()]),
-                                     disableCoord ? 1.0F : coord(required.size(), maxCoord));
-      }
-    }
-    
-    private Scorer excl(Scorer main, List<Scorer> prohibited) throws IOException {
-      if (prohibited.isEmpty()) {
-        return main;
-      } else if (prohibited.size() == 1) {
-        return new ReqExclScorer(main, prohibited.get(0));
-      } 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));
-      }
-    }
-    
-    private Scorer opt(List<Scorer> optional, int minShouldMatch, boolean disableCoord) throws IOException {
-      if (optional.size() == 1) {
-        Scorer opt = optional.get(0);
-        if (!disableCoord && maxCoord > 1) {
-          return new BooleanTopLevelScorers.BoostedScorer(opt, coord(1, maxCoord));
-        } else {
-          return opt;
-        }
-      } else {
-        float coords[];
-        if (disableCoord) {
-          coords = new float[optional.size()+1];
-          Arrays.fill(coords, 1F);
-        } else {
-          coords = coords();
-        }
-        if (minShouldMatch > 1) {
-          return new MinShouldMatchSumScorer(this, optional, minShouldMatch, coords);
-        } else {
-          return new DisjunctionSumScorer(this, 
-                                          optional.toArray(new Scorer[optional.size()]), 
-                                          coords);
-        }
-      }
-    }
-    
-    private float[] coords() {
-      float[] coords = new float[maxCoord+1];
-      coords[0] = 0F;
-      for (int i = 1; i < coords.length; i++) {
-        coords[i] = coord(i, maxCoord);
-      }
-      return coords;
-    }
-  }
-
   @Override
   public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
-    return new BooleanWeight(searcher, needsScores, disableCoord);
+    return new BooleanWeight(this, searcher, needsScores, disableCoord);
   }
 
   @Override

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java?rev=1657902&r1=1657901&r2=1657902&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java Fri Feb  6 17:20:32 2015
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 
-import org.apache.lucene.search.BooleanQuery.BooleanWeight;
+import org.apache.lucene.search.BooleanWeight;
 import org.apache.lucene.util.PriorityQueue;
 
 /**

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/package.html?rev=1657902&r1=1657901&r2=1657902&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/package.html (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/package.html Fri Feb  6 17:20:32 2015
@@ -569,7 +569,7 @@ on the built-in available scoring models
    by the {@link org.apache.lucene.search.Weight Weight} object depends on what type of Query was
    submitted. In most real world applications with multiple query terms, the 
    {@link org.apache.lucene.search.Scorer Scorer} is going to be a <code>BooleanScorer2</code> created
-   from {@link org.apache.lucene.search.BooleanQuery.BooleanWeight BooleanWeight} (see the section on
+   from {@link org.apache.lucene.search.BooleanWeight BooleanWeight} (see the section on
    <a href="#customQueriesExpert">custom queries</a> for info on changing this).
 </p>
 <p>Assuming a BooleanScorer2, we first initialize the Coordinator, which is used to apply the coord() 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java?rev=1657902&r1=1657901&r2=1657902&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java Fri Feb  6 17:20:32 2015
@@ -283,7 +283,7 @@ public class TestBooleanQueryVisitSubsco
 
     @Override
     public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
-      return new BooleanWeight(searcher, needsScores, false) {
+      return new BooleanWeight(this, searcher, needsScores, false) {
         @Override
         public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
           Scorer scorer = scorer(context, acceptDocs);

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java?rev=1657902&r1=1657901&r2=1657902&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java Fri Feb  6 17:20:32 2015
@@ -35,7 +35,6 @@ import org.apache.lucene.index.RandomInd
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
-import org.apache.lucene.search.BooleanQuery.BooleanWeight;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.search.similarities.Similarity.SimWeight;

Modified: lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java?rev=1657902&r1=1657901&r2=1657902&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java Fri Feb  6 17:20:32 2015
@@ -55,7 +55,7 @@ public class BoostingQuery extends Query
       BooleanQuery result = new BooleanQuery() {
         @Override
         public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
-          return new BooleanWeight(searcher, needsScores, false) {
+          return new BooleanWeight(this, searcher, needsScores, false) {
 
             @Override
             public float coord(int overlap, int max) {