You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2011/01/02 15:16:31 UTC

svn commit: r1054412 - in /lucene/dev/branches/branch_3x: ./ lucene/ lucene/contrib/remote/src/test/org/apache/lucene/search/ lucene/src/java/org/apache/lucene/search/ lucene/src/test/org/apache/lucene/search/ solr/

Author: uschindler
Date: Sun Jan  2 14:16:30 2011
New Revision: 1054412

URL: http://svn.apache.org/viewvc?rev=1054412&view=rev
Log:
LUCENE-2838: ConstantScoreQuery now directly supports wrapping a Query instance for stripping off scores

Added:
    lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/search/TestConstantScoreQuery.java
      - copied unchanged from r1054406, lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestConstantScoreQuery.java
Removed:
    lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/search/TestBooleanPrefixQuery.java
Modified:
    lucene/dev/branches/branch_3x/   (props changed)
    lucene/dev/branches/branch_3x/lucene/   (props changed)
    lucene/dev/branches/branch_3x/lucene/CHANGES.txt
    lucene/dev/branches/branch_3x/lucene/contrib/remote/src/test/org/apache/lucene/search/TestRemoteSearchable.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java
    lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java
    lucene/dev/branches/branch_3x/solr/   (props changed)

Modified: lucene/dev/branches/branch_3x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/CHANGES.txt?rev=1054412&r1=1054411&r2=1054412&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/lucene/CHANGES.txt Sun Jan  2 14:16:30 2011
@@ -394,6 +394,12 @@ New features
   Using this wrapper its easy to add fuzzy/wildcard to e.g. a SpanNearQuery.
   (Robert Muir, Uwe Schindler)
   
+* LUCENE-2838: ConstantScoreQuery now directly supports wrapping a Query
+  instance for stripping off scores. The use of a QueryWrapperFilter
+  is no longer needed and discouraged for that use case. Directly wrapping
+  Query improves performance, as out-of-order collection is now supported.
+  (Uwe Schindler)
+  
 Optimizations
 
 * LUCENE-2494: Use CompletionService in ParallelMultiSearcher instead of

Modified: lucene/dev/branches/branch_3x/lucene/contrib/remote/src/test/org/apache/lucene/search/TestRemoteSearchable.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/remote/src/test/org/apache/lucene/search/TestRemoteSearchable.java?rev=1054412&r1=1054411&r2=1054412&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/remote/src/test/org/apache/lucene/search/TestRemoteSearchable.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/remote/src/test/org/apache/lucene/search/TestRemoteSearchable.java Sun Jan  2 14:16:30 2011
@@ -122,8 +122,7 @@ public class TestRemoteSearchable extend
     Searchable[] searchables = { lookupRemote() };
     Searcher searcher = new MultiSearcher(searchables);
     ScoreDoc[] hits = searcher.search(
-          new ConstantScoreQuery(new QueryWrapperFilter(
-                                   new TermQuery(new Term("test", "test")))), null, 1000).scoreDocs;
+          new ConstantScoreQuery(new TermQuery(new Term("test", "test"))), null, 1000).scoreDocs;
     assertEquals(1, hits.length);
   }
 }

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java?rev=1054412&r1=1054411&r2=1054412&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ConstantScoreAutoRewrite.java Sun Jan  2 14:16:30 2011
@@ -98,7 +98,7 @@ class ConstantScoreAutoRewrite extends T
           addClause(bq, term, 1.0f);
         }
         // Strip scores
-        result = new ConstantScoreQuery(new QueryWrapperFilter(bq));
+        result = new ConstantScoreQuery(bq);
         result.setBoost(query.getBoost());
       }
       query.incTotalNumberOfTerms(col.pendingTerms.size());

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1054412&r1=1054411&r2=1054412&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java Sun Jan  2 14:16:30 2011
@@ -19,13 +19,15 @@ package org.apache.lucene.search;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.util.ToStringUtils;
 
 import java.io.IOException;
 import java.util.Set;
 
 /**
- * A query that wraps a filter and simply returns a constant score equal to the
- * query boost for every document in the filter.
+ * A query that wraps another query or a filter and simply returns a constant score equal to the
+ * query boost for every document that matches the filter or query.
+ * For queries it therefore simply strips of all scores and returns a constant one.
  *
  * <p><b>NOTE</b>: if the wrapped filter is an instance of
  * {@link CachingWrapperFilter}, you'll likely want to
@@ -35,34 +37,72 @@ import java.util.Set;
  */
 public class ConstantScoreQuery extends Query {
   protected final Filter filter;
+  protected final Query query;
 
+  /** Strips off scores from the passed in Query. The hits will get a constant score
+   * dependent on the boost factor of this query. */
+  public ConstantScoreQuery(Query query) {
+    if (query == null)
+      throw new NullPointerException("Query may not be null");
+    this.filter = null;
+    this.query = query;
+  }
+
+  /** Wraps a Filter as a Query. The hits will get a constant score
+   * dependent on the boost factor of this query.
+   * If you simply want to strip off scores from a Query, no longer use
+   * {@code new ConstantScoreQuery(new QueryWrapperFilter(query))}, instead
+   * use {@link #ConstantScoreQuery(Query)}!
+   */
   public ConstantScoreQuery(Filter filter) {
-    this.filter=filter;
+    if (filter == null)
+      throw new NullPointerException("Filter may not be null");
+    this.filter = filter;
+    this.query = null;
   }
 
-  /** Returns the encapsulated filter */
+  /** Returns the encapsulated filter, returns {@code null} if a query is wrapped. */
   public Filter getFilter() {
     return filter;
   }
 
+  /** Returns the encapsulated query, returns {@code null} if a filter is wrapped. */
+  public Query getQuery() {
+    return query;
+  }
+
   @Override
   public Query rewrite(IndexReader reader) throws IOException {
+    if (query != null) {
+      Query rewritten = query.rewrite(reader);
+      if (rewritten != query) {
+        rewritten = new ConstantScoreQuery(rewritten);
+        rewritten.setBoost(this.getBoost());
+        return rewritten;
+      }
+    }
     return this;
   }
 
   @Override
   public void extractTerms(Set<Term> terms) {
-    // OK to not add any terms when used for MultiSearcher,
-    // but may not be OK for highlighting
+    // TODO: OK to not add any terms when wrapped a filter
+    // and used with MultiSearcher, but may not be OK for
+    // highlighting.
+    // If a query was wrapped, we delegate to query.
+    if (query != null)
+      query.extractTerms(terms);
   }
 
   protected class ConstantWeight extends Weight {
-    private Similarity similarity;
+    private final Weight innerWeight;
+    private final Similarity similarity;
     private float queryNorm;
     private float queryWeight;
     
-    public ConstantWeight(Searcher searcher) {
+    public ConstantWeight(Searcher searcher) throws IOException {
       this.similarity = getSimilarity(searcher);
+      this.innerWeight = (query == null) ? null : query.createWeight(searcher);
     }
 
     @Override
@@ -77,6 +117,8 @@ public class ConstantScoreQuery extends 
 
     @Override
     public float sumOfSquaredWeights() throws IOException {
+      // we calculate sumOfSquaredWeights of the inner weight, but ignore it (just to initialize everything)
+      if (innerWeight != null) innerWeight.sumOfSquaredWeights();
       queryWeight = getBoost();
       return queryWeight * queryWeight;
     }
@@ -85,31 +127,48 @@ public class ConstantScoreQuery extends 
     public void normalize(float norm) {
       this.queryNorm = norm;
       queryWeight *= this.queryNorm;
+      // we normalize the inner weight, but ignore it (just to initialize everything)
+      if (innerWeight != null) innerWeight.normalize(norm);
     }
 
     @Override
     public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
-      return new ConstantScorer(similarity, reader, this);
+      final DocIdSetIterator disi;
+      if (filter != null) {
+        assert query == null;
+        final DocIdSet dis = filter.getDocIdSet(reader);
+        if (dis == null)
+          return null;
+        disi = dis.iterator();
+      } else {
+        assert query != null && innerWeight != null;
+        disi =
+          innerWeight.scorer(reader, scoreDocsInOrder, topScorer);
+      }
+      if (disi == null)
+        return null;
+      return new ConstantScorer(similarity, disi, this);
+    }
+    
+    @Override
+    public boolean scoresDocsOutOfOrder() {
+      return (innerWeight != null) ? innerWeight.scoresDocsOutOfOrder() : false;
     }
 
     @Override
     public Explanation explain(IndexReader reader, int doc) throws IOException {
-      
-      ConstantScorer cs = new ConstantScorer(similarity, reader, this);
-      boolean exists = cs.docIdSetIterator.advance(doc) == doc;
-
-      ComplexExplanation result = new ComplexExplanation();
+      final Scorer cs = scorer(reader, true, false);
+      final boolean exists = (cs != null && cs.advance(doc) == doc);
 
+      final ComplexExplanation result = new ComplexExplanation();
       if (exists) {
-        result.setDescription("ConstantScoreQuery(" + filter
-        + "), product of:");
+        result.setDescription(ConstantScoreQuery.this.toString() + ", product of:");
         result.setValue(queryWeight);
         result.setMatch(Boolean.TRUE);
         result.addDetail(new Explanation(getBoost(), "boost"));
-        result.addDetail(new Explanation(queryNorm,"queryNorm"));
+        result.addDetail(new Explanation(queryNorm, "queryNorm"));
       } else {
-        result.setDescription("ConstantScoreQuery(" + filter
-        + ") doesn't match id " + doc);
+        result.setDescription(ConstantScoreQuery.this.toString() + " doesn't match id " + doc);
         result.setValue(0);
         result.setMatch(Boolean.FALSE);
       }
@@ -120,22 +179,11 @@ public class ConstantScoreQuery extends 
   protected class ConstantScorer extends Scorer {
     final DocIdSetIterator docIdSetIterator;
     final float theScore;
-    int doc = -1;
 
-    public ConstantScorer(Similarity similarity, IndexReader reader, Weight w) throws IOException {
+    public ConstantScorer(Similarity similarity, DocIdSetIterator docIdSetIterator, Weight w) throws IOException {
       super(similarity,w);
       theScore = w.getValue();
-      DocIdSet docIdSet = filter.getDocIdSet(reader);
-      if (docIdSet == null) {
-        docIdSetIterator = DocIdSet.EMPTY_DOCIDSET.iterator();
-      } else {
-        DocIdSetIterator iter = docIdSet.iterator();
-        if (iter == null) {
-          docIdSetIterator = DocIdSet.EMPTY_DOCIDSET.iterator();
-        } else {
-          docIdSetIterator = iter;
-        }
-      }
+      this.docIdSetIterator = docIdSetIterator;
     }
 
     @Override
@@ -157,34 +205,88 @@ public class ConstantScoreQuery extends 
     public int advance(int target) throws IOException {
       return docIdSetIterator.advance(target);
     }
+    
+    private Collector wrapCollector(final Collector collector) {
+      return new Collector() {
+        @Override
+        public void setScorer(Scorer scorer) throws IOException {
+          // we must wrap again here, but using the scorer passed in as parameter:
+          collector.setScorer(new ConstantScorer(ConstantScorer.this.getSimilarity(),
+            scorer, ConstantScorer.this.weight));
+        }
+        
+        @Override
+        public void collect(int doc) throws IOException {
+          collector.collect(doc);
+        }
+        
+        @Override
+        public void setNextReader(IndexReader reader, int docBase) throws IOException {
+          collector.setNextReader(reader, docBase);
+        }
+        
+        @Override
+        public boolean acceptsDocsOutOfOrder() {
+          return collector.acceptsDocsOutOfOrder();
+        }
+      };
+    }
+
+    // this optimization allows out of order scoring as top scorer!
+    @Override
+    public void score(Collector collector) throws IOException {
+      if (docIdSetIterator instanceof Scorer) {
+        ((Scorer) docIdSetIterator).score(wrapCollector(collector));
+      } else {
+        super.score(collector);
+      }
+    }
+
+    // this optimization allows out of order scoring as top scorer,
+    // TODO: theoretically this method should not be called because its protected and
+    // this class does not use it, it should be public in Scorer!
+    @Override
+    protected boolean score(Collector collector, int max, int firstDocID) throws IOException {
+      if (docIdSetIterator instanceof Scorer) {
+        return ((Scorer) docIdSetIterator).score(wrapCollector(collector), max, firstDocID);
+      } else {
+        return super.score(collector, max, firstDocID);
+      }
+    }
   }
 
   @Override
-  public Weight createWeight(Searcher searcher) {
+  public Weight createWeight(Searcher searcher) throws IOException {
     return new ConstantScoreQuery.ConstantWeight(searcher);
   }
 
-  /** Prints a user-readable version of this query. */
   @Override
   public String toString(String field) {
-    return "ConstantScore(" + filter.toString()
-      + (getBoost()==1.0 ? ")" : "^" + getBoost());
+    return new StringBuilder("ConstantScore(")
+      .append((query == null) ? filter.toString() : query.toString(field))
+      .append(')')
+      .append(ToStringUtils.boost(getBoost()))
+      .toString();
   }
 
-  /** Returns true if <code>o</code> is equal to this. */
   @Override
   public boolean equals(Object o) {
     if (this == o) return true;
-    if (!(o instanceof ConstantScoreQuery)) return false;
-    ConstantScoreQuery other = (ConstantScoreQuery)o;
-    return this.getBoost()==other.getBoost() && filter.equals(other.filter);
+    if (!super.equals(o))
+      return false;
+    if (o instanceof ConstantScoreQuery) {
+      final ConstantScoreQuery other = (ConstantScoreQuery) o;
+      return 
+        ((this.filter == null) ? other.filter == null : this.filter.equals(other.filter)) &&
+        ((this.query == null) ? other.query == null : this.query.equals(other.query));
+    }
+    return false;
   }
 
-  /** Returns a hash code value for this object. */
   @Override
   public int hashCode() {
-    // Simple add is OK since no existing filter hashcode has a float component.
-    return filter.hashCode() + Float.floatToIntBits(getBoost());
+    return 31 * super.hashCode() +
+      ((query == null) ? filter : query).hashCode();
   }
 
 }

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java?rev=1054412&r1=1054411&r2=1054412&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java Sun Jan  2 14:16:30 2011
@@ -198,7 +198,7 @@ public abstract class MultiTermQuery ext
     
     @Override
     protected void addClause(BooleanQuery topLevel, Term term, float boost) {
-      final Query q = new ConstantScoreQuery(new QueryWrapperFilter(new TermQuery(term)));
+      final Query q = new ConstantScoreQuery(new TermQuery(term));
       q.setBoost(boost);
       topLevel.add(q, BooleanClause.Occur.SHOULD);
     }

Modified: lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java?rev=1054412&r1=1054411&r2=1054412&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java (original)
+++ lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/ScoringRewrite.java Sun Jan  2 14:16:30 2011
@@ -75,7 +75,7 @@ public abstract class ScoringRewrite<Q e
       if (bq.clauses().isEmpty())
         return bq;
       // strip the scores off
-      final Query result = new ConstantScoreQuery(new QueryWrapperFilter(bq));
+      final Query result = new ConstantScoreQuery(bq);
       result.setBoost(query.getBoost());
       return result;
     }