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/05/20 09:46:55 UTC

svn commit: r1680474 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/CHANGES.txt lucene/core/ lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java lucene/core/src/java/org/apache/lucene/search/TermQuery.java

Author: jpountz
Date: Wed May 20 07:46:55 2015
New Revision: 1680474

URL: http://svn.apache.org/r1680474
Log:
LUCENE-6458: Multi-term queries matching few terms per segment now execute like a disjunction.

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/MultiTermQueryConstantScoreWrapper.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TermQuery.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=1680474&r1=1680473&r2=1680474&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Wed May 20 07:46:55 2015
@@ -102,6 +102,9 @@ Optimizations
 * LUCENE-6350: TermsQuery is now compressed with PrefixCodedTerms.
   (Robert Muir, Mike McCandless, Adrien Grand)
 
+* LUCENE-6458: Multi-term queries matching few terms per segment now execute
+  like a disjunction. (Adrien Grand)
+
 Bug Fixes
 
 * LUCENE-6378: Fix all RuntimeExceptions to throw the underlying root cause.

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java?rev=1680474&r1=1680473&r2=1680474&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java Wed May 20 07:46:55 2015
@@ -18,29 +18,48 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.util.Objects;
+import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermContext;
+import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.util.BitDocIdSet;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
 
 /**
- * A wrapper for {@link MultiTermQuery}, that exposes its
- * functionality as a {@link Filter}.
- * <P>
- * <code>MultiTermQueryWrapperFilter</code> is not designed to
- * be used by itself. Normally you subclass it to provide a Filter
- * counterpart for a {@link MultiTermQuery} subclass.
- * <P>
  * This class also provides the functionality behind
- * {@link MultiTermQuery#CONSTANT_SCORE_REWRITE};
- * this is why it is not abstract.
+ * {@link MultiTermQuery#CONSTANT_SCORE_REWRITE}.
+ * It tries to rewrite per-segment as a boolean query
+ * that returns a constant score and otherwise fills a
+ * bit set with matches and builds a Scorer on top of
+ * this bit set.
  */
 final class MultiTermQueryConstantScoreWrapper<Q extends MultiTermQuery> extends Query {
 
+  // mtq that matches 16 terms or less will be executed as a regular disjunction
+  private static final int BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD = 16;
+
+  private static class TermAndState {
+    final BytesRef term;
+    final TermState state;
+    final int docFreq;
+    final long totalTermFreq;
+
+    TermAndState(BytesRef term, TermState state, int docFreq, long totalTermFreq) {
+      this.term = term;
+      this.state = state;
+      this.docFreq = docFreq;
+      this.totalTermFreq = totalTermFreq;
+    }
+  }
+
   protected final Q query;
 
   /**
@@ -57,28 +76,41 @@ final class MultiTermQueryConstantScoreW
   }
 
   @Override
-  @SuppressWarnings({"rawtypes"})
   public final boolean equals(final Object o) {
-    if (o==this) return true;
-    if (o==null) return false;
-    if (this.getClass().equals(o.getClass())) {
-      final MultiTermQueryConstantScoreWrapper that = (MultiTermQueryConstantScoreWrapper) o;
-      return this.query.equals(that.query) && this.getBoost() == that.getBoost();
+    if (super.equals(o) == false) {
+      return false;
     }
-    return false;
+    final MultiTermQueryConstantScoreWrapper<?> that = (MultiTermQueryConstantScoreWrapper<?>) o;
+    return this.query.equals(that.query) && this.getBoost() == that.getBoost();
   }
 
   @Override
   public final int hashCode() {
-    return Objects.hash(getClass(), query, getBoost());
+    return 31 * super.hashCode() + query.hashCode();
   }
 
   /** Returns the field name for this query */
   public final String getField() { return query.getField(); }
 
   @Override
-  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+  public Weight createWeight(final IndexSearcher searcher, final boolean needsScores) throws IOException {
     return new ConstantScoreWeight(this) {
+
+      /** Try to collect terms from the given terms enum and return true iff all
+       *  terms could be collected. If {@code false} is returned, the enum is
+       *  left positioned on the next term. */
+      private boolean collectTerms(LeafReaderContext context, TermsEnum termsEnum, List<TermAndState> terms) throws IOException {
+        final int threshold = Math.min(BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD, BooleanQuery.getMaxClauseCount());
+        for (int i = 0; i < threshold; ++i) {
+          final BytesRef term = termsEnum.next();
+          if (term == null) {
+            return true;
+          }
+          terms.add(new TermAndState(BytesRef.deepCopyOf(term), termsEnum.termState(), termsEnum.docFreq(), termsEnum.totalTermFreq()));
+        }
+        return termsEnum.next() == null;
+      }
+
       @Override
       public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
         final Terms terms = context.reader().terms(query.field);
@@ -92,10 +124,37 @@ final class MultiTermQueryConstantScoreW
 
         BitDocIdSet.Builder builder = new BitDocIdSet.Builder(context.reader().maxDoc());
         PostingsEnum docs = null;
-        while (termsEnum.next() != null) {
+
+        final List<TermAndState> collectedTerms = new ArrayList<>();
+        if (collectTerms(context, termsEnum, collectedTerms)) {
+          // build a boolean query
+          BooleanQuery bq = new BooleanQuery();
+          for (TermAndState t : collectedTerms) {
+            final TermContext termContext = new TermContext(searcher.getTopReaderContext());
+            termContext.register(t.state, context.ord, t.docFreq, t.totalTermFreq);
+            bq.add(new TermQuery(new Term(query.field, t.term), termContext), Occur.SHOULD);
+          }
+          Query q = new ConstantScoreQuery(bq);
+          q.setBoost(score());
+          return searcher.rewrite(q).createWeight(searcher, needsScores).scorer(context, acceptDocs);
+        }
+
+        // Too many terms: go back to the terms we already collected and start building the bit set
+        if (collectedTerms.isEmpty() == false) {
+          TermsEnum termsEnum2 = terms.iterator();
+          for (TermAndState t : collectedTerms) {
+            termsEnum2.seekExact(t.term, t.state);
+            docs = termsEnum2.postings(acceptDocs, docs, PostingsEnum.NONE);
+            builder.or(docs);
+          }
+        }
+
+        // Then keep filling the bit set with remaining terms
+        do {
           docs = termsEnum.postings(acceptDocs, docs, PostingsEnum.NONE);
           builder.or(docs);
-        }
+        } while (termsEnum.next() != null);
+
         final BitDocIdSet set = builder.build();
         if (set == null) {
           return null;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TermQuery.java?rev=1680474&r1=1680473&r2=1680474&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TermQuery.java Wed May 20 07:46:55 2015
@@ -56,9 +56,21 @@ public class TermQuery extends Query {
       assert termStates != null : "TermContext must not be null";
       this.termStates = termStates;
       this.similarity = searcher.getSimilarity();
-      this.stats = similarity.computeWeight(getBoost(),
-          searcher.collectionStatistics(term.field()),
-          searcher.termStatistics(term, termStates));
+      
+      final CollectionStatistics collectionStats;
+      final TermStatistics termStats;
+      if (needsScores) {
+        collectionStats = searcher.collectionStatistics(term.field());
+        termStats = searcher.termStatistics(term, termStates);
+      } else {
+        // do not bother computing actual stats, scores are not needed
+        final int maxDoc = searcher.getIndexReader().maxDoc();
+        final int docFreq = termStates.docFreq();
+        final long totalTermFreq = termStates.totalTermFreq();
+        collectionStats = new CollectionStatistics(term.field(), maxDoc, -1, -1, -1);
+        termStats = new TermStatistics(term.bytes(), docFreq, totalTermFreq);
+      }
+      this.stats = similarity.computeWeight(getBoost(), collectionStats, termStats);
     }
 
     @Override