fix: boolean query incorrectly dropping documents when AllScorer is present (#2760)

* Fixed the range issue.

* Fixed the second all scorer issue

* Improved docs + tests

* Improved code.

* Fixed lint issues.

* Improved tests + logic based on PR comments.

* Fixed lint issues.

* Increase the document count.

* Improved the prop-tests

* Expand the index size, and remove unused parameter.

---------

Co-authored-by: Stu Hood <stuhood@gmail.com>
This commit is contained in:
Moe
2025-12-16 13:52:02 -08:00
committed by GitHub
parent ba61ed6ef3
commit e3c9be1f92
2 changed files with 547 additions and 41 deletions

View File

@@ -97,6 +97,65 @@ fn into_box_scorer<TScoreCombiner: ScoreCombiner>(
}
}
/// Returns the effective MUST scorer, accounting for removed AllScorers.
///
/// When AllScorer instances are removed from must_scorers as an optimization,
/// we must restore the "match all" semantics if the list becomes empty.
fn effective_must_scorer(
must_scorers: Vec<Box<dyn Scorer>>,
removed_all_scorer_count: usize,
max_doc: DocId,
num_docs: u32,
) -> Option<Box<dyn Scorer>> {
if must_scorers.is_empty() {
if removed_all_scorer_count > 0 {
// Had AllScorer(s) only - all docs match
Some(Box::new(AllScorer::new(max_doc)))
} else {
// No MUST constraint at all
None
}
} else {
Some(intersect_scorers(must_scorers, num_docs))
}
}
/// Returns a SHOULD scorer with AllScorer union if any were removed.
///
/// For union semantics (OR): if any SHOULD clause was an AllScorer, the result
/// should include all documents. We restore this by unioning with AllScorer.
///
/// When `scoring_enabled` is false, we can just return AllScorer alone since
/// we don't need score contributions from the should_scorer.
fn effective_should_scorer_for_union<TScoreCombiner: ScoreCombiner>(
should_scorer: SpecializedScorer,
removed_all_scorer_count: usize,
max_doc: DocId,
num_docs: u32,
score_combiner_fn: impl Fn() -> TScoreCombiner,
scoring_enabled: bool,
) -> SpecializedScorer {
if removed_all_scorer_count > 0 {
if scoring_enabled {
// Need to union to get score contributions from both
let all_scorers: Vec<Box<dyn Scorer>> = vec![
into_box_scorer(should_scorer, &score_combiner_fn, num_docs),
Box::new(AllScorer::new(max_doc)),
];
SpecializedScorer::Other(Box::new(BufferedUnionScorer::build(
all_scorers,
score_combiner_fn,
num_docs,
)))
} else {
// Scoring disabled - AllScorer alone is sufficient
SpecializedScorer::Other(Box::new(AllScorer::new(max_doc)))
}
} else {
should_scorer
}
}
enum ShouldScorersCombinationMethod {
// Should scorers are irrelevant.
Ignored,
@@ -246,53 +305,78 @@ impl<TScoreCombiner: ScoreCombiner> BooleanWeight<TScoreCombiner> {
let include_scorer = match (should_scorers, must_scorers) {
(ShouldScorersCombinationMethod::Ignored, must_scorers) => {
let boxed_scorer: Box<dyn Scorer> = if must_scorers.is_empty() {
// We do not have any should scorers, nor all scorers.
// There are still two cases here.
//
// If this follows the removal of some AllScorers in the should/must clauses,
// then we match all documents.
//
// Otherwise, it is really just an EmptyScorer.
if must_special_scorer_counts.num_all_scorers
+ should_special_scorer_counts.num_all_scorers
> 0
{
Box::new(AllScorer::new(reader.max_doc()))
} else {
Box::new(EmptyScorer)
}
} else {
intersect_scorers(must_scorers, num_docs)
};
// No SHOULD clauses (or they were absorbed into MUST).
// Result depends entirely on MUST + any removed AllScorers.
let combined_all_scorer_count = must_special_scorer_counts.num_all_scorers
+ should_special_scorer_counts.num_all_scorers;
let boxed_scorer: Box<dyn Scorer> = effective_must_scorer(
must_scorers,
combined_all_scorer_count,
reader.max_doc(),
num_docs,
)
.unwrap_or_else(|| Box::new(EmptyScorer));
SpecializedScorer::Other(boxed_scorer)
}
(ShouldScorersCombinationMethod::Optional(should_scorer), must_scorers) => {
if must_scorers.is_empty() && must_special_scorer_counts.num_all_scorers == 0 {
// Optional options are promoted to required if no must scorers exists.
should_scorer
} else {
let must_scorer = intersect_scorers(must_scorers, num_docs);
if self.scoring_enabled {
SpecializedScorer::Other(Box::new(RequiredOptionalScorer::<
_,
_,
TScoreCombiner,
>::new(
must_scorer,
into_box_scorer(should_scorer, &score_combiner_fn, num_docs),
)))
} else {
SpecializedScorer::Other(must_scorer)
// Optional SHOULD: contributes to scoring but not required for matching.
match effective_must_scorer(
must_scorers,
must_special_scorer_counts.num_all_scorers,
reader.max_doc(),
num_docs,
) {
None => {
// No MUST constraint: promote SHOULD to required.
// Must preserve any removed AllScorers from SHOULD via union.
effective_should_scorer_for_union(
should_scorer,
should_special_scorer_counts.num_all_scorers,
reader.max_doc(),
num_docs,
&score_combiner_fn,
self.scoring_enabled,
)
}
Some(must_scorer) => {
// Has MUST constraint: SHOULD only affects scoring.
if self.scoring_enabled {
SpecializedScorer::Other(Box::new(RequiredOptionalScorer::<
_,
_,
TScoreCombiner,
>::new(
must_scorer,
into_box_scorer(should_scorer, &score_combiner_fn, num_docs),
)))
} else {
SpecializedScorer::Other(must_scorer)
}
}
}
}
(ShouldScorersCombinationMethod::Required(should_scorer), mut must_scorers) => {
if must_scorers.is_empty() {
should_scorer
} else {
must_scorers.push(into_box_scorer(should_scorer, &score_combiner_fn, num_docs));
SpecializedScorer::Other(intersect_scorers(must_scorers, num_docs))
(ShouldScorersCombinationMethod::Required(should_scorer), must_scorers) => {
// Required SHOULD: at least `minimum_number_should_match` must match.
// Semantics: (MUST constraint) AND (SHOULD constraint)
match effective_must_scorer(
must_scorers,
must_special_scorer_counts.num_all_scorers,
reader.max_doc(),
num_docs,
) {
None => {
// No MUST constraint: SHOULD alone determines matching.
should_scorer
}
Some(must_scorer) => {
// Has MUST constraint: intersect MUST with SHOULD.
let should_boxed =
into_box_scorer(should_scorer, &score_combiner_fn, num_docs);
SpecializedScorer::Other(intersect_scorers(
vec![must_scorer, should_boxed],
num_docs,
))
}
}
}
};

View File

@@ -417,4 +417,426 @@ mod tests {
Ok(())
}
// =========================================================================
// AllScorer Preservation Regression Tests
// =========================================================================
//
// These tests verify the fix for a bug where AllScorer instances (produced by
// queries matching all documents, such as range queries covering all values)
// were incorrectly removed from Boolean query processing, causing documents
// to be unexpectedly excluded from results.
//
// The bug manifested in several scenarios:
// 1. SHOULD + SHOULD where one clause is AllScorer
// 2. MUST (AllScorer) + SHOULD
// 3. Range queries in Boolean clauses when all documents match the range
/// Regression test: SHOULD clause with AllScorer combined with other SHOULD clauses.
///
/// When a SHOULD clause produces an AllScorer (e.g., from a range query matching
/// all documents), the Boolean query should still match all documents.
///
/// Bug before fix: AllScorer was removed during optimization, leaving only the
/// other SHOULD clauses, which incorrectly excluded documents.
#[test]
pub fn test_should_with_all_scorer_regression() -> crate::Result<()> {
let mut schema_builder = Schema::builder();
let text_field = schema_builder.add_text_field("text", TEXT);
let num_field =
schema_builder.add_i64_field("num", NumericOptions::default().set_fast().set_indexed());
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
let mut index_writer: IndexWriter = index.writer_for_tests()?;
// All docs have num > 0, so range query will return AllScorer
index_writer.add_document(doc!(text_field => "hello", num_field => 10i64))?;
index_writer.add_document(doc!(text_field => "world", num_field => 20i64))?;
index_writer.add_document(doc!(text_field => "hello world", num_field => 30i64))?;
index_writer.add_document(doc!(text_field => "foo", num_field => 40i64))?;
index_writer.add_document(doc!(text_field => "bar", num_field => 50i64))?;
index_writer.add_document(doc!(text_field => "baz", num_field => 60i64))?;
index_writer.commit()?;
let searcher = index.reader()?.searcher();
// Range query matching all docs (returns AllScorer)
let all_match_query: Box<dyn Query> = Box::new(RangeQuery::new(
Bound::Excluded(Term::from_field_i64(num_field, 0)),
Bound::Unbounded,
));
let term_query: Box<dyn Query> = Box::new(TermQuery::new(
Term::from_field_text(text_field, "hello"),
IndexRecordOption::Basic,
));
// Verify range matches all 6 docs
assert_eq!(searcher.search(all_match_query.as_ref(), &Count)?, 6);
// RangeQuery(all) OR TermQuery should match all 6 docs
let bool_query = BooleanQuery::new(vec![
(Occur::Should, all_match_query.box_clone()),
(Occur::Should, term_query.box_clone()),
]);
let count = searcher.search(&bool_query, &Count)?;
assert_eq!(count, 6, "SHOULD with AllScorer should match all docs");
// Order should not matter
let bool_query_reversed = BooleanQuery::new(vec![
(Occur::Should, term_query.box_clone()),
(Occur::Should, all_match_query.box_clone()),
]);
let count_reversed = searcher.search(&bool_query_reversed, &Count)?;
assert_eq!(
count_reversed, 6,
"Order of SHOULD clauses should not matter"
);
Ok(())
}
/// Regression test: MUST clause with AllScorer combined with SHOULD clause.
///
/// When MUST contains an AllScorer, all documents satisfy the MUST constraint.
/// The SHOULD clause should only affect scoring, not filtering.
///
/// Bug before fix: AllScorer was removed, leaving an empty must_scorers vector.
/// intersect_scorers([]) incorrectly returned EmptyScorer, matching 0 documents.
#[test]
pub fn test_must_all_with_should_regression() -> crate::Result<()> {
let mut schema_builder = Schema::builder();
let text_field = schema_builder.add_text_field("text", TEXT);
let num_field =
schema_builder.add_i64_field("num", NumericOptions::default().set_fast().set_indexed());
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
let mut index_writer: IndexWriter = index.writer_for_tests()?;
// All docs have num > 0, so range query will return AllScorer
index_writer.add_document(doc!(text_field => "apple", num_field => 10i64))?;
index_writer.add_document(doc!(text_field => "banana", num_field => 20i64))?;
index_writer.add_document(doc!(text_field => "cherry", num_field => 30i64))?;
index_writer.add_document(doc!(text_field => "date", num_field => 40i64))?;
index_writer.commit()?;
let searcher = index.reader()?.searcher();
// Range query matching all docs (returns AllScorer)
let all_match_query: Box<dyn Query> = Box::new(RangeQuery::new(
Bound::Excluded(Term::from_field_i64(num_field, 0)),
Bound::Unbounded,
));
let term_query: Box<dyn Query> = Box::new(TermQuery::new(
Term::from_field_text(text_field, "apple"),
IndexRecordOption::Basic,
));
// Verify range matches all 4 docs
assert_eq!(searcher.search(all_match_query.as_ref(), &Count)?, 4);
// MUST(range matching all) AND SHOULD(term) should match all 4 docs
let bool_query = BooleanQuery::new(vec![
(Occur::Must, all_match_query.box_clone()),
(Occur::Should, term_query.box_clone()),
]);
let count = searcher.search(&bool_query, &Count)?;
assert_eq!(count, 4, "MUST AllScorer + SHOULD should match all docs");
Ok(())
}
/// Regression test: Range queries in Boolean clauses when all documents match.
///
/// Range queries can return AllScorer as an optimization when all indexed values
/// fall within the range. This test ensures such queries work correctly in
/// Boolean combinations.
///
/// This is the most common real-world manifestation of the bug, occurring in
/// queries like: (age > 50 OR name = 'Alice') AND status = 'active'
/// when all documents have age > 50.
#[test]
pub fn test_range_query_all_match_in_boolean() -> crate::Result<()> {
let mut schema_builder = Schema::builder();
let name_field = schema_builder.add_text_field("name", TEXT);
let age_field =
schema_builder.add_i64_field("age", NumericOptions::default().set_fast().set_indexed());
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
let mut index_writer: IndexWriter = index.writer_for_tests()?;
// All documents have age > 50, so range query will return AllScorer
index_writer.add_document(doc!(name_field => "alice", age_field => 55_i64))?;
index_writer.add_document(doc!(name_field => "bob", age_field => 60_i64))?;
index_writer.add_document(doc!(name_field => "charlie", age_field => 70_i64))?;
index_writer.add_document(doc!(name_field => "diana", age_field => 80_i64))?;
index_writer.commit()?;
let searcher = index.reader()?.searcher();
let range_query: Box<dyn Query> = Box::new(RangeQuery::new(
Bound::Excluded(Term::from_field_i64(age_field, 50)),
Bound::Unbounded,
));
let term_query: Box<dyn Query> = Box::new(TermQuery::new(
Term::from_field_text(name_field, "alice"),
IndexRecordOption::Basic,
));
// Verify preconditions
assert_eq!(searcher.search(range_query.as_ref(), &Count)?, 4);
assert_eq!(searcher.search(term_query.as_ref(), &Count)?, 1);
// SHOULD(range) OR SHOULD(term): range matches all, so result is 4
let should_query = BooleanQuery::new(vec![
(Occur::Should, range_query.box_clone()),
(Occur::Should, term_query.box_clone()),
]);
assert_eq!(
searcher.search(&should_query, &Count)?,
4,
"SHOULD range OR term should match all"
);
// MUST(range) AND SHOULD(term): range matches all, term is optional
let must_should_query = BooleanQuery::new(vec![
(Occur::Must, range_query.box_clone()),
(Occur::Should, term_query.box_clone()),
]);
assert_eq!(
searcher.search(&must_should_query, &Count)?,
4,
"MUST range + SHOULD term should match all"
);
Ok(())
}
/// Test multiple AllScorer instances in different clause types.
///
/// Verifies correct behavior when AllScorers appear in multiple positions.
#[test]
pub fn test_multiple_all_scorers() -> crate::Result<()> {
let mut schema_builder = Schema::builder();
let text_field = schema_builder.add_text_field("text", TEXT);
let num_field =
schema_builder.add_i64_field("num", NumericOptions::default().set_fast().set_indexed());
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
let mut index_writer: IndexWriter = index.writer_for_tests()?;
// All docs have num > 0, so range queries will return AllScorer
index_writer.add_document(doc!(text_field => "doc1", num_field => 10i64))?;
index_writer.add_document(doc!(text_field => "doc2", num_field => 20i64))?;
index_writer.add_document(doc!(text_field => "doc3", num_field => 30i64))?;
index_writer.commit()?;
let searcher = index.reader()?.searcher();
// Two different range queries that both match all docs (return AllScorer)
let all_query1: Box<dyn Query> = Box::new(RangeQuery::new(
Bound::Excluded(Term::from_field_i64(num_field, 0)),
Bound::Unbounded,
));
let all_query2: Box<dyn Query> = Box::new(RangeQuery::new(
Bound::Excluded(Term::from_field_i64(num_field, 5)),
Bound::Unbounded,
));
let term_query: Box<dyn Query> = Box::new(TermQuery::new(
Term::from_field_text(text_field, "doc1"),
IndexRecordOption::Basic,
));
// Multiple AllScorers in SHOULD
let multi_all_should = BooleanQuery::new(vec![
(Occur::Should, all_query1.box_clone()),
(Occur::Should, all_query2.box_clone()),
(Occur::Should, term_query.box_clone()),
]);
assert_eq!(
searcher.search(&multi_all_should, &Count)?,
3,
"Multiple AllScorers in SHOULD"
);
// AllScorer in both MUST and SHOULD
let all_must_and_should = BooleanQuery::new(vec![
(Occur::Must, all_query1.box_clone()),
(Occur::Should, all_query2.box_clone()),
]);
assert_eq!(
searcher.search(&all_must_and_should, &Count)?,
3,
"AllScorer in both MUST and SHOULD"
);
Ok(())
}
}
/// A proptest which generates arbitrary permutations of a simple boolean AST, and then matches
/// the result against an index which contains all permutations of documents with N fields.
#[cfg(test)]
mod proptest_boolean_query {
use std::collections::{BTreeMap, HashSet};
use std::ops::Bound;
use proptest::collection::vec;
use proptest::prelude::*;
use crate::collector::DocSetCollector;
use crate::query::{AllQuery, BooleanQuery, Occur, Query, RangeQuery, TermQuery};
use crate::schema::{Field, NumericOptions, OwnedValue, Schema, TEXT};
use crate::{DocId, Index, Term};
#[derive(Debug, Clone)]
enum BooleanQueryAST {
/// Matches all documents via AllQuery (wraps AllScorer in BoostScorer)
All,
/// Matches all documents via RangeQuery (returns bare AllScorer)
/// This is the actual trigger for the AllScorer preservation bug
RangeAll,
/// Matches documents where the field has value "true"
Leaf {
field_idx: usize,
},
Union(Vec<BooleanQueryAST>),
Intersection(Vec<BooleanQueryAST>),
}
impl BooleanQueryAST {
fn matches(&self, doc_id: DocId) -> bool {
match self {
BooleanQueryAST::All => true,
BooleanQueryAST::RangeAll => true,
BooleanQueryAST::Leaf { field_idx } => Self::matches_field(doc_id, *field_idx),
BooleanQueryAST::Union(children) => {
children.iter().any(|child| child.matches(doc_id))
}
BooleanQueryAST::Intersection(children) => {
children.iter().all(|child| child.matches(doc_id))
}
}
}
fn matches_field(doc_id: DocId, field_idx: usize) -> bool {
((doc_id as usize) >> field_idx) & 1 == 1
}
fn to_query(&self, fields: &[Field], range_field: Field) -> Box<dyn Query> {
match self {
BooleanQueryAST::All => Box::new(AllQuery),
BooleanQueryAST::RangeAll => {
// Range query that matches all docs (all have value >= 0)
// This returns bare AllScorer, triggering the bug we fixed
Box::new(RangeQuery::new(
Bound::Included(Term::from_field_i64(range_field, 0)),
Bound::Unbounded,
))
}
BooleanQueryAST::Leaf { field_idx } => Box::new(TermQuery::new(
Term::from_field_text(fields[*field_idx], "true"),
crate::schema::IndexRecordOption::Basic,
)),
BooleanQueryAST::Union(children) => {
let sub_queries = children
.iter()
.map(|child| (Occur::Should, child.to_query(fields, range_field)))
.collect();
Box::new(BooleanQuery::new(sub_queries))
}
BooleanQueryAST::Intersection(children) => {
let sub_queries = children
.iter()
.map(|child| (Occur::Must, child.to_query(fields, range_field)))
.collect();
Box::new(BooleanQuery::new(sub_queries))
}
}
}
}
fn doc_ids(num_docs: usize, num_fields: usize) -> impl Iterator<Item = DocId> {
let permutations = 1 << num_fields;
let copies = (num_docs as f32 / permutations as f32).ceil() as u32;
(0..(permutations * copies)).into_iter()
}
fn create_index_with_boolean_permutations(
num_docs: usize,
num_fields: usize,
) -> (Index, Vec<Field>, Field) {
let mut schema_builder = Schema::builder();
let fields: Vec<Field> = (0..num_fields)
.map(|i| schema_builder.add_text_field(&format!("field_{}", i), TEXT))
.collect();
// Add a numeric field for RangeQuery tests - all docs have value = doc_id
let range_field = schema_builder.add_i64_field(
"range_field",
NumericOptions::default().set_fast().set_indexed(),
);
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
let mut writer = index.writer_for_tests().unwrap();
for doc_id in doc_ids(num_docs, num_fields) {
let mut doc: BTreeMap<_, OwnedValue> = BTreeMap::default();
for (field_idx, &field) in fields.iter().enumerate() {
if (doc_id >> field_idx) & 1 == 1 {
doc.insert(field, "true".into());
}
}
// All docs have non-negative values, so RangeQuery(>=0) matches all
doc.insert(range_field, (doc_id as i64).into());
writer.add_document(doc).unwrap();
}
writer.commit().unwrap();
(index, fields, range_field)
}
fn arb_boolean_query_ast(num_fields: usize) -> impl Strategy<Value = BooleanQueryAST> {
// Leaf strategies: term queries, AllQuery, and RangeQuery matching all docs
let leaf = prop_oneof![
(0..num_fields).prop_map(|field_idx| BooleanQueryAST::Leaf { field_idx }),
Just(BooleanQueryAST::All),
Just(BooleanQueryAST::RangeAll),
];
leaf.prop_recursive(
8, // 8 levels of recursion
256, // 256 nodes max
10, // 10 items per collection
|inner| {
prop_oneof![
vec(inner.clone(), 1..10).prop_map(BooleanQueryAST::Union),
vec(inner, 1..10).prop_map(BooleanQueryAST::Intersection),
]
},
)
}
#[test]
fn proptest_boolean_query() {
// In the presence of optimizations around buffering, it can take large numbers of
// documents to uncover some issues.
let num_docs = 10000;
let num_fields = 8;
let num_docs = 1 << num_fields;
let (index, fields, range_field) =
create_index_with_boolean_permutations(num_docs, num_fields);
let searcher = index.reader().unwrap().searcher();
proptest!(|(ast in arb_boolean_query_ast(num_fields))| {
let query = ast.to_query(&fields, range_field);
let mut matching_docs = HashSet::new();
for doc_id in doc_ids(num_docs, num_fields) {
if ast.matches(doc_id as DocId) {
matching_docs.insert(doc_id as DocId);
}
}
let doc_addresses = searcher.search(&*query, &DocSetCollector).unwrap();
let result_docs: HashSet<DocId> =
doc_addresses.into_iter().map(|doc_address| doc_address.doc_id).collect();
prop_assert_eq!(result_docs, matching_docs);
});
}
}