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 2022/06/21 15:29:48 UTC

[lucene] branch main updated: Handle more cases in `BooleanWeight#count`. (#961)

This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/main by this push:
     new 68f77dbf53c Handle more cases in `BooleanWeight#count`. (#961)
68f77dbf53c is described below

commit 68f77dbf53c70190c99370c86d8372df76158025
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Tue Jun 21 17:29:43 2022 +0200

    Handle more cases in `BooleanWeight#count`. (#961)
    
    As suggested by @zhaih on #950, we could support more cases in
    `BooleanWeight#count`. This PR adds support for these cases specifically:
     - Pure disjunctions where only one clause has a non-zero count.
     - Pure disjunctions where one clause matches all docs.
     - Negations where positive clauses match all docs (pure negation).
     - Negations where positive clauses match no docs.
     - Negations where negative clauses match no docs.
     - Negations where negative clauses match all docs.
---
 lucene/CHANGES.txt                                 |   3 +
 .../org/apache/lucene/search/BooleanWeight.java    |  97 ++++++++---
 .../org/apache/lucene/search/TestBooleanQuery.java | 189 +++++++++++++++++++++
 3 files changed, 267 insertions(+), 22 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index d3406e159ff..12c9c784af6 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -93,6 +93,9 @@ Optimizations
 ---------------------
 * LUCENE-8519: MultiDocValues.getNormValues should not call getMergedFieldInfos (Rushabh Shah)
 
+* GITHUB#961: BooleanQuery can return quick counts for simple boolean queries.
+  (Adrien Grand)
+
 * LUCENE-10618: Implement BooleanQuery rewrite rules based for minimumShouldMatch. (Fang Hou)
 
 Bug Fixes
diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
index 23cd81675c8..42708d22cbc 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
@@ -346,29 +346,50 @@ final class BooleanWeight extends Weight {
 
   @Override
   public int count(LeafReaderContext context) throws IOException {
-    // Implement counting for pure conjunctions in the case when one clause doesn't match any docs,
-    // or all clauses but one match all docs.
-    if (weightedClauses.isEmpty()) {
-      return 0;
+    final int numDocs = context.reader().numDocs();
+    int positiveCount;
+    if (query.isPureDisjunction()) {
+      positiveCount = optCount(context, Occur.SHOULD);
+    } else if ((query.getClauses(Occur.FILTER).isEmpty() == false
+            || query.getClauses(Occur.MUST).isEmpty() == false)
+        && query.getMinimumNumberShouldMatch() == 0) {
+      positiveCount = reqCount(context);
+    } else {
+      // The query has a non-zero min-should match. We could handles some cases, e.g.
+      // minShouldMatch=N and we can find N SHOULD clauses that match all docs, but are there
+      // real-world queries that would benefit from Lucene handling this case?
+      positiveCount = -1;
     }
-    for (WeightedBooleanClause weightedClause : weightedClauses) {
-      switch (weightedClause.clause.getOccur()) {
-        case FILTER:
-        case MUST:
-          break;
-        case MUST_NOT:
-        case SHOULD:
-        default:
-          return super.count(context);
-      }
+
+    if (positiveCount == 0) {
+      return 0;
     }
-    if (query.getMinimumNumberShouldMatch() > 0) {
-      return super.count(context);
+
+    int prohibitedCount = optCount(context, Occur.MUST_NOT);
+    if (prohibitedCount == -1) {
+      return -1;
+    } else if (prohibitedCount == 0) {
+      return positiveCount;
+    } else if (prohibitedCount == numDocs) {
+      return 0;
+    } else if (positiveCount == numDocs) {
+      return numDocs - prohibitedCount;
+    } else {
+      return -1;
     }
-    // From now on we know the query is a pure conjunction
+  }
+
+  /**
+   * Return the number of matches of required clauses, or -1 if unknown, or numDocs if there are no
+   * required clauses.
+   */
+  private int reqCount(LeafReaderContext context) throws IOException {
     final int numDocs = context.reader().numDocs();
-    int conjunctionCount = numDocs;
+    int reqCount = numDocs;
     for (WeightedBooleanClause weightedClause : weightedClauses) {
+      if (weightedClause.clause.isRequired() == false) {
+        continue;
+      }
       int count = weightedClause.weight.count(context);
       if (count == -1 || count == 0) {
         // If the count of one clause is unknown, then the count of the conjunction is unknown too.
@@ -376,17 +397,49 @@ final class BooleanWeight extends Weight {
         return count;
       } else if (count == numDocs) {
         // the query matches all docs, it can be safely ignored
-      } else if (conjunctionCount == numDocs) {
+      } else if (reqCount == numDocs) {
         // all clauses seen so far match all docs, so the count of the new clause is also the count
         // of the conjunction
-        conjunctionCount = count;
+        reqCount = count;
       } else {
         // We have two clauses whose count is in [1, numDocs), we can't figure out the number of
         // docs that match the conjunction without running the query.
-        return super.count(context);
+        return -1;
+      }
+    }
+    return reqCount;
+  }
+
+  /**
+   * Return the number of matches of optional clauses, or -1 if unknown, or 0 if there are no
+   * optional clauses.
+   */
+  private int optCount(LeafReaderContext context, Occur occur) throws IOException {
+    final int numDocs = context.reader().numDocs();
+    int optCount = 0;
+    for (WeightedBooleanClause weightedClause : weightedClauses) {
+      if (weightedClause.clause.getOccur() != occur) {
+        continue;
+      }
+      int count = weightedClause.weight.count(context);
+      if (count == -1 || count == numDocs) {
+        // If any of the clauses has a number of matches that is unknown, the number of matches of
+        // the disjunction is unknown.
+        // If either clause matches all docs, then the disjunction matches all docs.
+        return count;
+      } else if (count == 0) {
+        // We can safely ignore this clause, it doesn't affect the count.
+      } else if (optCount == 0) {
+        // This is the first clause we see that has a non-zero count, it becomes the count of the
+        // disjunction.
+        optCount = count;
+      } else {
+        // We have two clauses whose count is in [1, numDocs), we can't figure out the number of
+        // docs that match the disjunction without running the query.
+        return -1;
       }
     }
-    return conjunctionCount;
+    return optCount;
   }
 
   @Override
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java
index 5de0f67a999..6a391a38274 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java
@@ -839,6 +839,195 @@ public class TestBooleanQuery extends LuceneTestCase {
     dir.close();
   }
 
+  public void testDisjunctionMatchesCount() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
+    Document doc = new Document();
+    LongPoint longPoint = new LongPoint("long", 3L);
+    doc.add(longPoint);
+    StringField stringField = new StringField("string", "abc", Store.NO);
+    doc.add(stringField);
+    writer.addDocument(doc);
+    longPoint.setLongValue(10);
+    stringField.setStringValue("xyz");
+    writer.addDocument(doc);
+    IndexReader reader = DirectoryReader.open(writer);
+    writer.close();
+    IndexSearcher searcher = new IndexSearcher(reader);
+
+    Query query =
+        new BooleanQuery.Builder()
+            .add(new TermQuery(new Term("string", "abc")), Occur.SHOULD)
+            .add(LongPoint.newExactQuery("long", 3L), Occur.SHOULD)
+            .build();
+    Weight weight = searcher.createWeight(query, ScoreMode.COMPLETE, 1f);
+    // Both queries match a single doc, BooleanWeight can't figure out the count of the disjunction
+    assertEquals(-1, weight.count(reader.leaves().get(0)));
+
+    query =
+        new BooleanQuery.Builder()
+            .add(new TermQuery(new Term("string", "missing")), Occur.SHOULD)
+            .add(LongPoint.newExactQuery("long", 3L), Occur.SHOULD)
+            .build();
+    weight = searcher.createWeight(query, ScoreMode.COMPLETE, 1f);
+    // One query has a count of 0, the disjunction count is the other count
+    assertEquals(1, weight.count(reader.leaves().get(0)));
+
+    query =
+        new BooleanQuery.Builder()
+            .add(new TermQuery(new Term("string", "abc")), Occur.SHOULD)
+            .add(LongPoint.newExactQuery("long", 5L), Occur.SHOULD)
+            .build();
+    weight = searcher.createWeight(query, ScoreMode.COMPLETE, 1f);
+    // One query has a count of 0, the disjunction count is the other count
+    assertEquals(1, weight.count(reader.leaves().get(0)));
+
+    query =
+        new BooleanQuery.Builder()
+            .add(new TermQuery(new Term("string", "abc")), Occur.SHOULD)
+            .add(LongPoint.newRangeQuery("long", 0L, 10L), Occur.SHOULD)
+            .build();
+    // One query matches all docs, the count of the disjunction is the number of docs
+    weight = searcher.createWeight(query, ScoreMode.COMPLETE, 1f);
+    assertEquals(2, weight.count(reader.leaves().get(0)));
+
+    query =
+        new BooleanQuery.Builder()
+            .add(new MatchAllDocsQuery(), Occur.SHOULD)
+            .add(LongPoint.newRangeQuery("long", 1L, 5L), Occur.SHOULD)
+            .build();
+    // One query matches all docs, the count of the disjunction is the number of docs
+    weight = searcher.createWeight(query, ScoreMode.COMPLETE, 1f);
+    assertEquals(2, weight.count(reader.leaves().get(0)));
+
+    reader.close();
+    dir.close();
+  }
+
+  public void testProhibitedMatchesCount() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
+    Document doc = new Document();
+    LongPoint longPoint = new LongPoint("long", 3L);
+    doc.add(longPoint);
+    StringField stringField = new StringField("string", "abc", Store.NO);
+    doc.add(stringField);
+    writer.addDocument(doc);
+    longPoint.setLongValue(10);
+    stringField.setStringValue("xyz");
+    writer.addDocument(doc);
+    IndexReader reader = DirectoryReader.open(writer);
+    writer.close();
+    IndexSearcher searcher = new IndexSearcher(reader);
+
+    Query query =
+        new BooleanQuery.Builder()
+            .add(new TermQuery(new Term("string", "abc")), Occur.MUST)
+            .add(LongPoint.newExactQuery("long", 3L), Occur.MUST_NOT)
+            .build();
+    Weight weight = searcher.createWeight(query, ScoreMode.COMPLETE, 1f);
+    // Both queries match a single doc, BooleanWeight can't figure out the count of the query
+    assertEquals(-1, weight.count(reader.leaves().get(0)));
+
+    query =
+        new BooleanQuery.Builder()
+            .add(new TermQuery(new Term("string", "missing")), Occur.MUST)
+            .add(LongPoint.newExactQuery("long", 3L), Occur.MUST_NOT)
+            .build();
+    weight = searcher.createWeight(query, ScoreMode.COMPLETE, 1f);
+    // the positive clause doesn't match any docs, so the overall query doesn't either
+    assertEquals(0, weight.count(reader.leaves().get(0)));
+
+    query =
+        new BooleanQuery.Builder()
+            .add(new TermQuery(new Term("string", "abc")), Occur.MUST)
+            .add(LongPoint.newExactQuery("long", 5L), Occur.MUST_NOT)
+            .build();
+    weight = searcher.createWeight(query, ScoreMode.COMPLETE, 1f);
+    // the negative clause doesn't match any docs, so the overall count is the count of the positive
+    // clause
+    assertEquals(1, weight.count(reader.leaves().get(0)));
+
+    query =
+        new BooleanQuery.Builder()
+            .add(new TermQuery(new Term("string", "abc")), Occur.MUST)
+            .add(LongPoint.newRangeQuery("long", 0L, 10L), Occur.MUST_NOT)
+            .build();
+    // the negative clause matches all docs, so the query doesn't match any docs
+    weight = searcher.createWeight(query, ScoreMode.COMPLETE, 1f);
+    assertEquals(0, weight.count(reader.leaves().get(0)));
+
+    query =
+        new BooleanQuery.Builder()
+            .add(LongPoint.newRangeQuery("long", 0L, 10L), Occur.MUST)
+            .add(new TermQuery(new Term("string", "abc")), Occur.MUST_NOT)
+            .build();
+    // The positive clause matches all docs, so we can subtract the number of matches of the
+    // negative clause
+    weight = searcher.createWeight(query, ScoreMode.COMPLETE, 1f);
+    assertEquals(1, weight.count(reader.leaves().get(0)));
+
+    reader.close();
+    dir.close();
+  }
+
+  public void testRandomBooleanQueryMatchesCount() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig());
+    Document doc = new Document();
+    LongPoint longPoint = new LongPoint("long", 3L);
+    doc.add(longPoint);
+    StringField stringField = new StringField("string", "abc", Store.NO);
+    doc.add(stringField);
+    writer.addDocument(doc);
+    longPoint.setLongValue(10);
+    stringField.setStringValue("xyz");
+    writer.addDocument(doc);
+    IndexReader reader = DirectoryReader.open(writer);
+    writer.close();
+    IndexSearcher searcher = new IndexSearcher(reader);
+    for (int iter = 0; iter < 1000; ++iter) {
+      final int numClauses = TestUtil.nextInt(random(), 2, 5);
+      BooleanQuery.Builder builder = new BooleanQuery.Builder();
+      int numShouldClauses = 0;
+      for (int i = 0; i < numClauses; ++i) {
+        Query query;
+        switch (random().nextInt(6)) {
+          case 0:
+            query = new TermQuery(new Term("string", "abc"));
+            break;
+          case 1:
+            query = LongPoint.newExactQuery("long", 3L);
+            break;
+          case 2:
+            query = new TermQuery(new Term("string", "missing"));
+            break;
+          case 3:
+            query = LongPoint.newExactQuery("long", 5L);
+            break;
+          case 4:
+            query = new MatchAllDocsQuery();
+            break;
+          default:
+            query = LongPoint.newRangeQuery("long", 0L, 10L);
+            break;
+        }
+        Occur occur = RandomPicks.randomFrom(random(), Occur.values());
+        if (occur == Occur.SHOULD) {
+          numShouldClauses++;
+        }
+        builder.add(query, occur);
+      }
+      builder.setMinimumNumberShouldMatch(TestUtil.nextInt(random(), 0, numShouldClauses));
+      Query booleanQuery = builder.build();
+      assertEquals(
+          (int) searcher.search(booleanQuery, new TotalHitCountCollectorManager()),
+          searcher.count(booleanQuery));
+    }
+    reader.close();
+    dir.close();
+  }
+
   public void testToString() {
     BooleanQuery.Builder bq = new BooleanQuery.Builder();
     bq.add(new TermQuery(new Term("field", "a")), Occur.SHOULD);