Compare commits

...

11 Commits

Author SHA1 Message Date
Pascal Seitz
078f77e8ec inverted index reader as trait
convert inverted index reader as trait
fix performance regression: fix incorrect scorer cast for buffered union
bock wand
2026-01-29 17:18:03 +01:00
Pascal Seitz
697e1e94c2 add merge_segments benchmark 2026-01-29 17:17:52 +01:00
Pascal Seitz
6d2a0929ca fix merge conflicts 2026-01-28 15:23:32 +01:00
Pascal Seitz
3517714100 remove serialization from codec 2026-01-28 15:18:16 +01:00
Paul Masurel
347eafce0c First stab at tantivy's codec
For the moment, this only allows for postings codec.
Also, on the write side, it does not include positions yet.

Implementation details:
On the write side, we use static typing.

A lot of types are now generics over the codec, but with a default codec type
that makes it so, we should not break client projects too much.

On the read side, we rely on a ObjectSafeCodec contraption to avoid
the proliferation of generics.

That object's point is to make sure we can build TermScorer with a concrete
codec specific type before reboxing it. (same thing for PhraseScorer).
2026-01-28 15:17:41 +01:00
trinity-1686a
9b619998bd Merge pull request #2816 from evance-br/fix-closing-paren-elastic-range 2026-01-27 17:00:08 +01:00
Evance Soumaoro
765c448945 uncomment commented code when testing 2026-01-27 13:19:41 +00:00
Evance Soumaoro
943594ebaa uncomment commented code when testing 2026-01-27 13:08:38 +00:00
Evance Soumaoro
df17daae0d fix closing parenthesis error on elastic range queries for lenient parser 2026-01-27 13:01:14 +00:00
Paul Masurel
0ae94baef5 Remove temp file (#2815)
Co-authored-by: Paul Masurel <paul.masurel@datadoghq.com>
2026-01-27 09:22:11 +01:00
Paul Masurel
3f448ecf79 Bugfix on intersection. (#2812)
The intersection algorithm made it possible for .seek(..) with values
lower than the current doc id, breaking the DocSet contract.

The fix removes the optimization that caused left.seek(..) to be replaced
by a simpler left.advance(..).

Simply doing so lead to a performance regression.
I therefore integrated that idea within SegmentPostings.seek.

We now attempt to check the next doc systematically on seek,
PROVIDED the block is already loaded.

Closes #2811

Co-authored-by: Paul Masurel <paul.masurel@datadoghq.com>
2026-01-27 09:21:09 +01:00
71 changed files with 2614 additions and 1442 deletions

View File

@@ -193,3 +193,9 @@ harness = false
[[bench]]
name = "str_search_and_get"
harness = false
[[bench]]
name = "merge_segments"
harness = false

224
benches/merge_segments.rs Normal file
View File

@@ -0,0 +1,224 @@
// Benchmarks segment merging
//
// Notes:
// - Input segments are kept intact (no deletes / no IndexWriter merge).
// - Output is written to a `NullDirectory` that discards all files except
// fieldnorms (needed for merging).
use std::collections::HashMap;
use std::io::{self, Write};
use std::path::{Path, PathBuf};
use std::sync::{Arc, RwLock};
use binggan::{black_box, BenchRunner};
use rand::prelude::*;
use rand::rngs::StdRng;
use rand::SeedableRng;
use tantivy::directory::error::{DeleteError, OpenReadError, OpenWriteError};
use tantivy::directory::{
AntiCallToken, Directory, FileHandle, OwnedBytes, TerminatingWrite, WatchCallback, WatchHandle,
WritePtr,
};
use tantivy::indexer::{merge_filtered_segments, NoMergePolicy};
use tantivy::schema::{Schema, TEXT};
use tantivy::{doc, HasLen, Index, IndexSettings, Segment};
#[derive(Clone, Default, Debug)]
struct NullDirectory {
blobs: Arc<RwLock<HashMap<PathBuf, OwnedBytes>>>,
}
struct NullWriter;
impl Write for NullWriter {
fn write(&mut self, buf: &[u8]) -> io::Result<usize> {
Ok(buf.len())
}
fn flush(&mut self) -> io::Result<()> {
Ok(())
}
}
impl TerminatingWrite for NullWriter {
fn terminate_ref(&mut self, _token: AntiCallToken) -> io::Result<()> {
Ok(())
}
}
struct InMemoryWriter {
path: PathBuf,
buffer: Vec<u8>,
blobs: Arc<RwLock<HashMap<PathBuf, OwnedBytes>>>,
}
impl Write for InMemoryWriter {
fn write(&mut self, buf: &[u8]) -> io::Result<usize> {
self.buffer.extend_from_slice(buf);
Ok(buf.len())
}
fn flush(&mut self) -> io::Result<()> {
Ok(())
}
}
impl TerminatingWrite for InMemoryWriter {
fn terminate_ref(&mut self, _token: AntiCallToken) -> io::Result<()> {
let bytes = OwnedBytes::new(std::mem::take(&mut self.buffer));
self.blobs.write().unwrap().insert(self.path.clone(), bytes);
Ok(())
}
}
#[derive(Debug, Default)]
struct NullFileHandle;
impl HasLen for NullFileHandle {
fn len(&self) -> usize {
0
}
}
impl FileHandle for NullFileHandle {
fn read_bytes(&self, _range: std::ops::Range<usize>) -> io::Result<OwnedBytes> {
unimplemented!()
}
}
impl Directory for NullDirectory {
fn get_file_handle(&self, path: &Path) -> Result<Arc<dyn FileHandle>, OpenReadError> {
if let Some(bytes) = self.blobs.read().unwrap().get(path) {
return Ok(Arc::new(bytes.clone()));
}
Ok(Arc::new(NullFileHandle))
}
fn delete(&self, _path: &Path) -> Result<(), DeleteError> {
Ok(())
}
fn exists(&self, _path: &Path) -> Result<bool, OpenReadError> {
Ok(true)
}
fn open_write(&self, path: &Path) -> Result<WritePtr, OpenWriteError> {
let path_buf = path.to_path_buf();
if path.to_string_lossy().ends_with(".fieldnorm") {
let writer = InMemoryWriter {
path: path_buf,
buffer: Vec::new(),
blobs: Arc::clone(&self.blobs),
};
Ok(io::BufWriter::new(Box::new(writer)))
} else {
Ok(io::BufWriter::new(Box::new(NullWriter)))
}
}
fn atomic_read(&self, path: &Path) -> Result<Vec<u8>, OpenReadError> {
if let Some(bytes) = self.blobs.read().unwrap().get(path) {
return Ok(bytes.as_slice().to_vec());
}
Err(OpenReadError::FileDoesNotExist(path.to_path_buf()))
}
fn atomic_write(&self, _path: &Path, _data: &[u8]) -> io::Result<()> {
Ok(())
}
fn sync_directory(&self) -> io::Result<()> {
Ok(())
}
fn watch(&self, _watch_callback: WatchCallback) -> tantivy::Result<WatchHandle> {
Ok(WatchHandle::empty())
}
}
struct MergeScenario {
#[allow(dead_code)]
index: Index,
segments: Vec<Segment>,
settings: IndexSettings,
label: String,
}
fn build_index(
num_segments: usize,
docs_per_segment: usize,
tokens_per_doc: usize,
vocab_size: usize,
) -> MergeScenario {
let mut schema_builder = Schema::builder();
let body = schema_builder.add_text_field("body", TEXT);
let schema = schema_builder.build();
let index = Index::create_in_ram(schema.clone());
assert!(vocab_size > 0);
let total_tokens = num_segments * docs_per_segment * tokens_per_doc;
let use_unique_terms = vocab_size >= total_tokens;
let mut rng = StdRng::from_seed([7u8; 32]);
let mut next_token_id: u64 = 0;
{
let mut writer = index.writer_with_num_threads(1, 256_000_000).unwrap();
writer.set_merge_policy(Box::new(NoMergePolicy));
for _ in 0..num_segments {
for _ in 0..docs_per_segment {
let mut tokens = Vec::with_capacity(tokens_per_doc);
for _ in 0..tokens_per_doc {
let token_id = if use_unique_terms {
let id = next_token_id;
next_token_id += 1;
id
} else {
rng.random_range(0..vocab_size as u64)
};
tokens.push(format!("term_{token_id}"));
}
writer.add_document(doc!(body => tokens.join(" "))).unwrap();
}
writer.commit().unwrap();
}
}
let segments = index.searchable_segments().unwrap();
let settings = index.settings().clone();
let label = format!(
"segments={}, docs/seg={}, tokens/doc={}, vocab={}",
num_segments, docs_per_segment, tokens_per_doc, vocab_size
);
MergeScenario {
index,
segments,
settings,
label,
}
}
fn main() {
let scenarios = vec![
build_index(8, 50_000, 12, 8),
build_index(16, 50_000, 12, 8),
build_index(16, 100_000, 12, 8),
build_index(8, 50_000, 8, 8 * 50_000 * 8),
];
let mut runner = BenchRunner::new();
for scenario in scenarios {
let mut group = runner.new_group();
group.set_name(format!("merge_segments — {}", scenario.label));
let segments = scenario.segments.clone();
let settings = scenario.settings.clone();
group.register("merge", move |_| {
let output_dir = NullDirectory::default();
let filter_doc_ids = vec![None; segments.len()];
let merged_index =
merge_filtered_segments(&segments, settings.clone(), filter_doc_ids, output_dir)
.unwrap();
black_box(merged_index);
});
group.run();
}
}

View File

@@ -45,7 +45,7 @@ fn build_shared_indices(num_docs: usize, distribution: &str) -> BenchIndex {
match distribution {
"dense_random" => {
for _doc_id in 0..num_docs {
let suffix = rng.gen_range(0u64..1000u64);
let suffix = rng.random_range(0u64..1000u64);
let str_val = format!("str_{:03}", suffix);
writer
@@ -71,7 +71,7 @@ fn build_shared_indices(num_docs: usize, distribution: &str) -> BenchIndex {
}
"sparse_random" => {
for _doc_id in 0..num_docs {
let suffix = rng.gen_range(0u64..1000000u64);
let suffix = rng.random_range(0u64..1000000u64);
let str_val = format!("str_{:07}", suffix);
writer

View File

@@ -178,13 +178,11 @@ impl TinySet {
#[derive(Clone)]
pub struct BitSet {
tinysets: Box<[TinySet]>,
len: u64,
max_value: u32,
}
impl std::fmt::Debug for BitSet {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
f.debug_struct("BitSet")
.field("len", &self.len)
.field("max_value", &self.max_value)
.finish()
}
@@ -212,7 +210,6 @@ impl BitSet {
let tinybitsets = vec![TinySet::empty(); num_buckets as usize].into_boxed_slice();
BitSet {
tinysets: tinybitsets,
len: 0,
max_value,
}
}
@@ -230,7 +227,6 @@ impl BitSet {
}
BitSet {
tinysets: tinybitsets,
len: max_value as u64,
max_value,
}
}
@@ -249,17 +245,19 @@ impl BitSet {
/// Intersect with tinysets
fn intersect_update_with_iter(&mut self, other: impl Iterator<Item = TinySet>) {
self.len = 0;
for (left, right) in self.tinysets.iter_mut().zip(other) {
*left = left.intersect(right);
self.len += left.len() as u64;
}
}
/// Returns the number of elements in the `BitSet`.
#[inline]
pub fn len(&self) -> usize {
self.len as usize
self.tinysets
.iter()
.copied()
.map(|tinyset| tinyset.len())
.sum::<u32>() as usize
}
/// Inserts an element in the `BitSet`
@@ -268,7 +266,7 @@ impl BitSet {
// we do not check saturated els.
let higher = el / 64u32;
let lower = el % 64u32;
self.len += u64::from(self.tinysets[higher as usize].insert_mut(lower));
self.tinysets[higher as usize].insert_mut(lower);
}
/// Inserts an element in the `BitSet`
@@ -277,7 +275,7 @@ impl BitSet {
// we do not check saturated els.
let higher = el / 64u32;
let lower = el % 64u32;
self.len -= u64::from(self.tinysets[higher as usize].remove_mut(lower));
self.tinysets[higher as usize].remove_mut(lower);
}
/// Returns true iff the elements is in the `BitSet`.
@@ -299,6 +297,9 @@ impl BitSet {
.map(|delta_bucket| bucket + delta_bucket as u32)
}
/// Returns the maximum number of elements in the bitset.
///
/// Warning: The largest element the bitset can contain is `max_value - 1`.
#[inline]
pub fn max_value(&self) -> u32 {
self.max_value

View File

@@ -91,46 +91,10 @@ fn main() -> tantivy::Result<()> {
}
}
// A `Term` is a text token associated with a field.
// Let's go through all docs containing the term `title:the` and access their position
let term_the = Term::from_field_text(title, "the");
// Some other powerful operations (especially `.skip_to`) may be useful to consume these
// Some other powerful operations (especially `.seek`) may be useful to consume these
// posting lists rapidly.
// You can check for them in the [`DocSet`](https://docs.rs/tantivy/~0/tantivy/trait.DocSet.html) trait
// and the [`Postings`](https://docs.rs/tantivy/~0/tantivy/trait.Postings.html) trait
// Also, for some VERY specific high performance use case like an OLAP analysis of logs,
// you can get better performance by accessing directly the blocks of doc ids.
for segment_reader in searcher.segment_readers() {
// A segment contains different data structure.
// Inverted index stands for the combination of
// - the term dictionary
// - the inverted lists associated with each terms and their positions
let inverted_index = segment_reader.inverted_index(title)?;
// This segment posting object is like a cursor over the documents matching the term.
// The `IndexRecordOption` arguments tells tantivy we will be interested in both term
// frequencies and positions.
//
// If you don't need all this information, you may get better performance by decompressing
// less information.
if let Some(mut block_segment_postings) =
inverted_index.read_block_postings(&term_the, IndexRecordOption::Basic)?
{
loop {
let docs = block_segment_postings.docs();
if docs.is_empty() {
break;
}
// Once again these docs MAY contains deleted documents as well.
let docs = block_segment_postings.docs();
// Prints `Docs [0, 2].`
println!("Docs {docs:?}");
block_segment_postings.advance();
}
}
}
Ok(())
}

View File

@@ -560,7 +560,7 @@ fn range_infallible(inp: &str) -> JResult<&str, UserInputLeaf> {
(
(
value((), tag(">=")),
map(word_infallible("", false), |(bound, err)| {
map(word_infallible(")", false), |(bound, err)| {
(
(
bound
@@ -574,7 +574,7 @@ fn range_infallible(inp: &str) -> JResult<&str, UserInputLeaf> {
),
(
value((), tag("<=")),
map(word_infallible("", false), |(bound, err)| {
map(word_infallible(")", false), |(bound, err)| {
(
(
UserInputBound::Unbounded,
@@ -588,7 +588,7 @@ fn range_infallible(inp: &str) -> JResult<&str, UserInputLeaf> {
),
(
value((), tag(">")),
map(word_infallible("", false), |(bound, err)| {
map(word_infallible(")", false), |(bound, err)| {
(
(
bound
@@ -602,7 +602,7 @@ fn range_infallible(inp: &str) -> JResult<&str, UserInputLeaf> {
),
(
value((), tag("<")),
map(word_infallible("", false), |(bound, err)| {
map(word_infallible(")", false), |(bound, err)| {
(
(
UserInputBound::Unbounded,
@@ -1323,6 +1323,14 @@ mod test {
test_parse_query_to_ast_helper("<a", "{\"*\" TO \"a\"}");
test_parse_query_to_ast_helper("<=a", "{\"*\" TO \"a\"]");
test_parse_query_to_ast_helper("<=bsd", "{\"*\" TO \"bsd\"]");
test_parse_query_to_ast_helper("(<=42)", "{\"*\" TO \"42\"]");
test_parse_query_to_ast_helper("(<=42 )", "{\"*\" TO \"42\"]");
test_parse_query_to_ast_helper("(age:>5)", "\"age\":{\"5\" TO \"*\"}");
test_parse_query_to_ast_helper(
"(title:bar AND age:>12)",
"(+\"title\":bar +\"age\":{\"12\" TO \"*\"})",
);
}
#[test]

254
src/codec/mod.rs Normal file
View File

@@ -0,0 +1,254 @@
/// Codec specific to postings data.
pub mod postings;
/// Standard tantivy codec. This is the codec you use by default.
pub mod standard;
use std::io;
pub use standard::StandardCodec;
use crate::codec::postings::PostingsCodec;
use crate::fieldnorm::FieldNormReader;
use crate::postings::{Postings, TermInfo};
use crate::query::score_combiner::DoNothingCombiner;
use crate::query::term_query::TermScorer;
use crate::query::{
box_scorer, Bm25Weight, BufferedUnionScorer, PhraseScorer, Scorer, SumCombiner,
};
use crate::schema::IndexRecordOption;
use crate::{DocId, InvertedIndexReader, Score};
/// Codecs describes how data is layed out on disk.
///
/// For the moment, only postings codec can be custom.
pub trait Codec: Clone + std::fmt::Debug + Send + Sync + 'static {
/// The specific postings type used by this codec.
type PostingsCodec: PostingsCodec;
/// ID of the codec. It should be unique to your codec.
/// Make it human-readable, descriptive, short and unique.
const ID: &'static str;
/// Load codec based on the codec configuration.
fn from_json_props(json_value: &serde_json::Value) -> crate::Result<Self>;
/// Get codec configuration.
fn to_json_props(&self) -> serde_json::Value;
/// Returns the postings codec.
fn postings_codec(&self) -> &Self::PostingsCodec;
}
/// Object-safe codec is a Codec that can be used in a trait object.
///
/// The point of it is to offer a way to use a codec without a proliferation of generics.
pub trait ObjectSafeCodec: 'static + Send + Sync {
/// Loads a type-erased Postings object for the given term.
///
/// If the schema used to build the index did not provide enough
/// information to match the requested `option`, a Postings is still
/// returned in a best-effort manner.
fn load_postings_type_erased(
&self,
term_info: &TermInfo,
option: IndexRecordOption,
inverted_index_reader: &dyn InvertedIndexReader,
) -> io::Result<Box<dyn Postings>>;
/// Loads a type-erased TermScorer object for the given term.
///
/// If the schema used to build the index did not provide enough
/// information to match the requested `option`, a TermScorer is still
/// returned in a best-effort manner.
///
/// The point of this contraption is that the return TermScorer is backed,
/// not by Box<dyn Postings> but by the codec's concrete Postings type.
fn load_term_scorer_type_erased(
&self,
term_info: &TermInfo,
option: IndexRecordOption,
inverted_index_reader: &dyn InvertedIndexReader,
fieldnorm_reader: FieldNormReader,
similarity_weight: Bm25Weight,
) -> io::Result<Box<dyn Scorer>>;
/// Loads a type-erased PhraseScorer object for the given term.
///
/// If the schema used to build the index did not provide enough
/// information to match the requested `option`, a TermScorer is still
/// returned in a best-effort manner.
///
/// The point of this contraption is that the return PhraseScorer is backed,
/// not by Box<dyn Postings> but by the codec's concrete Postings type.
fn new_phrase_scorer_type_erased(
&self,
term_infos: &[(usize, TermInfo)],
similarity_weight: Option<Bm25Weight>,
fieldnorm_reader: FieldNormReader,
slop: u32,
inverted_index_reader: &dyn InvertedIndexReader,
) -> io::Result<Box<dyn Scorer>>;
/// Performs a for_each_pruning operation on the given scorer.
///
/// The function will go through matching documents and call the callback
/// function for all docs with a score exceeding the threshold.
///
/// The function itself will return a larger threshold value,
/// meant to update the threshold value.
///
/// If the codec and the scorer allow it, this function can rely on
/// optimizations like the block-max wand.
fn for_each_pruning(
&self,
threshold: Score,
scorer: Box<dyn Scorer>,
callback: &mut dyn FnMut(DocId, Score) -> Score,
);
/// Builds a union scorer possibly specialized if
/// all scorers are `Term<Self::Postings>`.
fn build_union_scorer_with_sum_combiner(
&self,
scorers: Vec<Box<dyn Scorer>>,
num_docs: DocId,
score_combiner_type: SumOrDoNothingCombiner,
) -> Box<dyn Scorer>;
}
impl<TCodec: Codec> ObjectSafeCodec for TCodec {
fn load_postings_type_erased(
&self,
term_info: &TermInfo,
option: IndexRecordOption,
inverted_index_reader: &dyn InvertedIndexReader,
) -> io::Result<Box<dyn Postings>> {
let postings_data = inverted_index_reader.read_postings_data(term_info, option)?;
let postings = self.postings_codec().load_postings(
term_info.doc_freq,
postings_data.postings_data,
postings_data.record_option,
postings_data.effective_option,
postings_data.positions_data,
)?;
Ok(Box::new(postings))
}
fn load_term_scorer_type_erased(
&self,
term_info: &TermInfo,
option: IndexRecordOption,
inverted_index_reader: &dyn InvertedIndexReader,
fieldnorm_reader: FieldNormReader,
similarity_weight: Bm25Weight,
) -> io::Result<Box<dyn Scorer>> {
let postings_data = inverted_index_reader.read_postings_data(term_info, option)?;
let postings = self.postings_codec().load_postings(
term_info.doc_freq,
postings_data.postings_data,
postings_data.record_option,
postings_data.effective_option,
postings_data.positions_data,
)?;
let scorer = TermScorer::new(postings, fieldnorm_reader, similarity_weight);
Ok(box_scorer(scorer))
}
fn new_phrase_scorer_type_erased(
&self,
term_infos: &[(usize, TermInfo)],
similarity_weight: Option<Bm25Weight>,
fieldnorm_reader: FieldNormReader,
slop: u32,
inverted_index_reader: &dyn InvertedIndexReader,
) -> io::Result<Box<dyn Scorer>> {
let mut offset_and_term_postings: Vec<(
usize,
<<Self as Codec>::PostingsCodec as PostingsCodec>::Postings,
)> = Vec::with_capacity(term_infos.len());
for (offset, term_info) in term_infos {
let postings_data = inverted_index_reader
.read_postings_data(term_info, IndexRecordOption::WithFreqsAndPositions)?;
let postings = self.postings_codec().load_postings(
term_info.doc_freq,
postings_data.postings_data,
postings_data.record_option,
postings_data.effective_option,
postings_data.positions_data,
)?;
offset_and_term_postings.push((*offset, postings));
}
let scorer = PhraseScorer::new(
offset_and_term_postings,
similarity_weight,
fieldnorm_reader,
slop,
);
Ok(box_scorer(scorer))
}
fn build_union_scorer_with_sum_combiner(
&self,
scorers: Vec<Box<dyn Scorer>>,
num_docs: DocId,
sum_or_do_nothing_combiner: SumOrDoNothingCombiner,
) -> Box<dyn Scorer> {
if !scorers.iter().all(|scorer| {
scorer.is::<TermScorer<<<Self as Codec>::PostingsCodec as PostingsCodec>::Postings>>()
}) {
return box_scorer(BufferedUnionScorer::build(
scorers,
SumCombiner::default,
num_docs,
));
}
let specialized_scorers: Vec<
TermScorer<<<Self as Codec>::PostingsCodec as PostingsCodec>::Postings>,
> = scorers
.into_iter()
.map(|scorer| {
*scorer.downcast::<TermScorer<_>>().ok().expect(
"Downcast failed despite the fact we already checked the type was correct",
)
})
.collect();
match sum_or_do_nothing_combiner {
SumOrDoNothingCombiner::Sum => box_scorer(BufferedUnionScorer::build(
specialized_scorers,
SumCombiner::default,
num_docs,
)),
SumOrDoNothingCombiner::DoNothing => box_scorer(BufferedUnionScorer::build(
specialized_scorers,
DoNothingCombiner::default,
num_docs,
)),
}
}
fn for_each_pruning(
&self,
threshold: Score,
scorer: Box<dyn Scorer>,
callback: &mut dyn FnMut(DocId, Score) -> Score,
) {
let accerelerated_foreach_pruning_res =
<TCodec as Codec>::PostingsCodec::try_accelerated_for_each_pruning(
threshold, scorer, callback,
);
if let Err(mut scorer) = accerelerated_foreach_pruning_res {
// No acceleration available. We need to do things manually.
scorer.for_each_pruning(threshold, callback);
}
}
}
/// SumCombiner or DoNothingCombiner
#[derive(Copy, Clone)]
pub enum SumOrDoNothingCombiner {
/// Sum scores together
Sum,
/// Do not track any score.
DoNothing,
}

View File

@@ -1,5 +1,6 @@
use std::ops::{Deref, DerefMut};
use crate::codec::postings::PostingsWithBlockMax;
use crate::query::term_query::TermScorer;
use crate::query::Scorer;
use crate::{DocId, DocSet, Score, TERMINATED};
@@ -13,8 +14,8 @@ use crate::{DocId, DocSet, Score, TERMINATED};
/// We always have `before_pivot_len` < `pivot_len`.
///
/// `None` is returned if we establish that no document can exceed the threshold.
fn find_pivot_doc(
term_scorers: &[TermScorerWithMaxScore],
fn find_pivot_doc<TPostings: PostingsWithBlockMax>(
term_scorers: &[TermScorerWithMaxScore<TPostings>],
threshold: Score,
) -> Option<(usize, usize, DocId)> {
let mut max_score = 0.0;
@@ -46,8 +47,8 @@ fn find_pivot_doc(
/// the next doc candidate defined by the min of `last_doc_in_block + 1` for
/// scorer in scorers[..pivot_len] and `scorer.doc()` for scorer in scorers[pivot_len..].
/// Note: before and after calling this method, scorers need to be sorted by their `.doc()`.
fn block_max_was_too_low_advance_one_scorer(
scorers: &mut [TermScorerWithMaxScore],
fn block_max_was_too_low_advance_one_scorer<TPostings: PostingsWithBlockMax>(
scorers: &mut [TermScorerWithMaxScore<TPostings>],
pivot_len: usize,
) {
debug_assert!(is_sorted(scorers.iter().map(|scorer| scorer.doc())));
@@ -82,7 +83,10 @@ fn block_max_was_too_low_advance_one_scorer(
// Given a list of term_scorers and a `ord` and assuming that `term_scorers[ord]` is sorted
// except term_scorers[ord] that might be in advance compared to its ranks,
// bubble up term_scorers[ord] in order to restore the ordering.
fn restore_ordering(term_scorers: &mut [TermScorerWithMaxScore], ord: usize) {
fn restore_ordering<TPostings: PostingsWithBlockMax>(
term_scorers: &mut [TermScorerWithMaxScore<TPostings>],
ord: usize,
) {
let doc = term_scorers[ord].doc();
for i in ord + 1..term_scorers.len() {
if term_scorers[i].doc() >= doc {
@@ -97,9 +101,10 @@ fn restore_ordering(term_scorers: &mut [TermScorerWithMaxScore], ord: usize) {
// If this works, return true.
// If this fails (ie: one of the term_scorer does not contain `pivot_doc` and seek goes past the
// pivot), reorder the term_scorers to ensure the list is still sorted and returns `false`.
// If a term_scorer reach TERMINATED in the process return false remove the term_scorer and return.
fn align_scorers(
term_scorers: &mut Vec<TermScorerWithMaxScore>,
// If a term_scorer reach TERMINATED in the process return false remove the term_scorer and
// return.
fn align_scorers<TPostings: PostingsWithBlockMax>(
term_scorers: &mut Vec<TermScorerWithMaxScore<TPostings>>,
pivot_doc: DocId,
before_pivot_len: usize,
) -> bool {
@@ -126,7 +131,10 @@ fn align_scorers(
// Assumes terms_scorers[..pivot_len] are positioned on the same doc (pivot_doc).
// Advance term_scorers[..pivot_len] and out of these removes the terminated scores.
// Restores the ordering of term_scorers.
fn advance_all_scorers_on_pivot(term_scorers: &mut Vec<TermScorerWithMaxScore>, pivot_len: usize) {
fn advance_all_scorers_on_pivot<TPostings: PostingsWithBlockMax>(
term_scorers: &mut Vec<TermScorerWithMaxScore<TPostings>>,
pivot_len: usize,
) {
for term_scorer in &mut term_scorers[..pivot_len] {
term_scorer.advance();
}
@@ -145,12 +153,12 @@ fn advance_all_scorers_on_pivot(term_scorers: &mut Vec<TermScorerWithMaxScore>,
/// Implements the WAND (Weak AND) algorithm for dynamic pruning
/// described in the paper "Faster Top-k Document Retrieval Using Block-Max Indexes".
/// Link: <http://engineering.nyu.edu/~suel/papers/bmw.pdf>
pub fn block_wand(
mut scorers: Vec<TermScorer>,
pub fn block_wand<TPostings: PostingsWithBlockMax>(
mut scorers: Vec<TermScorer<TPostings>>,
mut threshold: Score,
callback: &mut dyn FnMut(u32, Score) -> Score,
) {
let mut scorers: Vec<TermScorerWithMaxScore> = scorers
let mut scorers: Vec<TermScorerWithMaxScore<TPostings>> = scorers
.iter_mut()
.map(TermScorerWithMaxScore::from)
.collect();
@@ -166,10 +174,7 @@ pub fn block_wand(
let block_max_score_upperbound: Score = scorers[..pivot_len]
.iter_mut()
.map(|scorer| {
scorer.seek_block(pivot_doc);
scorer.block_max_score()
})
.map(|scorer| scorer.seek_block_max(pivot_doc))
.sum();
// Beware after shallow advance, skip readers can be in advance compared to
@@ -220,21 +225,22 @@ pub fn block_wand(
/// - On a block, advance until the end and execute `callback` when the doc score is greater or
/// equal to the `threshold`.
pub fn block_wand_single_scorer(
mut scorer: TermScorer,
mut scorer: TermScorer<impl PostingsWithBlockMax>,
mut threshold: Score,
callback: &mut dyn FnMut(u32, Score) -> Score,
) {
let mut doc = scorer.doc();
let mut block_max_score = scorer.seek_block_max(doc);
loop {
// We position the scorer on a block that can reach
// the threshold.
while scorer.block_max_score() < threshold {
while block_max_score < threshold {
let last_doc_in_block = scorer.last_doc_in_block();
if last_doc_in_block == TERMINATED {
return;
}
doc = last_doc_in_block + 1;
scorer.seek_block(doc);
block_max_score = scorer.seek_block_max(doc);
}
// Seek will effectively load that block.
doc = scorer.seek(doc);
@@ -256,31 +262,33 @@ pub fn block_wand_single_scorer(
}
}
doc += 1;
scorer.seek_block(doc);
block_max_score = scorer.seek_block_max(doc);
}
}
struct TermScorerWithMaxScore<'a> {
scorer: &'a mut TermScorer,
struct TermScorerWithMaxScore<'a, TPostings: PostingsWithBlockMax> {
scorer: &'a mut TermScorer<TPostings>,
max_score: Score,
}
impl<'a> From<&'a mut TermScorer> for TermScorerWithMaxScore<'a> {
fn from(scorer: &'a mut TermScorer) -> Self {
impl<'a, TPostings: PostingsWithBlockMax> From<&'a mut TermScorer<TPostings>>
for TermScorerWithMaxScore<'a, TPostings>
{
fn from(scorer: &'a mut TermScorer<TPostings>) -> Self {
let max_score = scorer.max_score();
TermScorerWithMaxScore { scorer, max_score }
}
}
impl Deref for TermScorerWithMaxScore<'_> {
type Target = TermScorer;
impl<TPostings: PostingsWithBlockMax> Deref for TermScorerWithMaxScore<'_, TPostings> {
type Target = TermScorer<TPostings>;
fn deref(&self) -> &Self::Target {
self.scorer
}
}
impl DerefMut for TermScorerWithMaxScore<'_> {
impl<TPostings: PostingsWithBlockMax> DerefMut for TermScorerWithMaxScore<'_, TPostings> {
fn deref_mut(&mut self) -> &mut Self::Target {
self.scorer
}

75
src/codec/postings/mod.rs Normal file
View File

@@ -0,0 +1,75 @@
use std::io;
/// Block-max WAND algorithm.
pub mod block_wand;
use common::OwnedBytes;
use crate::fieldnorm::FieldNormReader;
use crate::postings::Postings;
use crate::query::{Bm25Weight, Scorer};
use crate::schema::IndexRecordOption;
use crate::{DocId, Score};
/// Postings codec (read path).
pub trait PostingsCodec: Send + Sync + 'static {
/// Postings type for the postings codec.
type Postings: Postings + Clone;
/// Loads postings
///
/// Record option is the option that was passed at indexing time.
/// Requested option is the option that is requested.
///
/// For instance, we may have term_freq in the posting list
/// but we can skip decompressing as we read the posting list.
///
/// If record option does not support the requested option,
/// this method does NOT return an error and will in fact restrict
/// requested_option to what is available.
fn load_postings(
&self,
doc_freq: u32,
postings_data: OwnedBytes,
record_option: IndexRecordOption,
requested_option: IndexRecordOption,
positions_data: Option<OwnedBytes>,
) -> io::Result<Self::Postings>;
/// If your codec supports different ways to accelerate `for_each_pruning` that's
/// where you should implement it.
///
/// Returning `Err(scorer)` without mutating the scorer nor calling the callback function,
/// is never "wrong". It just leaves the responsability to the caller to call a fallback
/// implementation on the scorer.
///
/// If your codec supports BlockMax-Wand, you just need to have your
/// postings implement `PostingsWithBlockMax` and copy what is done in the StandardPostings
/// codec to enable it.
fn try_accelerated_for_each_pruning(
_threshold: Score,
scorer: Box<dyn Scorer>,
_callback: &mut dyn FnMut(DocId, Score) -> Score,
) -> Result<(), Box<dyn Scorer>> {
Err(scorer)
}
}
/// A light complement interface to Postings to allow block-max wand acceleration.
pub trait PostingsWithBlockMax: Postings {
/// Moves the postings to the block containign `target_doc` and returns
/// an upperbound of the score for documents in the block.
///
/// `Warning`: Calling this method may leave the postings in an invalid state.
/// callers are required to call seek before calling any other of the
/// `Postings` method (like doc / advance etc.).
fn seek_block_max(
&mut self,
target_doc: crate::DocId,
fieldnorm_reader: &FieldNormReader,
similarity_weight: &Bm25Weight,
) -> Score;
/// Returns the last document in the current block (or Terminated if this
/// is the last block).
fn last_doc_in_block(&self) -> crate::DocId;
}

35
src/codec/standard/mod.rs Normal file
View File

@@ -0,0 +1,35 @@
use serde::{Deserialize, Serialize};
use crate::codec::standard::postings::StandardPostingsCodec;
use crate::codec::Codec;
/// Tantivy's default postings codec.
pub mod postings;
/// Tantivy's default codec.
#[derive(Debug, Default, Clone, Serialize, Deserialize)]
pub struct StandardCodec;
impl Codec for StandardCodec {
type PostingsCodec = StandardPostingsCodec;
const ID: &'static str = "tantivy-default";
fn from_json_props(json_value: &serde_json::Value) -> crate::Result<Self> {
if !json_value.is_null() {
return Err(crate::TantivyError::InvalidArgument(format!(
"Codec property for the StandardCodec are unexpected. expected null, got {}",
json_value.as_str().unwrap_or("null")
)));
}
Ok(StandardCodec)
}
fn to_json_props(&self) -> serde_json::Value {
serde_json::Value::Null
}
fn postings_codec(&self) -> &Self::PostingsCodec {
&StandardPostingsCodec
}
}

View File

@@ -1,28 +1,19 @@
use std::io;
use common::VInt;
use common::{OwnedBytes, VInt};
use crate::directory::{FileSlice, OwnedBytes};
use crate::codec::standard::postings::FreqReadingOption;
use crate::fieldnorm::FieldNormReader;
use crate::postings::compression::{BlockDecoder, VIntDecoder, COMPRESSION_BLOCK_SIZE};
use crate::postings::{BlockInfo, FreqReadingOption, SkipReader};
use crate::postings::compression::{BlockDecoder, VIntDecoder as _, COMPRESSION_BLOCK_SIZE};
use crate::postings::skip::{BlockInfo, SkipReader};
use crate::query::Bm25Weight;
use crate::schema::IndexRecordOption;
use crate::{DocId, Score, TERMINATED};
fn max_score<I: Iterator<Item = Score>>(mut it: I) -> Option<Score> {
it.next().map(|first| it.fold(first, Score::max))
}
/// `BlockSegmentPostings` is a cursor iterating over blocks
/// of documents.
///
/// # Warning
///
/// While it is useful for some very specific high-performance
/// use cases, you should prefer using `SegmentPostings` for most usage.
#[derive(Clone)]
pub struct BlockSegmentPostings {
pub(crate) struct BlockSegmentPostings {
pub(crate) doc_decoder: BlockDecoder,
block_loaded: bool,
freq_decoder: BlockDecoder,
@@ -88,7 +79,7 @@ fn split_into_skips_and_postings(
}
impl BlockSegmentPostings {
/// Opens a `BlockSegmentPostings`.
/// Opens a `StandardPostingsReader`.
/// `doc_freq` is the number of documents in the posting list.
/// `record_option` represents the amount of data available according to the schema.
/// `requested_option` is the amount of data requested by the user.
@@ -96,11 +87,10 @@ impl BlockSegmentPostings {
/// term frequency blocks.
pub(crate) fn open(
doc_freq: u32,
data: FileSlice,
bytes: OwnedBytes,
mut record_option: IndexRecordOption,
requested_option: IndexRecordOption,
) -> io::Result<BlockSegmentPostings> {
let bytes = data.read_bytes()?;
let (skip_data_opt, postings_data) = split_into_skips_and_postings(doc_freq, bytes)?;
let skip_reader = match skip_data_opt {
Some(skip_data) => {
@@ -138,6 +128,86 @@ impl BlockSegmentPostings {
block_segment_postings.load_block();
Ok(block_segment_postings)
}
}
fn max_score<I: Iterator<Item = Score>>(mut it: I) -> Option<Score> {
it.next().map(|first| it.fold(first, Score::max))
}
impl BlockSegmentPostings {
/// Returns the overall number of documents in the block postings.
/// It does not take in account whether documents are deleted or not.
///
/// This `doc_freq` is simply the sum of the length of all of the blocks
/// length, and it does not take in account deleted documents.
pub fn doc_freq(&self) -> u32 {
self.doc_freq
}
/// Returns the array of docs in the current block.
///
/// Before the first call to `.advance()`, the block
/// returned by `.docs()` is empty.
#[inline]
pub fn docs(&self) -> &[DocId] {
debug_assert!(self.block_loaded);
self.doc_decoder.output_array()
}
/// Return the document at index `idx` of the block.
#[inline]
pub fn doc(&self, idx: usize) -> u32 {
self.doc_decoder.output(idx)
}
/// Return the array of `term freq` in the block.
#[inline]
pub fn freqs(&self) -> &[u32] {
debug_assert!(self.block_loaded);
self.freq_decoder.output_array()
}
/// Return the frequency at index `idx` of the block.
#[inline]
pub fn freq(&self, idx: usize) -> u32 {
debug_assert!(self.block_loaded);
self.freq_decoder.output(idx)
}
/// Position on a block that may contains `target_doc`.
///
/// If all docs are smaller than target, the block loaded may be empty,
/// or be the last an incomplete VInt block.
pub fn seek(&mut self, target_doc: DocId) -> usize {
// Move to the block that might contain our document.
self.seek_block_without_loading(target_doc);
self.load_block();
// At this point we are on the block that might contain our document.
let doc = self.doc_decoder.seek_within_block(target_doc);
// The last block is not full and padded with TERMINATED,
// so we are guaranteed to have at least one value (real or padding)
// that is >= target_doc.
debug_assert!(doc < COMPRESSION_BLOCK_SIZE);
// `doc` is now the first element >= `target_doc`.
// If all docs are smaller than target, the current block is incomplete and padded
// with TERMINATED. After the search, the cursor points to the first TERMINATED.
doc
}
pub fn position_offset(&self) -> u64 {
self.skip_reader.position_offset()
}
/// Advance to the next block.
pub fn advance(&mut self) {
self.skip_reader.advance();
self.block_loaded = false;
self.block_max_score_cache = None;
self.load_block();
}
/// Returns the block_max_score for the current block.
/// It does not require the block to be loaded. For instance, it is ok to call this method
@@ -160,7 +230,7 @@ impl BlockSegmentPostings {
}
// this is the last block of the segment posting list.
// If it is actually loaded, we can compute block max manually.
if self.block_is_loaded() {
if self.block_loaded {
let docs = self.doc_decoder.output_array().iter().cloned();
let freqs = self.freq_decoder.output_array().iter().cloned();
let bm25_scores = docs.zip(freqs).map(|(doc, term_freq)| {
@@ -177,112 +247,25 @@ impl BlockSegmentPostings {
// We do not cache it however, so that it gets computed when once block is loaded.
bm25_weight.max_score()
}
}
pub(crate) fn freq_reading_option(&self) -> FreqReadingOption {
self.freq_reading_option
}
// Resets the block segment postings on another position
// in the postings file.
//
// This is useful for enumerating through a list of terms,
// and consuming the associated posting lists while avoiding
// reallocating a `BlockSegmentPostings`.
//
// # Warning
//
// This does not reset the positions list.
pub(crate) fn reset(&mut self, doc_freq: u32, postings_data: OwnedBytes) -> io::Result<()> {
let (skip_data_opt, postings_data) =
split_into_skips_and_postings(doc_freq, postings_data)?;
self.data = postings_data;
self.block_max_score_cache = None;
self.block_loaded = false;
if let Some(skip_data) = skip_data_opt {
self.skip_reader.reset(skip_data, doc_freq);
} else {
self.skip_reader.reset(OwnedBytes::empty(), doc_freq);
impl BlockSegmentPostings {
/// Returns an empty segment postings object
pub fn empty() -> BlockSegmentPostings {
BlockSegmentPostings {
doc_decoder: BlockDecoder::with_val(TERMINATED),
block_loaded: true,
freq_decoder: BlockDecoder::with_val(1),
freq_reading_option: FreqReadingOption::NoFreq,
block_max_score_cache: None,
doc_freq: 0,
data: OwnedBytes::empty(),
skip_reader: SkipReader::new(OwnedBytes::empty(), 0, IndexRecordOption::Basic),
}
self.doc_freq = doc_freq;
self.load_block();
Ok(())
}
/// Returns the overall number of documents in the block postings.
/// It does not take in account whether documents are deleted or not.
///
/// This `doc_freq` is simply the sum of the length of all of the blocks
/// length, and it does not take in account deleted documents.
pub fn doc_freq(&self) -> u32 {
self.doc_freq
}
/// Returns the array of docs in the current block.
///
/// Before the first call to `.advance()`, the block
/// returned by `.docs()` is empty.
#[inline]
pub fn docs(&self) -> &[DocId] {
debug_assert!(self.block_is_loaded());
self.doc_decoder.output_array()
}
/// Return the document at index `idx` of the block.
#[inline]
pub fn doc(&self, idx: usize) -> u32 {
self.doc_decoder.output(idx)
}
/// Return the array of `term freq` in the block.
#[inline]
pub fn freqs(&self) -> &[u32] {
debug_assert!(self.block_is_loaded());
self.freq_decoder.output_array()
}
/// Return the frequency at index `idx` of the block.
#[inline]
pub fn freq(&self, idx: usize) -> u32 {
debug_assert!(self.block_is_loaded());
self.freq_decoder.output(idx)
}
/// Returns the length of the current block.
///
/// All blocks have a length of `NUM_DOCS_PER_BLOCK`,
/// except the last block that may have a length
/// of any number between 1 and `NUM_DOCS_PER_BLOCK - 1`
#[inline]
pub fn block_len(&self) -> usize {
debug_assert!(self.block_is_loaded());
self.doc_decoder.output_len
}
/// Position on a block that may contains `target_doc`.
///
/// If all docs are smaller than target, the block loaded may be empty,
/// or be the last an incomplete VInt block.
pub fn seek(&mut self, target_doc: DocId) -> usize {
// Move to the block that might contain our document.
self.seek_block(target_doc);
self.load_block();
// At this point we are on the block that might contain our document.
let doc = self.doc_decoder.seek_within_block(target_doc);
// The last block is not full and padded with TERMINATED,
// so we are guaranteed to have at least one value (real or padding)
// that is >= target_doc.
debug_assert!(doc < COMPRESSION_BLOCK_SIZE);
// `doc` is now the first element >= `target_doc`.
// If all docs are smaller than target, the current block is incomplete and padded
// with TERMINATED. After the search, the cursor points to the first TERMINATED.
doc
}
pub(crate) fn position_offset(&self) -> u64 {
self.skip_reader.position_offset()
pub(crate) fn skip_reader(&self) -> &SkipReader {
&self.skip_reader
}
/// Dangerous API! This calls seeks the next block on the skip list,
@@ -291,22 +274,18 @@ impl BlockSegmentPostings {
/// `.load_block()` needs to be called manually afterwards.
/// If all docs are smaller than target, the block loaded may be empty,
/// or be the last an incomplete VInt block.
pub(crate) fn seek_block(&mut self, target_doc: DocId) {
pub(crate) fn seek_block_without_loading(&mut self, target_doc: DocId) {
if self.skip_reader.seek(target_doc) {
self.block_max_score_cache = None;
self.block_loaded = false;
}
}
pub(crate) fn block_is_loaded(&self) -> bool {
self.block_loaded
}
pub(crate) fn load_block(&mut self) {
let offset = self.skip_reader.byte_offset();
if self.block_is_loaded() {
if self.block_loaded {
return;
}
let offset = self.skip_reader.byte_offset();
match self.skip_reader.block_info() {
BlockInfo::BitPacked {
doc_num_bits,
@@ -351,68 +330,39 @@ impl BlockSegmentPostings {
}
self.block_loaded = true;
}
/// Advance to the next block.
pub fn advance(&mut self) {
self.skip_reader.advance();
self.block_loaded = false;
self.block_max_score_cache = None;
self.load_block();
}
/// Returns an empty segment postings object
pub fn empty() -> BlockSegmentPostings {
BlockSegmentPostings {
doc_decoder: BlockDecoder::with_val(TERMINATED),
block_loaded: true,
freq_decoder: BlockDecoder::with_val(1),
freq_reading_option: FreqReadingOption::NoFreq,
block_max_score_cache: None,
doc_freq: 0,
data: OwnedBytes::empty(),
skip_reader: SkipReader::new(OwnedBytes::empty(), 0, IndexRecordOption::Basic),
}
}
pub(crate) fn skip_reader(&self) -> &SkipReader {
&self.skip_reader
}
}
#[cfg(test)]
mod tests {
use common::HasLen;
use common::OwnedBytes;
use super::BlockSegmentPostings;
use crate::codec::standard::postings::segment_postings::SegmentPostings;
use crate::docset::{DocSet, TERMINATED};
use crate::index::Index;
use crate::postings::compression::COMPRESSION_BLOCK_SIZE;
use crate::postings::postings::Postings;
use crate::postings::SegmentPostings;
use crate::schema::{IndexRecordOption, Schema, Term, INDEXED};
use crate::DocId;
use crate::postings::serializer::PostingsSerializer;
use crate::schema::IndexRecordOption;
#[test]
fn test_empty_segment_postings() {
let mut postings = SegmentPostings::empty();
assert_eq!(postings.doc(), TERMINATED);
assert_eq!(postings.advance(), TERMINATED);
assert_eq!(postings.advance(), TERMINATED);
assert_eq!(postings.doc_freq(), 0);
assert_eq!(postings.len(), 0);
}
#[test]
fn test_empty_postings_doc_returns_terminated() {
let mut postings = SegmentPostings::empty();
assert_eq!(postings.doc(), TERMINATED);
assert_eq!(postings.advance(), TERMINATED);
}
#[test]
fn test_empty_postings_doc_term_freq_returns_0() {
let postings = SegmentPostings::empty();
assert_eq!(postings.term_freq(), 1);
#[cfg(test)]
fn build_block_postings(docs: &[u32]) -> BlockSegmentPostings {
let doc_freq = docs.len() as u32;
let mut postings_serializer =
PostingsSerializer::new(1.0f32, IndexRecordOption::Basic, None);
postings_serializer.new_term(docs.len() as u32, false);
for doc in docs {
postings_serializer.write_doc(*doc, 1u32);
}
let mut buffer: Vec<u8> = Vec::new();
postings_serializer
.close_term(doc_freq, &mut buffer)
.unwrap();
BlockSegmentPostings::open(
doc_freq,
OwnedBytes::new(buffer),
IndexRecordOption::Basic,
IndexRecordOption::Basic,
)
.unwrap()
}
#[test]
@@ -427,7 +377,7 @@ mod tests {
#[test]
fn test_block_segment_postings() -> crate::Result<()> {
let mut block_segments = build_block_postings(&(0..100_000).collect::<Vec<u32>>())?;
let mut block_segments = build_block_postings(&(0..100_000).collect::<Vec<u32>>());
let mut offset: u32 = 0u32;
// checking that the `doc_freq` is correct
assert_eq!(block_segments.doc_freq(), 100_000);
@@ -452,7 +402,7 @@ mod tests {
doc_ids.push(129);
doc_ids.push(130);
{
let block_segments = build_block_postings(&doc_ids)?;
let block_segments = build_block_postings(&doc_ids);
let mut docset = SegmentPostings::from_block_postings(block_segments, None);
assert_eq!(docset.seek(128), 129);
assert_eq!(docset.doc(), 129);
@@ -461,7 +411,7 @@ mod tests {
assert_eq!(docset.advance(), TERMINATED);
}
{
let block_segments = build_block_postings(&doc_ids).unwrap();
let block_segments = build_block_postings(&doc_ids);
let mut docset = SegmentPostings::from_block_postings(block_segments, None);
assert_eq!(docset.seek(129), 129);
assert_eq!(docset.doc(), 129);
@@ -470,7 +420,7 @@ mod tests {
assert_eq!(docset.advance(), TERMINATED);
}
{
let block_segments = build_block_postings(&doc_ids)?;
let block_segments = build_block_postings(&doc_ids);
let mut docset = SegmentPostings::from_block_postings(block_segments, None);
assert_eq!(docset.doc(), 0);
assert_eq!(docset.seek(131), TERMINATED);
@@ -479,38 +429,13 @@ mod tests {
Ok(())
}
fn build_block_postings(docs: &[DocId]) -> crate::Result<BlockSegmentPostings> {
let mut schema_builder = Schema::builder();
let int_field = schema_builder.add_u64_field("id", INDEXED);
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
let mut index_writer = index.writer_for_tests()?;
let mut last_doc = 0u32;
for &doc in docs {
for _ in last_doc..doc {
index_writer.add_document(doc!(int_field=>1u64))?;
}
index_writer.add_document(doc!(int_field=>0u64))?;
last_doc = doc + 1;
}
index_writer.commit()?;
let searcher = index.reader()?.searcher();
let segment_reader = searcher.segment_reader(0);
let inverted_index = segment_reader.inverted_index(int_field).unwrap();
let term = Term::from_field_u64(int_field, 0u64);
let term_info = inverted_index.get_term_info(&term)?.unwrap();
let block_postings = inverted_index
.read_block_postings_from_terminfo(&term_info, IndexRecordOption::Basic)?;
Ok(block_postings)
}
#[test]
fn test_block_segment_postings_seek() -> crate::Result<()> {
let mut docs = vec![0];
let mut docs = Vec::new();
for i in 0..1300 {
docs.push((i * i / 100) + i);
}
let mut block_postings = build_block_postings(&docs[..])?;
let mut block_postings = build_block_postings(&docs[..]);
for i in &[0, 424, 10000] {
block_postings.seek(*i);
let docs = block_postings.docs();
@@ -521,40 +446,4 @@ mod tests {
assert_eq!(block_postings.doc(COMPRESSION_BLOCK_SIZE - 1), TERMINATED);
Ok(())
}
#[test]
fn test_reset_block_segment_postings() -> crate::Result<()> {
let mut schema_builder = Schema::builder();
let int_field = schema_builder.add_u64_field("id", INDEXED);
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
let mut index_writer = index.writer_for_tests()?;
// create two postings list, one containing even number,
// the other containing odd numbers.
for i in 0..6 {
let doc = doc!(int_field=> (i % 2) as u64);
index_writer.add_document(doc)?;
}
index_writer.commit()?;
let searcher = index.reader()?.searcher();
let segment_reader = searcher.segment_reader(0);
let mut block_segments;
{
let term = Term::from_field_u64(int_field, 0u64);
let inverted_index = segment_reader.inverted_index(int_field)?;
let term_info = inverted_index.get_term_info(&term)?.unwrap();
block_segments = inverted_index
.read_block_postings_from_terminfo(&term_info, IndexRecordOption::Basic)?;
}
assert_eq!(block_segments.docs(), &[0, 2, 4]);
{
let term = Term::from_field_u64(int_field, 1u64);
let inverted_index = segment_reader.inverted_index(int_field)?;
let term_info = inverted_index.get_term_info(&term)?.unwrap();
inverted_index.reset_block_postings_from_terminfo(&term_info, &mut block_segments)?;
}
assert_eq!(block_segments.docs(), &[1, 3, 5]);
Ok(())
}
}

View File

@@ -0,0 +1,134 @@
use std::io;
use crate::codec::postings::block_wand::{block_wand, block_wand_single_scorer};
use crate::codec::postings::PostingsCodec;
use crate::codec::standard::postings::block_segment_postings::BlockSegmentPostings;
pub use crate::codec::standard::postings::segment_postings::SegmentPostings;
use crate::positions::PositionReader;
use crate::query::term_query::TermScorer;
use crate::query::{BufferedUnionScorer, Scorer, SumCombiner};
use crate::schema::IndexRecordOption;
use crate::{DocSet as _, Score, TERMINATED};
mod block_segment_postings;
mod segment_postings;
pub use segment_postings::SegmentPostings as StandardPostings;
/// The default postings codec for tantivy.
pub struct StandardPostingsCodec;
#[expect(clippy::enum_variant_names)]
#[derive(Debug, PartialEq, Clone, Copy, Eq)]
pub(crate) enum FreqReadingOption {
NoFreq,
SkipFreq,
ReadFreq,
}
impl PostingsCodec for StandardPostingsCodec {
type Postings = SegmentPostings;
fn load_postings(
&self,
doc_freq: u32,
postings_data: common::OwnedBytes,
record_option: IndexRecordOption,
requested_option: IndexRecordOption,
positions_data_opt: Option<common::OwnedBytes>,
) -> io::Result<Self::Postings> {
// Rationalize record_option/requested_option.
let requested_option = requested_option.downgrade(record_option);
let block_segment_postings =
BlockSegmentPostings::open(doc_freq, postings_data, record_option, requested_option)?;
let position_reader = positions_data_opt.map(PositionReader::open).transpose()?;
Ok(SegmentPostings::from_block_postings(
block_segment_postings,
position_reader,
))
}
fn try_accelerated_for_each_pruning(
mut threshold: Score,
mut scorer: Box<dyn Scorer>,
callback: &mut dyn FnMut(crate::DocId, Score) -> Score,
) -> Result<(), Box<dyn Scorer>> {
scorer = match scorer.downcast::<TermScorer<Self::Postings>>() {
Ok(term_scorer) => {
block_wand_single_scorer(*term_scorer, threshold, callback);
return Ok(());
}
Err(scorer) => scorer,
};
let mut union_scorer =
scorer.downcast::<BufferedUnionScorer<TermScorer<Self::Postings>, SumCombiner>>()?;
let doc = union_scorer.doc();
if doc == TERMINATED {
return Ok(());
}
let score = union_scorer.score();
if score > threshold {
threshold = callback(doc, score);
}
let scorers: Vec<TermScorer<Self::Postings>> = union_scorer.into_scorers();
block_wand(scorers, threshold, callback);
Ok(())
}
}
#[cfg(test)]
mod tests {
use common::OwnedBytes;
use super::*;
use crate::postings::serializer::PostingsSerializer;
use crate::postings::Postings as _;
fn test_segment_postings_tf_aux(num_docs: u32, include_term_freq: bool) -> SegmentPostings {
let mut postings_serializer =
PostingsSerializer::new(1.0f32, IndexRecordOption::WithFreqs, None);
let mut buffer = Vec::new();
postings_serializer.new_term(num_docs, include_term_freq);
for i in 0..num_docs {
postings_serializer.write_doc(i, 2);
}
postings_serializer
.close_term(num_docs, &mut buffer)
.unwrap();
StandardPostingsCodec
.load_postings(
num_docs,
OwnedBytes::new(buffer),
IndexRecordOption::WithFreqs,
IndexRecordOption::WithFreqs,
None,
)
.unwrap()
}
#[test]
fn test_segment_postings_small_block_with_and_without_freq() {
let small_block_without_term_freq = test_segment_postings_tf_aux(1, false);
assert!(!small_block_without_term_freq.has_freq());
assert_eq!(small_block_without_term_freq.doc(), 0);
assert_eq!(small_block_without_term_freq.term_freq(), 1);
let small_block_with_term_freq = test_segment_postings_tf_aux(1, true);
assert!(small_block_with_term_freq.has_freq());
assert_eq!(small_block_with_term_freq.doc(), 0);
assert_eq!(small_block_with_term_freq.term_freq(), 2);
}
#[test]
fn test_segment_postings_large_block_with_and_without_freq() {
let large_block_without_term_freq = test_segment_postings_tf_aux(128, false);
assert!(!large_block_without_term_freq.has_freq());
assert_eq!(large_block_without_term_freq.doc(), 0);
assert_eq!(large_block_without_term_freq.term_freq(), 1);
let large_block_with_term_freq = test_segment_postings_tf_aux(128, true);
assert!(large_block_with_term_freq.has_freq());
assert_eq!(large_block_with_term_freq.doc(), 0);
assert_eq!(large_block_with_term_freq.term_freq(), 2);
}
}

View File

@@ -1,11 +1,14 @@
use common::HasLen;
use common::BitSet;
use super::BlockSegmentPostings;
use crate::codec::postings::PostingsWithBlockMax;
use crate::docset::DocSet;
use crate::fastfield::AliveBitSet;
use crate::fieldnorm::FieldNormReader;
use crate::positions::PositionReader;
use crate::postings::compression::COMPRESSION_BLOCK_SIZE;
use crate::postings::{BlockSegmentPostings, Postings};
use crate::{DocId, TERMINATED};
use crate::postings::{DocFreq, Postings};
use crate::query::Bm25Weight;
use crate::{DocId, Score};
/// `SegmentPostings` represents the inverted list or postings associated with
/// a term in a `Segment`.
@@ -29,31 +32,6 @@ impl SegmentPostings {
}
}
/// Compute the number of non-deleted documents.
///
/// This method will clone and scan through the posting lists.
/// (this is a rather expensive operation).
pub fn doc_freq_given_deletes(&self, alive_bitset: &AliveBitSet) -> u32 {
let mut docset = self.clone();
let mut doc_freq = 0;
loop {
let doc = docset.doc();
if doc == TERMINATED {
return doc_freq;
}
if alive_bitset.is_alive(doc) {
doc_freq += 1u32;
}
docset.advance();
}
}
/// Returns the overall number of documents in the block postings.
/// It does not take in account whether documents are deleted or not.
pub fn doc_freq(&self) -> u32 {
self.block_cursor.doc_freq()
}
/// Creates a segment postings object with the given documents
/// and no frequency encoded.
///
@@ -64,11 +42,13 @@ impl SegmentPostings {
/// buffer with the serialized data.
#[cfg(test)]
pub fn create_from_docs(docs: &[u32]) -> SegmentPostings {
use crate::directory::FileSlice;
use crate::postings::serializer::PostingsSerializer;
use common::OwnedBytes;
use crate::schema::IndexRecordOption;
let mut buffer = Vec::new();
{
use crate::postings::serializer::PostingsSerializer;
let mut postings_serializer =
PostingsSerializer::new(0.0, IndexRecordOption::Basic, None);
postings_serializer.new_term(docs.len() as u32, false);
@@ -81,7 +61,7 @@ impl SegmentPostings {
}
let block_segment_postings = BlockSegmentPostings::open(
docs.len() as u32,
FileSlice::from(buffer),
OwnedBytes::new(buffer),
IndexRecordOption::Basic,
IndexRecordOption::Basic,
)
@@ -95,7 +75,8 @@ impl SegmentPostings {
doc_and_tfs: &[(u32, u32)],
fieldnorms: Option<&[u32]>,
) -> SegmentPostings {
use crate::directory::FileSlice;
use common::OwnedBytes;
use crate::fieldnorm::FieldNormReader;
use crate::postings::serializer::PostingsSerializer;
use crate::schema::IndexRecordOption;
@@ -128,7 +109,7 @@ impl SegmentPostings {
.unwrap();
let block_segment_postings = BlockSegmentPostings::open(
doc_and_tfs.len() as u32,
FileSlice::from(buffer),
OwnedBytes::new(buffer),
IndexRecordOption::WithFreqs,
IndexRecordOption::WithFreqs,
)
@@ -158,7 +139,6 @@ impl DocSet for SegmentPostings {
// next needs to be called a first time to point to the correct element.
#[inline]
fn advance(&mut self) -> DocId {
debug_assert!(self.block_cursor.block_is_loaded());
if self.cur == COMPRESSION_BLOCK_SIZE - 1 {
self.cur = 0;
self.block_cursor.advance();
@@ -168,12 +148,20 @@ impl DocSet for SegmentPostings {
self.doc()
}
#[inline]
fn seek(&mut self, target: DocId) -> DocId {
debug_assert!(self.doc() <= target);
if self.doc() >= target {
return self.doc();
}
// As an optimization, if the block is already loaded, we can
// cheaply check the next doc.
self.cur = (self.cur + 1).min(COMPRESSION_BLOCK_SIZE - 1);
if self.doc() >= target {
return self.doc();
}
// Delegate block-local search to BlockSegmentPostings::seek, which returns
// the in-block index of the first doc >= target.
self.cur = self.block_cursor.seek(target);
@@ -189,13 +177,31 @@ impl DocSet for SegmentPostings {
}
fn size_hint(&self) -> u32 {
self.len() as u32
self.doc_freq().into()
}
}
impl HasLen for SegmentPostings {
fn len(&self) -> usize {
self.block_cursor.doc_freq() as usize
fn fill_bitset(&mut self, bitset: &mut BitSet) {
let bitset_max_value: DocId = bitset.max_value();
loop {
let docs = self.block_cursor.docs();
let Some(&last_doc) = docs.last() else {
break;
};
if last_doc < bitset_max_value {
// All docs are within the range of the bitset
for &doc in docs {
bitset.insert(doc);
}
} else {
for &doc in docs {
if doc < bitset_max_value {
bitset.insert(doc);
}
}
break;
}
self.block_cursor.advance();
}
}
}
@@ -221,6 +227,13 @@ impl Postings for SegmentPostings {
self.block_cursor.freq(self.cur)
}
/// Returns the overall number of documents in the block postings.
/// It does not take in account whether documents are deleted or not.
#[inline(always)]
fn doc_freq(&self) -> DocFreq {
DocFreq::Exact(self.block_cursor.doc_freq())
}
fn append_positions_with_offset(&mut self, offset: u32, output: &mut Vec<u32>) {
let term_freq = self.term_freq();
let prev_len = output.len();
@@ -244,24 +257,44 @@ impl Postings for SegmentPostings {
}
}
}
fn has_freq(&self) -> bool {
!self.block_cursor.freqs().is_empty()
}
}
impl PostingsWithBlockMax for SegmentPostings {
#[inline]
fn seek_block_max(
&mut self,
target_doc: crate::DocId,
fieldnorm_reader: &FieldNormReader,
similarity_weight: &Bm25Weight,
) -> Score {
self.block_cursor.seek_block_without_loading(target_doc);
self.block_cursor
.block_max_score(fieldnorm_reader, similarity_weight)
}
#[inline]
fn last_doc_in_block(&self) -> crate::DocId {
self.block_cursor.skip_reader().last_doc_in_block()
}
}
#[cfg(test)]
mod tests {
use common::HasLen;
use super::SegmentPostings;
use crate::docset::{DocSet, TERMINATED};
use crate::fastfield::AliveBitSet;
use crate::postings::postings::Postings;
use crate::postings::Postings;
#[test]
fn test_empty_segment_postings() {
let mut postings = SegmentPostings::empty();
assert_eq!(postings.doc(), TERMINATED);
assert_eq!(postings.advance(), TERMINATED);
assert_eq!(postings.advance(), TERMINATED);
assert_eq!(postings.len(), 0);
assert_eq!(postings.doc_freq(), crate::postings::DocFreq::Exact(0));
}
#[test]
@@ -276,15 +309,4 @@ mod tests {
let postings = SegmentPostings::empty();
assert_eq!(postings.term_freq(), 1);
}
#[test]
fn test_doc_freq() {
let docs = SegmentPostings::create_from_docs(&[0, 2, 10]);
assert_eq!(docs.doc_freq(), 3);
let alive_bitset = AliveBitSet::for_test_from_deleted_docs(&[2], 12);
assert_eq!(docs.doc_freq_given_deletes(&alive_bitset), 2);
let all_deleted =
AliveBitSet::for_test_from_deleted_docs(&[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11], 12);
assert_eq!(docs.doc_freq_given_deletes(&all_deleted), 0);
}
}

View File

@@ -4,7 +4,7 @@ use common::{replace_in_place, JsonPathWriter};
use rustc_hash::FxHashMap;
use crate::indexer::indexing_term::IndexingTerm;
use crate::postings::{IndexingContext, IndexingPosition, PostingsWriter};
use crate::postings::{IndexingContext, IndexingPosition, PostingsWriter as _, PostingsWriterEnum};
use crate::schema::document::{ReferenceValue, ReferenceValueLeaf, Value};
use crate::schema::{Type, DATE_TIME_PRECISION_INDEXED};
use crate::time::format_description::well_known::Rfc3339;
@@ -80,7 +80,7 @@ fn index_json_object<'a, V: Value<'a>>(
text_analyzer: &mut TextAnalyzer,
term_buffer: &mut IndexingTerm,
json_path_writer: &mut JsonPathWriter,
postings_writer: &mut dyn PostingsWriter,
postings_writer: &mut PostingsWriterEnum,
ctx: &mut IndexingContext,
positions_per_path: &mut IndexingPositionsPerPath,
) {
@@ -110,7 +110,7 @@ pub(crate) fn index_json_value<'a, V: Value<'a>>(
text_analyzer: &mut TextAnalyzer,
term_buffer: &mut IndexingTerm,
json_path_writer: &mut JsonPathWriter,
postings_writer: &mut dyn PostingsWriter,
postings_writer: &mut PostingsWriterEnum,
ctx: &mut IndexingContext,
positions_per_path: &mut IndexingPositionsPerPath,
) {

View File

@@ -676,7 +676,7 @@ mod tests {
let num_segments = reader.searcher().segment_readers().len();
assert!(num_segments <= 4);
let num_components_except_deletes_and_tempstore =
crate::index::SegmentComponent::iterator().len() - 2;
crate::index::SegmentComponent::iterator().len() - 1;
let max_num_mmapped = num_components_except_deletes_and_tempstore * num_segments;
assert_eventually(|| {
let num_mmapped = mmap_directory.get_cache_info().mmapped.len();

View File

@@ -1,4 +1,7 @@
use std::borrow::{Borrow, BorrowMut};
use std::borrow::BorrowMut;
use std::ops::{Deref as _, DerefMut as _};
use common::BitSet;
use crate::fastfield::AliveBitSet;
use crate::DocId;
@@ -65,8 +68,8 @@ pub trait DocSet: Send {
/// `seek_danger(..)` until it returns `Found`, and get back to a valid state.
///
/// `seek_lower_bound` can be any `DocId` (in the docset or not) as long as it is in
/// `(target .. seek_result]` where `seek_result` is the first document in the docset greater
/// than to `target`.
/// `(target .. seek_result] U {TERMINATED}` where `seek_result` is the first document in the
/// docset greater than to `target`.
///
/// `seek_danger` may return `SeekLowerBound(TERMINATED)`.
///
@@ -98,7 +101,7 @@ pub trait DocSet: Send {
if doc == target {
SeekDangerResult::Found
} else {
SeekDangerResult::SeekLowerBound(self.doc())
SeekDangerResult::SeekLowerBound(doc)
}
}
@@ -130,6 +133,19 @@ pub trait DocSet: Send {
buffer.len()
}
/// Fills the given bitset with the documents in the docset.
///
/// If the docset max_doc is smaller than the largest doc, this function might not consume the
/// docset entirely.
fn fill_bitset(&mut self, bitset: &mut BitSet) {
let bitset_max_value: u32 = bitset.max_value();
let mut doc = self.doc();
while doc < bitset_max_value {
bitset.insert(doc);
doc = self.advance();
}
}
/// Returns the current document
/// Right after creating a new `DocSet`, the docset points to the first document.
///
@@ -233,51 +249,59 @@ impl DocSet for &mut dyn DocSet {
fn count_including_deleted(&mut self) -> u32 {
(**self).count_including_deleted()
}
fn fill_bitset(&mut self, bitset: &mut BitSet) {
(**self).fill_bitset(bitset);
}
}
impl<TDocSet: DocSet + ?Sized> DocSet for Box<TDocSet> {
#[inline]
fn advance(&mut self) -> DocId {
let unboxed: &mut TDocSet = self.borrow_mut();
unboxed.advance()
self.deref_mut().advance()
}
#[inline]
fn seek(&mut self, target: DocId) -> DocId {
let unboxed: &mut TDocSet = self.borrow_mut();
unboxed.seek(target)
self.deref_mut().seek(target)
}
#[inline]
fn seek_danger(&mut self, target: DocId) -> SeekDangerResult {
let unboxed: &mut TDocSet = self.borrow_mut();
unboxed.seek_danger(target)
}
#[inline]
fn fill_buffer(&mut self, buffer: &mut [DocId; COLLECT_BLOCK_BUFFER_LEN]) -> usize {
let unboxed: &mut TDocSet = self.borrow_mut();
unboxed.fill_buffer(buffer)
self.deref_mut().fill_buffer(buffer)
}
#[inline]
fn doc(&self) -> DocId {
let unboxed: &TDocSet = self.borrow();
unboxed.doc()
self.deref().doc()
}
#[inline]
fn size_hint(&self) -> u32 {
let unboxed: &TDocSet = self.borrow();
unboxed.size_hint()
self.deref().size_hint()
}
#[inline]
fn cost(&self) -> u64 {
let unboxed: &TDocSet = self.borrow();
unboxed.cost()
self.deref().cost()
}
#[inline]
fn count(&mut self, alive_bitset: &AliveBitSet) -> u32 {
let unboxed: &mut TDocSet = self.borrow_mut();
unboxed.count(alive_bitset)
self.deref_mut().count(alive_bitset)
}
fn count_including_deleted(&mut self) -> u32 {
let unboxed: &mut TDocSet = self.borrow_mut();
unboxed.count_including_deleted()
self.deref_mut().count_including_deleted()
}
fn fill_bitset(&mut self, bitset: &mut BitSet) {
self.deref_mut().fill_bitset(bitset);
}
}

View File

@@ -0,0 +1,49 @@
use std::borrow::Cow;
use serde::{Deserialize, Serialize};
use crate::codec::{Codec, StandardCodec};
/// A Codec configuration is just a serializable object.
#[derive(Serialize, Deserialize, Clone, Debug)]
pub struct CodecConfiguration {
codec_id: Cow<'static, str>,
#[serde(default, skip_serializing_if = "serde_json::Value::is_null")]
props: serde_json::Value,
}
impl CodecConfiguration {
/// Returns true if the codec is the standard codec.
pub fn is_standard(&self) -> bool {
self.codec_id == StandardCodec::ID && self.props.is_null()
}
/// Creates a codec instance from the configuration.
///
/// If the codec id does not match the code's name, an error is returned.
pub fn to_codec<C: Codec>(&self) -> crate::Result<C> {
if self.codec_id != C::ID {
return Err(crate::TantivyError::InvalidArgument(format!(
"Codec id mismatch: expected {}, got {}",
C::ID,
self.codec_id
)));
}
C::from_json_props(&self.props)
}
}
impl<'a, C: Codec> From<&'a C> for CodecConfiguration {
fn from(codec: &'a C) -> Self {
CodecConfiguration {
codec_id: Cow::Borrowed(C::ID),
props: codec.to_json_props(),
}
}
}
impl Default for CodecConfiguration {
fn default() -> Self {
CodecConfiguration::from(&StandardCodec)
}
}

View File

@@ -8,12 +8,14 @@ use std::thread::available_parallelism;
use super::segment::Segment;
use super::segment_reader::merge_field_meta_data;
use super::{FieldMetadata, IndexSettings};
use crate::codec::StandardCodec;
use crate::core::{Executor, META_FILEPATH};
use crate::directory::error::OpenReadError;
#[cfg(feature = "mmap")]
use crate::directory::MmapDirectory;
use crate::directory::{Directory, ManagedDirectory, RamDirectory, INDEX_WRITER_LOCK};
use crate::error::{DataCorruption, TantivyError};
use crate::index::codec_configuration::CodecConfiguration;
use crate::index::{IndexMeta, SegmentId, SegmentMeta, SegmentMetaInventory};
use crate::indexer::index_writer::{
IndexWriterOptions, MAX_NUM_THREAD, MEMORY_BUDGET_NUM_BYTES_MIN,
@@ -59,6 +61,7 @@ fn save_new_metas(
schema: Schema,
index_settings: IndexSettings,
directory: &dyn Directory,
codec: CodecConfiguration,
) -> crate::Result<()> {
save_metas(
&IndexMeta {
@@ -67,6 +70,7 @@ fn save_new_metas(
schema,
opstamp: 0u64,
payload: None,
codec,
},
directory,
)?;
@@ -101,18 +105,21 @@ fn save_new_metas(
/// };
/// let index = Index::builder().schema(schema).settings(settings).create_in_ram();
/// ```
pub struct IndexBuilder {
pub struct IndexBuilder<Codec: crate::codec::Codec = StandardCodec> {
schema: Option<Schema>,
index_settings: IndexSettings,
tokenizer_manager: TokenizerManager,
fast_field_tokenizer_manager: TokenizerManager,
codec: Codec,
}
impl Default for IndexBuilder {
impl Default for IndexBuilder<StandardCodec> {
fn default() -> Self {
IndexBuilder::new()
}
}
impl IndexBuilder {
impl IndexBuilder<StandardCodec> {
/// Creates a new `IndexBuilder`
pub fn new() -> Self {
Self {
@@ -120,6 +127,21 @@ impl IndexBuilder {
index_settings: IndexSettings::default(),
tokenizer_manager: TokenizerManager::default(),
fast_field_tokenizer_manager: TokenizerManager::default(),
codec: StandardCodec,
}
}
}
impl<Codec: crate::codec::Codec> IndexBuilder<Codec> {
/// Set the codec
#[must_use]
pub fn codec<NewCodec: crate::codec::Codec>(self, codec: NewCodec) -> IndexBuilder<NewCodec> {
IndexBuilder {
schema: self.schema,
index_settings: self.index_settings,
tokenizer_manager: self.tokenizer_manager,
fast_field_tokenizer_manager: self.fast_field_tokenizer_manager,
codec,
}
}
@@ -154,7 +176,7 @@ impl IndexBuilder {
/// The index will be allocated in anonymous memory.
/// This is useful for indexing small set of documents
/// for instances like unit test or temporary in memory index.
pub fn create_in_ram(self) -> Result<Index, TantivyError> {
pub fn create_in_ram(self) -> Result<Index<Codec>, TantivyError> {
let ram_directory = RamDirectory::create();
self.create(ram_directory)
}
@@ -165,7 +187,7 @@ impl IndexBuilder {
/// If a previous index was in this directory, it returns an
/// [`TantivyError::IndexAlreadyExists`] error.
#[cfg(feature = "mmap")]
pub fn create_in_dir<P: AsRef<Path>>(self, directory_path: P) -> crate::Result<Index> {
pub fn create_in_dir<P: AsRef<Path>>(self, directory_path: P) -> crate::Result<Index<Codec>> {
let mmap_directory: Box<dyn Directory> = Box::new(MmapDirectory::open(directory_path)?);
if Index::exists(&*mmap_directory)? {
return Err(TantivyError::IndexAlreadyExists);
@@ -186,7 +208,7 @@ impl IndexBuilder {
self,
dir: impl Into<Box<dyn Directory>>,
mem_budget: usize,
) -> crate::Result<SingleSegmentIndexWriter<D>> {
) -> crate::Result<SingleSegmentIndexWriter<Codec, D>> {
let index = self.create(dir)?;
let index_simple_writer = SingleSegmentIndexWriter::new(index, mem_budget)?;
Ok(index_simple_writer)
@@ -202,7 +224,7 @@ impl IndexBuilder {
/// For other unit tests, prefer the [`RamDirectory`], see:
/// [`IndexBuilder::create_in_ram()`].
#[cfg(feature = "mmap")]
pub fn create_from_tempdir(self) -> crate::Result<Index> {
pub fn create_from_tempdir(self) -> crate::Result<Index<Codec>> {
let mmap_directory: Box<dyn Directory> = Box::new(MmapDirectory::create_from_tempdir()?);
self.create(mmap_directory)
}
@@ -215,12 +237,15 @@ impl IndexBuilder {
}
/// Opens or creates a new index in the provided directory
pub fn open_or_create<T: Into<Box<dyn Directory>>>(self, dir: T) -> crate::Result<Index> {
pub fn open_or_create<T: Into<Box<dyn Directory>>>(
self,
dir: T,
) -> crate::Result<Index<Codec>> {
let dir: Box<dyn Directory> = dir.into();
if !Index::exists(&*dir)? {
return self.create(dir);
}
let mut index = Index::open(dir)?;
let mut index: Index<Codec> = Index::<Codec>::open_with_codec(dir)?;
index.set_tokenizers(self.tokenizer_manager.clone());
if index.schema() == self.get_expect_schema()? {
Ok(index)
@@ -244,18 +269,25 @@ impl IndexBuilder {
/// Creates a new index given an implementation of the trait `Directory`.
///
/// If a directory previously existed, it will be erased.
fn create<T: Into<Box<dyn Directory>>>(self, dir: T) -> crate::Result<Index> {
pub fn create<T: Into<Box<dyn Directory>>>(self, dir: T) -> crate::Result<Index<Codec>> {
self.create_avoid_monomorphization(dir.into())
}
fn create_avoid_monomorphization(self, dir: Box<dyn Directory>) -> crate::Result<Index<Codec>> {
self.validate()?;
let dir = dir.into();
let directory = ManagedDirectory::wrap(dir)?;
let codec: CodecConfiguration = CodecConfiguration::from(&self.codec);
save_new_metas(
self.get_expect_schema()?,
self.index_settings.clone(),
&directory,
codec,
)?;
let mut metas = IndexMeta::with_schema(self.get_expect_schema()?);
let schema = self.get_expect_schema()?;
let mut metas = IndexMeta::with_schema_and_codec(schema, &self.codec);
metas.index_settings = self.index_settings;
let mut index = Index::open_from_metas(directory, &metas, SegmentMetaInventory::default());
let mut index: Index<Codec> =
Index::<Codec>::open_from_metas(directory, &metas, SegmentMetaInventory::default())?;
index.set_tokenizers(self.tokenizer_manager);
index.set_fast_field_tokenizers(self.fast_field_tokenizer_manager);
Ok(index)
@@ -264,7 +296,7 @@ impl IndexBuilder {
/// Search Index
#[derive(Clone)]
pub struct Index {
pub struct Index<Codec: crate::codec::Codec = crate::codec::StandardCodec> {
directory: ManagedDirectory,
schema: Schema,
settings: IndexSettings,
@@ -272,6 +304,7 @@ pub struct Index {
tokenizers: TokenizerManager,
fast_field_tokenizers: TokenizerManager,
inventory: SegmentMetaInventory,
codec: Codec,
}
impl Index {
@@ -279,41 +312,6 @@ impl Index {
pub fn builder() -> IndexBuilder {
IndexBuilder::new()
}
/// Examines the directory to see if it contains an index.
///
/// Effectively, it only checks for the presence of the `meta.json` file.
pub fn exists(dir: &dyn Directory) -> Result<bool, OpenReadError> {
dir.exists(&META_FILEPATH)
}
/// Accessor to the search executor.
///
/// This pool is used by default when calling `searcher.search(...)`
/// to perform search on the individual segments.
///
/// By default the executor is single thread, and simply runs in the calling thread.
pub fn search_executor(&self) -> &Executor {
&self.executor
}
/// Replace the default single thread search executor pool
/// by a thread pool with a given number of threads.
pub fn set_multithread_executor(&mut self, num_threads: usize) -> crate::Result<()> {
self.executor = Executor::multi_thread(num_threads, "tantivy-search-")?;
Ok(())
}
/// Custom thread pool by a outer thread pool.
pub fn set_executor(&mut self, executor: Executor) {
self.executor = executor;
}
/// Replace the default single thread search executor pool
/// by a thread pool with as many threads as there are CPUs on the system.
pub fn set_default_multithread_executor(&mut self) -> crate::Result<()> {
let default_num_threads = available_parallelism()?.get();
self.set_multithread_executor(default_num_threads)
}
/// Creates a new index using the [`RamDirectory`].
///
@@ -324,6 +322,13 @@ impl Index {
IndexBuilder::new().schema(schema).create_in_ram().unwrap()
}
/// Examines the directory to see if it contains an index.
///
/// Effectively, it only checks for the presence of the `meta.json` file.
pub fn exists(directory: &dyn Directory) -> Result<bool, OpenReadError> {
directory.exists(&META_FILEPATH)
}
/// Creates a new index in a given filepath.
/// The index will use the [`MmapDirectory`].
///
@@ -370,20 +375,108 @@ impl Index {
schema: Schema,
settings: IndexSettings,
) -> crate::Result<Index> {
let dir: Box<dyn Directory> = dir.into();
Self::create_to_avoid_monomorphization(dir.into(), schema, settings)
}
fn create_to_avoid_monomorphization(
dir: Box<dyn Directory>,
schema: Schema,
settings: IndexSettings,
) -> crate::Result<Index> {
let mut builder = IndexBuilder::new().schema(schema);
builder = builder.settings(settings);
builder.create(dir)
}
/// Opens a new directory from an index path.
#[cfg(feature = "mmap")]
pub fn open_in_dir<P: AsRef<Path>>(directory_path: P) -> crate::Result<Index> {
Self::open_in_dir_to_avoid_monomorphization(directory_path.as_ref())
}
#[cfg(feature = "mmap")]
#[inline(never)]
fn open_in_dir_to_avoid_monomorphization(directory_path: &Path) -> crate::Result<Index> {
let mmap_directory = MmapDirectory::open(directory_path)?;
Index::open(mmap_directory)
}
/// Open the index using the provided directory
pub fn open<T: Into<Box<dyn Directory>>>(directory: T) -> crate::Result<Index> {
Index::<StandardCodec>::open_with_codec(directory.into())
}
}
impl<Codec: crate::codec::Codec> Index<Codec> {
/// Returns a version of this index with the standard codec.
/// This is useful when you need to pass the index to APIs that
/// don't care about the codec (e.g., for reading).
pub(crate) fn with_standard_codec(&self) -> Index<StandardCodec> {
Index {
directory: self.directory.clone(),
schema: self.schema.clone(),
settings: self.settings.clone(),
executor: self.executor.clone(),
tokenizers: self.tokenizers.clone(),
fast_field_tokenizers: self.fast_field_tokenizers.clone(),
inventory: self.inventory.clone(),
codec: StandardCodec,
}
}
/// Open the index using the provided directory
#[inline(never)]
pub fn open_with_codec(directory: Box<dyn Directory>) -> crate::Result<Index<Codec>> {
let directory = ManagedDirectory::wrap(directory)?;
let inventory = SegmentMetaInventory::default();
let metas = load_metas(&directory, &inventory)?;
let index: Index<Codec> = Index::<Codec>::open_from_metas(directory, &metas, inventory)?;
Ok(index)
}
/// Accessor to the codec.
pub fn codec(&self) -> &Codec {
&self.codec
}
/// Accessor to the search executor.
///
/// This pool is used by default when calling `searcher.search(...)`
/// to perform search on the individual segments.
///
/// By default the executor is single thread, and simply runs in the calling thread.
pub fn search_executor(&self) -> &Executor {
&self.executor
}
/// Replace the default single thread search executor pool
/// by a thread pool with a given number of threads.
pub fn set_multithread_executor(&mut self, num_threads: usize) -> crate::Result<()> {
self.executor = Executor::multi_thread(num_threads, "tantivy-search-")?;
Ok(())
}
/// Custom thread pool by a outer thread pool.
pub fn set_executor(&mut self, executor: Executor) {
self.executor = executor;
}
/// Replace the default single thread search executor pool
/// by a thread pool with as many threads as there are CPUs on the system.
pub fn set_default_multithread_executor(&mut self) -> crate::Result<()> {
let default_num_threads = available_parallelism()?.get();
self.set_multithread_executor(default_num_threads)
}
/// Creates a new index given a directory and an [`IndexMeta`].
fn open_from_metas(
fn open_from_metas<C: crate::codec::Codec>(
directory: ManagedDirectory,
metas: &IndexMeta,
inventory: SegmentMetaInventory,
) -> Index {
) -> crate::Result<Index<C>> {
let schema = metas.schema.clone();
Index {
let codec = metas.codec.to_codec::<C>()?;
Ok(Index {
settings: metas.index_settings.clone(),
directory,
schema,
@@ -391,7 +484,8 @@ impl Index {
fast_field_tokenizers: TokenizerManager::default(),
executor: Executor::single_thread(),
inventory,
}
codec,
})
}
/// Setter for the tokenizer manager.
@@ -447,7 +541,7 @@ impl Index {
/// Create a default [`IndexReader`] for the given index.
///
/// See [`Index.reader_builder()`].
pub fn reader(&self) -> crate::Result<IndexReader> {
pub fn reader(&self) -> crate::Result<IndexReader<Codec>> {
self.reader_builder().try_into()
}
@@ -455,17 +549,10 @@ impl Index {
///
/// Most project should create at most one reader for a given index.
/// This method is typically called only once per `Index` instance.
pub fn reader_builder(&self) -> IndexReaderBuilder {
pub fn reader_builder(&self) -> IndexReaderBuilder<Codec> {
IndexReaderBuilder::new(self.clone())
}
/// Opens a new directory from an index path.
#[cfg(feature = "mmap")]
pub fn open_in_dir<P: AsRef<Path>>(directory_path: P) -> crate::Result<Index> {
let mmap_directory = MmapDirectory::open(directory_path)?;
Index::open(mmap_directory)
}
/// Returns the list of the segment metas tracked by the index.
///
/// Such segments can of course be part of the index,
@@ -506,16 +593,6 @@ impl Index {
self.inventory.new_segment_meta(segment_id, max_doc)
}
/// Open the index using the provided directory
pub fn open<T: Into<Box<dyn Directory>>>(directory: T) -> crate::Result<Index> {
let directory = directory.into();
let directory = ManagedDirectory::wrap(directory)?;
let inventory = SegmentMetaInventory::default();
let metas = load_metas(&directory, &inventory)?;
let index = Index::open_from_metas(directory, &metas, inventory);
Ok(index)
}
/// Reads the index meta file from the directory.
pub fn load_metas(&self) -> crate::Result<IndexMeta> {
load_metas(self.directory(), &self.inventory)
@@ -539,7 +616,7 @@ impl Index {
pub fn writer_with_options<D: Document>(
&self,
options: IndexWriterOptions,
) -> crate::Result<IndexWriter<D>> {
) -> crate::Result<IndexWriter<Codec, D>> {
let directory_lock = self
.directory
.acquire_lock(&INDEX_WRITER_LOCK)
@@ -581,7 +658,7 @@ impl Index {
&self,
num_threads: usize,
overall_memory_budget_in_bytes: usize,
) -> crate::Result<IndexWriter<D>> {
) -> crate::Result<IndexWriter<Codec, D>> {
let memory_arena_in_bytes_per_thread = overall_memory_budget_in_bytes / num_threads;
let options = IndexWriterOptions::builder()
.num_worker_threads(num_threads)
@@ -595,7 +672,7 @@ impl Index {
/// That index writer only simply has a single thread and a memory budget of 15 MB.
/// Using a single thread gives us a deterministic allocation of DocId.
#[cfg(test)]
pub fn writer_for_tests<D: Document>(&self) -> crate::Result<IndexWriter<D>> {
pub fn writer_for_tests<D: Document>(&self) -> crate::Result<IndexWriter<Codec, D>> {
self.writer_with_num_threads(1, MEMORY_BUDGET_NUM_BYTES_MIN)
}
@@ -613,7 +690,7 @@ impl Index {
pub fn writer<D: Document>(
&self,
memory_budget_in_bytes: usize,
) -> crate::Result<IndexWriter<D>> {
) -> crate::Result<IndexWriter<Codec, D>> {
let mut num_threads = std::cmp::min(available_parallelism()?.get(), MAX_NUM_THREAD);
let memory_budget_num_bytes_per_thread = memory_budget_in_bytes / num_threads;
if memory_budget_num_bytes_per_thread < MEMORY_BUDGET_NUM_BYTES_MIN {
@@ -640,7 +717,7 @@ impl Index {
}
/// Returns the list of segments that are searchable
pub fn searchable_segments(&self) -> crate::Result<Vec<Segment>> {
pub fn searchable_segments(&self) -> crate::Result<Vec<Segment<Codec>>> {
Ok(self
.searchable_segment_metas()?
.into_iter()
@@ -649,12 +726,12 @@ impl Index {
}
#[doc(hidden)]
pub fn segment(&self, segment_meta: SegmentMeta) -> Segment {
pub fn segment(&self, segment_meta: SegmentMeta) -> Segment<Codec> {
Segment::for_index(self.clone(), segment_meta)
}
/// Creates a new segment.
pub fn new_segment(&self) -> Segment {
pub fn new_segment(&self) -> Segment<Codec> {
let segment_meta = self
.inventory
.new_segment_meta(SegmentId::generate_random(), 0);
@@ -708,7 +785,7 @@ impl Index {
}
impl fmt::Debug for Index {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "Index({:?})", self.directory)
}
}

View File

@@ -1,13 +1,12 @@
use std::collections::HashSet;
use std::fmt;
use std::path::PathBuf;
use std::sync::atomic::AtomicBool;
use std::sync::Arc;
use serde::{Deserialize, Serialize};
use super::SegmentComponent;
use crate::index::SegmentId;
use crate::codec::Codec;
use crate::index::{CodecConfiguration, SegmentId};
use crate::schema::Schema;
use crate::store::Compressor;
use crate::{Inventory, Opstamp, TrackedObject};
@@ -37,7 +36,6 @@ impl SegmentMetaInventory {
let inner = InnerSegmentMeta {
segment_id,
max_doc,
include_temp_doc_store: Arc::new(AtomicBool::new(true)),
deletes: None,
};
SegmentMeta::from(self.inventory.track(inner))
@@ -85,15 +83,6 @@ impl SegmentMeta {
self.tracked.segment_id
}
/// Removes the Component::TempStore from the alive list and
/// therefore marks the temp docstore file to be deleted by
/// the garbage collection.
pub fn untrack_temp_docstore(&self) {
self.tracked
.include_temp_doc_store
.store(false, std::sync::atomic::Ordering::Relaxed);
}
/// Returns the number of deleted documents.
pub fn num_deleted_docs(&self) -> u32 {
self.tracked
@@ -111,20 +100,9 @@ impl SegmentMeta {
/// is by removing all files that have been created by tantivy
/// and are not used by any segment anymore.
pub fn list_files(&self) -> HashSet<PathBuf> {
if self
.tracked
.include_temp_doc_store
.load(std::sync::atomic::Ordering::Relaxed)
{
SegmentComponent::iterator()
.map(|component| self.relative_path(*component))
.collect::<HashSet<PathBuf>>()
} else {
SegmentComponent::iterator()
.filter(|comp| *comp != &SegmentComponent::TempStore)
.map(|component| self.relative_path(*component))
.collect::<HashSet<PathBuf>>()
}
SegmentComponent::iterator()
.map(|component| self.relative_path(*component))
.collect::<HashSet<PathBuf>>()
}
/// Returns the relative path of a component of our segment.
@@ -138,7 +116,6 @@ impl SegmentMeta {
SegmentComponent::Positions => ".pos".to_string(),
SegmentComponent::Terms => ".term".to_string(),
SegmentComponent::Store => ".store".to_string(),
SegmentComponent::TempStore => ".store.temp".to_string(),
SegmentComponent::FastFields => ".fast".to_string(),
SegmentComponent::FieldNorms => ".fieldnorm".to_string(),
SegmentComponent::Delete => format!(".{}.del", self.delete_opstamp().unwrap_or(0)),
@@ -183,7 +160,6 @@ impl SegmentMeta {
segment_id: inner_meta.segment_id,
max_doc,
deletes: None,
include_temp_doc_store: Arc::new(AtomicBool::new(true)),
});
SegmentMeta { tracked }
}
@@ -202,7 +178,6 @@ impl SegmentMeta {
let tracked = self.tracked.map(move |inner_meta| InnerSegmentMeta {
segment_id: inner_meta.segment_id,
max_doc: inner_meta.max_doc,
include_temp_doc_store: Arc::new(AtomicBool::new(true)),
deletes: Some(delete_meta),
});
SegmentMeta { tracked }
@@ -214,14 +189,6 @@ struct InnerSegmentMeta {
segment_id: SegmentId,
max_doc: u32,
pub deletes: Option<DeleteMeta>,
/// If you want to avoid the SegmentComponent::TempStore file to be covered by
/// garbage collection and deleted, set this to true. This is used during merge.
#[serde(skip)]
#[serde(default = "default_temp_store")]
pub(crate) include_temp_doc_store: Arc<AtomicBool>,
}
fn default_temp_store() -> Arc<AtomicBool> {
Arc::new(AtomicBool::new(false))
}
impl InnerSegmentMeta {
@@ -320,8 +287,10 @@ pub struct IndexMeta {
/// This payload is entirely unused by tantivy.
#[serde(skip_serializing_if = "Option::is_none")]
pub payload: Option<String>,
/// Codec configuration for the index.
#[serde(skip_serializing_if = "CodecConfiguration::is_standard")]
pub codec: CodecConfiguration,
}
#[derive(Deserialize, Debug)]
struct UntrackedIndexMeta {
pub segments: Vec<InnerSegmentMeta>,
@@ -331,6 +300,8 @@ struct UntrackedIndexMeta {
pub opstamp: Opstamp,
#[serde(skip_serializing_if = "Option::is_none")]
pub payload: Option<String>,
#[serde(default)]
pub codec: CodecConfiguration,
}
impl UntrackedIndexMeta {
@@ -345,6 +316,7 @@ impl UntrackedIndexMeta {
schema: self.schema,
opstamp: self.opstamp,
payload: self.payload,
codec: self.codec,
}
}
}
@@ -355,13 +327,14 @@ impl IndexMeta {
///
/// This new index does not contains any segments.
/// Opstamp will the value `0u64`.
pub fn with_schema(schema: Schema) -> IndexMeta {
pub fn with_schema_and_codec<C: Codec>(schema: Schema, codec: &C) -> IndexMeta {
IndexMeta {
index_settings: IndexSettings::default(),
segments: vec![],
schema,
opstamp: 0u64,
payload: None,
codec: CodecConfiguration::from(codec),
}
}
@@ -412,14 +385,38 @@ mod tests {
schema,
opstamp: 0u64,
payload: None,
codec: Default::default(),
};
let json = serde_json::ser::to_string(&index_metas).expect("serialization failed");
let json_value: serde_json::Value =
serde_json::to_value(&index_metas).expect("serialization failed");
assert_eq!(
json,
r#"{"index_settings":{"docstore_compression":"none","docstore_blocksize":16384},"segments":[],"schema":[{"name":"text","type":"text","options":{"indexing":{"record":"position","fieldnorms":true,"tokenizer":"default"},"stored":false,"fast":false}}],"opstamp":0}"#
&json_value,
&serde_json::json!(
{
"index_settings": {
"docstore_compression": "none",
"docstore_blocksize": 16384
},
"segments": [],
"schema": [
{
"name": "text",
"type": "text",
"options": {
"indexing": {
"record": "position",
"fieldnorms": true,
"tokenizer": "default"
},
"stored": false,
"fast": false
}
}
],
"opstamp": 0
})
);
let deser_meta: UntrackedIndexMeta = serde_json::from_str(&json).unwrap();
let deser_meta: UntrackedIndexMeta = serde_json::from_value(json_value).unwrap();
assert_eq!(index_metas.index_settings, deser_meta.index_settings);
assert_eq!(index_metas.schema, deser_meta.schema);
assert_eq!(index_metas.opstamp, deser_meta.opstamp);
@@ -445,14 +442,39 @@ mod tests {
schema,
opstamp: 0u64,
payload: None,
codec: Default::default(),
};
let json = serde_json::ser::to_string(&index_metas).expect("serialization failed");
let json_value = serde_json::to_value(&index_metas).expect("serialization failed");
assert_eq!(
json,
r#"{"index_settings":{"docstore_compression":"zstd(compression_level=4)","docstore_blocksize":1000000},"segments":[],"schema":[{"name":"text","type":"text","options":{"indexing":{"record":"position","fieldnorms":true,"tokenizer":"default"},"stored":false,"fast":false}}],"opstamp":0}"#
&json_value,
&serde_json::json!(
{
"index_settings": {
"docstore_compression": "zstd(compression_level=4)",
"docstore_blocksize": 1000000
},
"segments": [],
"schema": [
{
"name": "text",
"type": "text",
"options": {
"indexing": {
"record": "position",
"fieldnorms": true,
"tokenizer": "default"
},
"stored": false,
"fast": false
}
}
],
"opstamp": 0
}
)
);
let deser_meta: UntrackedIndexMeta = serde_json::from_str(&json).unwrap();
let deser_meta: UntrackedIndexMeta = serde_json::from_value(json_value).unwrap();
assert_eq!(index_metas.index_settings, deser_meta.index_settings);
assert_eq!(index_metas.schema, deser_meta.schema);
assert_eq!(index_metas.opstamp, deser_meta.opstamp);

View File

@@ -1,7 +1,12 @@
#[cfg(feature = "quickwit")]
use std::future::Future;
use std::io;
#[cfg(feature = "quickwit")]
use std::pin::Pin;
use std::sync::Arc;
use common::json_path_writer::JSON_END_OF_PATH;
use common::{BinarySerializable, ByteCount};
use common::{BinarySerializable, ByteCount, OwnedBytes};
#[cfg(feature = "quickwit")]
use futures_util::{FutureExt, StreamExt, TryStreamExt};
#[cfg(feature = "quickwit")]
@@ -9,38 +14,123 @@ use itertools::Itertools;
#[cfg(feature = "quickwit")]
use tantivy_fst::automaton::{AlwaysMatch, Automaton};
use crate::codec::{ObjectSafeCodec, StandardCodec};
use crate::directory::FileSlice;
use crate::positions::PositionReader;
use crate::postings::{BlockSegmentPostings, SegmentPostings, TermInfo};
use crate::fieldnorm::FieldNormReader;
use crate::postings::{Postings, TermInfo};
use crate::query::{Bm25Weight, Scorer};
use crate::schema::{IndexRecordOption, Term, Type};
use crate::termdict::TermDictionary;
/// Postings data returned by [`InvertedIndexReader::read_postings_data`].
pub struct PostingsData {
/// Raw postings bytes for the term.
pub postings_data: OwnedBytes,
/// Raw positions bytes for the term, if positions are available.
pub positions_data: Option<OwnedBytes>,
/// Record option of the indexed field.
pub record_option: IndexRecordOption,
/// Effective record option after downgrading to the indexed field capability.
pub effective_option: IndexRecordOption,
}
/// Trait defining the contract for inverted index readers.
pub trait InvertedIndexReader: Send + Sync {
/// Returns the term info associated with the term.
fn get_term_info(&self, term: &Term) -> io::Result<Option<TermInfo>>;
/// Return the term dictionary datastructure.
fn terms(&self) -> &TermDictionary;
/// Return the fields and types encoded in the dictionary in lexicographic order.
/// Only valid on JSON fields.
///
/// Notice: This requires a full scan and therefore **very expensive**.
fn list_encoded_json_fields(&self) -> io::Result<Vec<InvertedIndexFieldSpace>>;
/// Read postings bytes and metadata for a given term.
fn read_postings_data(
&self,
term_info: &TermInfo,
option: IndexRecordOption,
) -> io::Result<PostingsData>;
/// Build a new term scorer.
fn new_term_scorer(
&self,
term_info: &TermInfo,
option: IndexRecordOption,
fieldnorm_reader: FieldNormReader,
similarity_weight: Bm25Weight,
) -> io::Result<Box<dyn Scorer>>;
/// Returns a posting object given a `term_info`.
/// This method is for an advanced usage only.
///
/// Most users should prefer using [`Self::read_postings()`] instead.
fn read_postings_from_terminfo(
&self,
term_info: &TermInfo,
option: IndexRecordOption,
) -> io::Result<Box<dyn Postings>>;
/// Returns the total number of tokens recorded for all documents
/// (including deleted documents).
fn total_num_tokens(&self) -> u64;
/// Returns the segment postings associated with the term, and with the given option,
/// or `None` if the term has never been encountered and indexed.
fn read_postings(
&self,
term: &Term,
option: IndexRecordOption,
) -> io::Result<Option<Box<dyn Postings>>>;
/// Returns the number of documents containing the term.
fn doc_freq(&self, term: &Term) -> io::Result<u32>;
/// Returns the number of documents containing the term asynchronously.
#[cfg(feature = "quickwit")]
fn doc_freq_async<'a>(
&'a self,
term: &'a Term,
) -> Pin<Box<dyn Future<Output = io::Result<u32>> + Send + 'a>>;
}
/// Tantivy's default inverted index reader implementation.
///
/// The inverted index reader is in charge of accessing
/// the inverted index associated with a specific field.
///
/// # Note
///
/// It is safe to delete the segment associated with
/// an `InvertedIndexReader`. As long as it is open,
/// an `InvertedIndexReader` implementation. As long as it is open,
/// the [`FileSlice`] it is relying on should
/// stay available.
///
/// `InvertedIndexReader` are created by calling
/// `TantivyInvertedIndexReader` instances are created by calling
/// [`SegmentReader::inverted_index()`](crate::SegmentReader::inverted_index).
pub struct InvertedIndexReader {
pub struct TantivyInvertedIndexReader {
termdict: TermDictionary,
postings_file_slice: FileSlice,
positions_file_slice: FileSlice,
record_option: IndexRecordOption,
total_num_tokens: u64,
codec: Arc<dyn ObjectSafeCodec>,
}
/// Object that records the amount of space used by a field in an inverted index.
pub(crate) struct InvertedIndexFieldSpace {
pub struct InvertedIndexFieldSpace {
/// Field name as encoded in the term dictionary.
pub field_name: String,
/// Value type for the encoded field.
pub field_type: Type,
/// Total bytes used by postings for this field.
pub postings_size: ByteCount,
/// Total bytes used by positions for this field.
pub positions_size: ByteCount,
/// Number of terms in the field.
pub num_terms: u64,
}
@@ -62,52 +152,50 @@ impl InvertedIndexFieldSpace {
}
}
impl InvertedIndexReader {
impl TantivyInvertedIndexReader {
pub(crate) fn new(
termdict: TermDictionary,
postings_file_slice: FileSlice,
positions_file_slice: FileSlice,
record_option: IndexRecordOption,
) -> io::Result<InvertedIndexReader> {
codec: Arc<dyn ObjectSafeCodec>,
) -> io::Result<TantivyInvertedIndexReader> {
let (total_num_tokens_slice, postings_body) = postings_file_slice.split(8);
let total_num_tokens = u64::deserialize(&mut total_num_tokens_slice.read_bytes()?)?;
Ok(InvertedIndexReader {
Ok(TantivyInvertedIndexReader {
termdict,
postings_file_slice: postings_body,
positions_file_slice,
record_option,
total_num_tokens,
codec,
})
}
/// Creates an empty `InvertedIndexReader` object, which
/// Creates an empty `TantivyInvertedIndexReader` object, which
/// contains no terms at all.
pub fn empty(record_option: IndexRecordOption) -> InvertedIndexReader {
InvertedIndexReader {
pub fn empty(record_option: IndexRecordOption) -> TantivyInvertedIndexReader {
TantivyInvertedIndexReader {
termdict: TermDictionary::empty(),
postings_file_slice: FileSlice::empty(),
positions_file_slice: FileSlice::empty(),
record_option,
total_num_tokens: 0u64,
codec: Arc::new(StandardCodec),
}
}
}
/// Returns the term info associated with the term.
pub fn get_term_info(&self, term: &Term) -> io::Result<Option<TermInfo>> {
impl InvertedIndexReader for TantivyInvertedIndexReader {
fn get_term_info(&self, term: &Term) -> io::Result<Option<TermInfo>> {
self.termdict.get(term.serialized_value_bytes())
}
/// Return the term dictionary datastructure.
pub fn terms(&self) -> &TermDictionary {
fn terms(&self) -> &TermDictionary {
&self.termdict
}
/// Return the fields and types encoded in the dictionary in lexicographic order.
/// Only valid on JSON fields.
///
/// Notice: This requires a full scan and therefore **very expensive**.
/// TODO: Move to sstable to use the index.
pub(crate) fn list_encoded_json_fields(&self) -> io::Result<Vec<InvertedIndexFieldSpace>> {
fn list_encoded_json_fields(&self) -> io::Result<Vec<InvertedIndexFieldSpace>> {
let mut stream = self.termdict.stream()?;
let mut fields: Vec<InvertedIndexFieldSpace> = Vec::new();
@@ -160,129 +248,97 @@ impl InvertedIndexReader {
Ok(fields)
}
/// Resets the block segment to another position of the postings
/// file.
///
/// This is useful for enumerating through a list of terms,
/// and consuming the associated posting lists while avoiding
/// reallocating a [`BlockSegmentPostings`].
///
/// # Warning
///
/// This does not reset the positions list.
pub fn reset_block_postings_from_terminfo(
fn read_postings_data(
&self,
term_info: &TermInfo,
block_postings: &mut BlockSegmentPostings,
) -> io::Result<()> {
let postings_slice = self
.postings_file_slice
.slice(term_info.postings_range.clone());
let postings_bytes = postings_slice.read_bytes()?;
block_postings.reset(term_info.doc_freq, postings_bytes)?;
Ok(())
}
/// Returns a block postings given a `Term`.
/// This method is for an advanced usage only.
///
/// Most users should prefer using [`Self::read_postings()`] instead.
pub fn read_block_postings(
&self,
term: &Term,
option: IndexRecordOption,
) -> io::Result<Option<BlockSegmentPostings>> {
self.get_term_info(term)?
.map(move |term_info| self.read_block_postings_from_terminfo(&term_info, option))
.transpose()
}
/// Returns a block postings given a `term_info`.
/// This method is for an advanced usage only.
///
/// Most users should prefer using [`Self::read_postings()`] instead.
pub fn read_block_postings_from_terminfo(
&self,
term_info: &TermInfo,
requested_option: IndexRecordOption,
) -> io::Result<BlockSegmentPostings> {
) -> io::Result<PostingsData> {
let effective_option = option.downgrade(self.record_option);
let postings_data = self
.postings_file_slice
.slice(term_info.postings_range.clone());
BlockSegmentPostings::open(
term_info.doc_freq,
.slice(term_info.postings_range.clone())
.read_bytes()?;
let positions_data: Option<OwnedBytes> = if effective_option.has_positions() {
let positions_data = self
.positions_file_slice
.slice(term_info.positions_range.clone())
.read_bytes()?;
Some(positions_data)
} else {
None
};
Ok(PostingsData {
postings_data,
self.record_option,
requested_option,
)
positions_data,
record_option: self.record_option,
effective_option,
})
}
/// Returns a posting object given a `term_info`.
/// This method is for an advanced usage only.
///
/// Most users should prefer using [`Self::read_postings()`] instead.
pub fn read_postings_from_terminfo(
fn new_term_scorer(
&self,
term_info: &TermInfo,
option: IndexRecordOption,
) -> io::Result<SegmentPostings> {
let option = option.downgrade(self.record_option);
let block_postings = self.read_block_postings_from_terminfo(term_info, option)?;
let position_reader = {
if option.has_positions() {
let positions_data = self
.positions_file_slice
.read_bytes_slice(term_info.positions_range.clone())?;
let position_reader = PositionReader::open(positions_data)?;
Some(position_reader)
} else {
None
}
};
Ok(SegmentPostings::from_block_postings(
block_postings,
position_reader,
))
fieldnorm_reader: FieldNormReader,
similarity_weight: Bm25Weight,
) -> io::Result<Box<dyn Scorer>> {
let term_scorer = self.codec.load_term_scorer_type_erased(
term_info,
option,
self,
fieldnorm_reader,
similarity_weight,
)?;
Ok(term_scorer)
}
/// Returns the total number of tokens recorded for all documents
/// (including deleted documents).
pub fn total_num_tokens(&self) -> u64 {
fn read_postings_from_terminfo(
&self,
term_info: &TermInfo,
option: IndexRecordOption,
) -> io::Result<Box<dyn Postings>> {
self.codec
.load_postings_type_erased(term_info, option, self)
}
fn total_num_tokens(&self) -> u64 {
self.total_num_tokens
}
/// Returns the segment postings associated with the term, and with the given option,
/// or `None` if the term has never been encountered and indexed.
///
/// If the field was not indexed with the indexing options that cover
/// the requested options, the returned [`SegmentPostings`] the method does not fail
/// and returns a `SegmentPostings` with as much information as possible.
///
/// For instance, requesting [`IndexRecordOption::WithFreqs`] for a
/// [`TextOptions`](crate::schema::TextOptions) that does not index position
/// will return a [`SegmentPostings`] with `DocId`s and frequencies.
pub fn read_postings(
fn read_postings(
&self,
term: &Term,
option: IndexRecordOption,
) -> io::Result<Option<SegmentPostings>> {
) -> io::Result<Option<Box<dyn Postings>>> {
self.get_term_info(term)?
.map(move |term_info| self.read_postings_from_terminfo(&term_info, option))
.transpose()
}
/// Returns the number of documents containing the term.
pub fn doc_freq(&self, term: &Term) -> io::Result<u32> {
fn doc_freq(&self, term: &Term) -> io::Result<u32> {
Ok(self
.get_term_info(term)?
.map(|term_info| term_info.doc_freq)
.unwrap_or(0u32))
}
#[cfg(feature = "quickwit")]
fn doc_freq_async<'a>(
&'a self,
term: &'a Term,
) -> Pin<Box<dyn Future<Output = io::Result<u32>> + Send + 'a>> {
Box::pin(async move {
Ok(self
.get_term_info_async(term)
.await?
.map(|term_info| term_info.doc_freq)
.unwrap_or(0u32))
})
}
}
#[cfg(feature = "quickwit")]
impl InvertedIndexReader {
impl TantivyInvertedIndexReader {
pub(crate) async fn get_term_info_async(&self, term: &Term) -> io::Result<Option<TermInfo>> {
self.termdict.get_async(term.serialized_value_bytes()).await
}
@@ -482,13 +538,4 @@ impl InvertedIndexReader {
}
Ok(())
}
/// Returns the number of documents containing the term asynchronously.
pub async fn doc_freq_async(&self, term: &Term) -> io::Result<u32> {
Ok(self
.get_term_info_async(term)
.await?
.map(|term_info| term_info.doc_freq)
.unwrap_or(0u32))
}
}

View File

@@ -2,6 +2,7 @@
//!
//! It contains `Index` and `Segment`, where a `Index` consists of one or more `Segment`s.
mod codec_configuration;
mod index;
mod index_meta;
mod inverted_index_reader;
@@ -10,10 +11,13 @@ mod segment_component;
mod segment_id;
mod segment_reader;
pub use self::codec_configuration::CodecConfiguration;
pub use self::index::{Index, IndexBuilder};
pub(crate) use self::index_meta::SegmentMetaInventory;
pub use self::index_meta::{IndexMeta, IndexSettings, Order, SegmentMeta};
pub use self::inverted_index_reader::InvertedIndexReader;
pub use self::inverted_index_reader::{
InvertedIndexFieldSpace, InvertedIndexReader, PostingsData, TantivyInvertedIndexReader,
};
pub use self::segment::Segment;
pub use self::segment_component::SegmentComponent;
pub use self::segment_id::SegmentId;

View File

@@ -2,6 +2,7 @@ use std::fmt;
use std::path::PathBuf;
use super::SegmentComponent;
use crate::codec::StandardCodec;
use crate::directory::error::{OpenReadError, OpenWriteError};
use crate::directory::{Directory, FileSlice, WritePtr};
use crate::index::{Index, SegmentId, SegmentMeta};
@@ -10,25 +11,25 @@ use crate::Opstamp;
/// A segment is a piece of the index.
#[derive(Clone)]
pub struct Segment {
index: Index,
pub struct Segment<C: crate::codec::Codec = StandardCodec> {
index: Index<C>,
meta: SegmentMeta,
}
impl fmt::Debug for Segment {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
impl<C: crate::codec::Codec> fmt::Debug for Segment<C> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "Segment({:?})", self.id().uuid_string())
}
}
impl Segment {
impl<C: crate::codec::Codec> Segment<C> {
/// Creates a new segment given an `Index` and a `SegmentId`
pub(crate) fn for_index(index: Index, meta: SegmentMeta) -> Segment {
pub(crate) fn for_index(index: Index<C>, meta: SegmentMeta) -> Segment<C> {
Segment { index, meta }
}
/// Returns the index the segment belongs to.
pub fn index(&self) -> &Index {
pub fn index(&self) -> &Index<C> {
&self.index
}
@@ -46,7 +47,7 @@ impl Segment {
///
/// This method is only used when updating `max_doc` from 0
/// as we finalize a fresh new segment.
pub fn with_max_doc(self, max_doc: u32) -> Segment {
pub fn with_max_doc(self, max_doc: u32) -> Segment<C> {
Segment {
index: self.index,
meta: self.meta.with_max_doc(max_doc),
@@ -55,7 +56,7 @@ impl Segment {
#[doc(hidden)]
#[must_use]
pub fn with_delete_meta(self, num_deleted_docs: u32, opstamp: Opstamp) -> Segment {
pub fn with_delete_meta(self, num_deleted_docs: u32, opstamp: Opstamp) -> Segment<C> {
Segment {
index: self.index,
meta: self.meta.with_delete_meta(num_deleted_docs, opstamp),

View File

@@ -23,8 +23,6 @@ pub enum SegmentComponent {
/// Accessing a document from the store is relatively slow, as it
/// requires to decompress the entire block it belongs to.
Store,
/// Temporary storage of the documents, before streamed to `Store`.
TempStore,
/// Bitset describing which document of the segment is alive.
/// (It was representing deleted docs but changed to represent alive docs from v0.17)
Delete,
@@ -33,14 +31,13 @@ pub enum SegmentComponent {
impl SegmentComponent {
/// Iterates through the components.
pub fn iterator() -> slice::Iter<'static, SegmentComponent> {
static SEGMENT_COMPONENTS: [SegmentComponent; 8] = [
static SEGMENT_COMPONENTS: [SegmentComponent; 7] = [
SegmentComponent::Postings,
SegmentComponent::Positions,
SegmentComponent::FastFields,
SegmentComponent::FieldNorms,
SegmentComponent::Terms,
SegmentComponent::Store,
SegmentComponent::TempStore,
SegmentComponent::Delete,
];
SEGMENT_COMPONENTS.iter()

View File

@@ -6,11 +6,14 @@ use common::{ByteCount, HasLen};
use fnv::FnvHashMap;
use itertools::Itertools;
use crate::codec::ObjectSafeCodec;
use crate::directory::{CompositeFile, FileSlice};
use crate::error::DataCorruption;
use crate::fastfield::{intersect_alive_bitsets, AliveBitSet, FacetReader, FastFieldReaders};
use crate::fieldnorm::{FieldNormReader, FieldNormReaders};
use crate::index::{InvertedIndexReader, Segment, SegmentComponent, SegmentId};
use crate::index::{
InvertedIndexReader, Segment, SegmentComponent, SegmentId, TantivyInvertedIndexReader,
};
use crate::json_utils::json_path_sep_to_dot;
use crate::schema::{Field, IndexRecordOption, Schema, Type};
use crate::space_usage::SegmentSpaceUsage;
@@ -30,7 +33,7 @@ use crate::{DocId, Opstamp};
/// as close to all of the memory data is mmapped.
#[derive(Clone)]
pub struct SegmentReader {
inv_idx_reader_cache: Arc<RwLock<HashMap<Field, Arc<InvertedIndexReader>>>>,
inv_idx_reader_cache: Arc<RwLock<HashMap<Field, Arc<dyn InvertedIndexReader>>>>,
segment_id: SegmentId,
delete_opstamp: Option<Opstamp>,
@@ -47,6 +50,7 @@ pub struct SegmentReader {
store_file: FileSlice,
alive_bitset_opt: Option<AliveBitSet>,
schema: Schema,
codec: Arc<dyn ObjectSafeCodec>,
}
impl SegmentReader {
@@ -67,6 +71,11 @@ impl SegmentReader {
&self.schema
}
/// Returns the index codec.
pub fn codec(&self) -> &dyn ObjectSafeCodec {
&*self.codec
}
/// Return the number of documents that have been
/// deleted in the segment.
pub fn num_deleted_docs(&self) -> DocId {
@@ -140,15 +149,16 @@ impl SegmentReader {
}
/// Open a new segment for reading.
pub fn open(segment: &Segment) -> crate::Result<SegmentReader> {
pub fn open<C: crate::codec::Codec>(segment: &Segment<C>) -> crate::Result<SegmentReader> {
Self::open_with_custom_alive_set(segment, None)
}
/// Open a new segment for reading.
pub fn open_with_custom_alive_set(
segment: &Segment,
pub fn open_with_custom_alive_set<C: crate::codec::Codec>(
segment: &Segment<C>,
custom_bitset: Option<AliveBitSet>,
) -> crate::Result<SegmentReader> {
let codec: Arc<dyn ObjectSafeCodec> = Arc::new(segment.index().codec().clone());
let termdict_file = segment.open_read(SegmentComponent::Terms)?;
let termdict_composite = CompositeFile::open(&termdict_file)?;
@@ -204,22 +214,23 @@ impl SegmentReader {
alive_bitset_opt,
positions_composite,
schema,
codec,
})
}
/// Returns a field reader associated with the field given in argument.
/// If the field was not present in the index during indexing time,
/// the InvertedIndexReader is empty.
/// the `InvertedIndexReader` is empty.
///
/// The field reader is in charge of iterating through the
/// term dictionary associated with a specific field,
/// and opening the posting list associated with any term.
///
/// If the field is not marked as index, a warning is logged and an empty `InvertedIndexReader`
/// is returned.
/// If the field is not marked as index, a warning is logged and an empty
/// `InvertedIndexReader` is returned.
/// Similarly, if the field is marked as indexed but no term has been indexed for the given
/// index, an empty `InvertedIndexReader` is returned (but no warning is logged).
pub fn inverted_index(&self, field: Field) -> crate::Result<Arc<InvertedIndexReader>> {
pub fn inverted_index(&self, field: Field) -> crate::Result<Arc<dyn InvertedIndexReader>> {
if let Some(inv_idx_reader) = self
.inv_idx_reader_cache
.read()
@@ -244,7 +255,9 @@ impl SegmentReader {
//
// Returns an empty inverted index.
let record_option = record_option_opt.unwrap_or(IndexRecordOption::Basic);
return Ok(Arc::new(InvertedIndexReader::empty(record_option)));
let inv_idx_reader: Arc<dyn InvertedIndexReader> =
Arc::new(TantivyInvertedIndexReader::empty(record_option));
return Ok(inv_idx_reader);
}
let record_option = record_option_opt.unwrap();
@@ -268,12 +281,14 @@ impl SegmentReader {
DataCorruption::comment_only(error_msg)
})?;
let inv_idx_reader = Arc::new(InvertedIndexReader::new(
TermDictionary::open(termdict_file)?,
postings_file,
positions_file,
record_option,
)?);
let inv_idx_reader: Arc<dyn InvertedIndexReader> =
Arc::new(TantivyInvertedIndexReader::new(
TermDictionary::open(termdict_file)?,
postings_file,
positions_file,
record_option,
self.codec.clone(),
)?);
// by releasing the lock in between, we may end up opening the inverting index
// twice, but this is fine.

View File

@@ -9,6 +9,7 @@ use smallvec::smallvec;
use super::operation::{AddOperation, UserOperation};
use super::segment_updater::SegmentUpdater;
use super::{AddBatch, AddBatchReceiver, AddBatchSender, PreparedCommit};
use crate::codec::{Codec, StandardCodec};
use crate::directory::{DirectoryLock, GarbageCollectionResult, TerminatingWrite};
use crate::error::TantivyError;
use crate::fastfield::write_alive_bitset;
@@ -68,12 +69,12 @@ pub struct IndexWriterOptions {
/// indexing queue.
/// Each indexing thread builds its own independent [`Segment`], via
/// a `SegmentWriter` object.
pub struct IndexWriter<D: Document = TantivyDocument> {
pub struct IndexWriter<C: Codec = StandardCodec, D: Document = TantivyDocument> {
// the lock is just used to bind the
// lifetime of the lock with that of the IndexWriter.
_directory_lock: Option<DirectoryLock>,
index: Index,
index: Index<C>,
options: IndexWriterOptions,
@@ -82,7 +83,7 @@ pub struct IndexWriter<D: Document = TantivyDocument> {
index_writer_status: IndexWriterStatus<D>,
operation_sender: AddBatchSender<D>,
segment_updater: SegmentUpdater,
segment_updater: SegmentUpdater<C>,
worker_id: usize,
@@ -128,8 +129,8 @@ fn compute_deleted_bitset(
/// is `==` target_opstamp.
/// For instance, there was no delete operation between the state of the `segment_entry` and
/// the `target_opstamp`, `segment_entry` is not updated.
pub fn advance_deletes(
mut segment: Segment,
pub fn advance_deletes<C: Codec>(
mut segment: Segment<C>,
segment_entry: &mut SegmentEntry,
target_opstamp: Opstamp,
) -> crate::Result<()> {
@@ -179,11 +180,11 @@ pub fn advance_deletes(
Ok(())
}
fn index_documents<D: Document>(
fn index_documents<C: crate::codec::Codec, D: Document>(
memory_budget: usize,
segment: Segment,
segment: Segment<C>,
grouped_document_iterator: &mut dyn Iterator<Item = AddBatch<D>>,
segment_updater: &SegmentUpdater,
segment_updater: &SegmentUpdater<C>,
mut delete_cursor: DeleteCursor,
) -> crate::Result<()> {
let mut segment_writer = SegmentWriter::for_segment(memory_budget, segment.clone())?;
@@ -218,7 +219,7 @@ fn index_documents<D: Document>(
let alive_bitset_opt = apply_deletes(&segment_with_max_doc, &mut delete_cursor, &doc_opstamps)?;
let meta = segment_with_max_doc.meta().clone();
meta.untrack_temp_docstore();
// update segment_updater inventory to remove tempstore
let segment_entry = SegmentEntry::new(meta, delete_cursor, alive_bitset_opt);
segment_updater.schedule_add_segment(segment_entry).wait()?;
@@ -226,8 +227,8 @@ fn index_documents<D: Document>(
}
/// `doc_opstamps` is required to be non-empty.
fn apply_deletes(
segment: &Segment,
fn apply_deletes<C: crate::codec::Codec>(
segment: &Segment<C>,
delete_cursor: &mut DeleteCursor,
doc_opstamps: &[Opstamp],
) -> crate::Result<Option<BitSet>> {
@@ -262,7 +263,7 @@ fn apply_deletes(
})
}
impl<D: Document> IndexWriter<D> {
impl<C: Codec, D: Document> IndexWriter<C, D> {
/// Create a new index writer. Attempts to acquire a lockfile.
///
/// The lockfile should be deleted on drop, but it is possible
@@ -278,7 +279,7 @@ impl<D: Document> IndexWriter<D> {
/// If the memory arena per thread is too small or too big, returns
/// `TantivyError::InvalidArgument`
pub(crate) fn new(
index: &Index,
index: &Index<C>,
options: IndexWriterOptions,
directory_lock: DirectoryLock,
) -> crate::Result<Self> {
@@ -345,7 +346,7 @@ impl<D: Document> IndexWriter<D> {
}
/// Accessor to the index.
pub fn index(&self) -> &Index {
pub fn index(&self) -> &Index<C> {
&self.index
}
@@ -393,7 +394,7 @@ impl<D: Document> IndexWriter<D> {
/// It is safe to start writing file associated with the new `Segment`.
/// These will not be garbage collected as long as an instance object of
/// `SegmentMeta` object associated with the new `Segment` is "alive".
pub fn new_segment(&self) -> Segment {
pub fn new_segment(&self) -> Segment<C> {
self.index.new_segment()
}
@@ -615,7 +616,7 @@ impl<D: Document> IndexWriter<D> {
/// It is also possible to add a payload to the `commit`
/// using this API.
/// See [`PreparedCommit::set_payload()`].
pub fn prepare_commit(&mut self) -> crate::Result<PreparedCommit<'_, D>> {
pub fn prepare_commit(&mut self) -> crate::Result<PreparedCommit<'_, C, D>> {
// Here, because we join all of the worker threads,
// all of the segment update for this commit have been
// sent.
@@ -665,7 +666,7 @@ impl<D: Document> IndexWriter<D> {
self.prepare_commit()?.commit()
}
pub(crate) fn segment_updater(&self) -> &SegmentUpdater {
pub(crate) fn segment_updater(&self) -> &SegmentUpdater<C> {
&self.segment_updater
}
@@ -804,7 +805,7 @@ impl<D: Document> IndexWriter<D> {
}
}
impl<D: Document> Drop for IndexWriter<D> {
impl<C: Codec, D: Document> Drop for IndexWriter<C, D> {
fn drop(&mut self) {
self.segment_updater.kill();
self.drop_sender();

View File

@@ -1,9 +1,11 @@
#[cfg(test)]
mod tests {
use crate::codec::postings::PostingsCodec;
use crate::codec::{Codec, StandardCodec};
use crate::collector::TopDocs;
use crate::fastfield::AliveBitSet;
use crate::index::Index;
use crate::postings::Postings;
use crate::postings::{DocFreq, Postings};
use crate::query::QueryParser;
use crate::schema::{
self, BytesOptions, Facet, FacetOptions, IndexRecordOption, NumericOptions,
@@ -121,21 +123,35 @@ mod tests {
let my_text_field = index.schema().get_field("text_field").unwrap();
let term_a = Term::from_field_text(my_text_field, "text");
let inverted_index = segment_reader.inverted_index(my_text_field).unwrap();
let mut postings = inverted_index
.read_postings(&term_a, IndexRecordOption::WithFreqsAndPositions)
.unwrap()
let term_info = inverted_index.get_term_info(&term_a).unwrap().unwrap();
let postings_data = inverted_index
.read_postings_data(&term_info, IndexRecordOption::WithFreqsAndPositions)
.unwrap();
assert_eq!(postings.doc_freq(), 2);
let postings = StandardCodec
.postings_codec()
.load_postings(
term_info.doc_freq,
postings_data.postings_data,
postings_data.record_option,
postings_data.effective_option,
postings_data.positions_data,
)
.unwrap();
assert_eq!(postings.doc_freq(), DocFreq::Exact(2));
let fallback_bitset = AliveBitSet::for_test_from_deleted_docs(&[0], 100);
assert_eq!(
postings.doc_freq_given_deletes(
crate::indexer::merger::doc_freq_given_deletes(
&postings,
segment_reader.alive_bitset().unwrap_or(&fallback_bitset)
),
2
);
let mut postings = inverted_index
.read_postings_from_terminfo(&term_info, IndexRecordOption::WithFreqsAndPositions)
.unwrap();
assert_eq!(postings.term_freq(), 1);
let mut output = vec![];
let mut output = Vec::new();
postings.positions(&mut output);
assert_eq!(output, vec![1]);
postings.advance();

View File

@@ -7,6 +7,8 @@ use common::ReadOnlyBitSet;
use itertools::Itertools;
use measure_time::debug_time;
use crate::codec::postings::PostingsCodec;
use crate::codec::{Codec, StandardCodec};
use crate::directory::WritePtr;
use crate::docset::{DocSet, TERMINATED};
use crate::error::DataCorruption;
@@ -15,7 +17,7 @@ use crate::fieldnorm::{FieldNormReader, FieldNormReaders, FieldNormsSerializer,
use crate::index::{Segment, SegmentComponent, SegmentReader};
use crate::indexer::doc_id_mapping::{MappingType, SegmentDocIdMapping};
use crate::indexer::SegmentSerializer;
use crate::postings::{InvertedIndexSerializer, Postings, SegmentPostings};
use crate::postings::{InvertedIndexSerializer, Postings};
use crate::schema::{value_type_to_column_type, Field, FieldType, Schema};
use crate::store::StoreWriter;
use crate::termdict::{TermMerger, TermOrdinal};
@@ -76,10 +78,11 @@ fn estimate_total_num_tokens(readers: &[SegmentReader], field: Field) -> crate::
Ok(total_num_tokens)
}
pub struct IndexMerger {
pub struct IndexMerger<C: Codec = StandardCodec> {
schema: Schema,
pub(crate) readers: Vec<SegmentReader>,
max_doc: u32,
codec: C,
}
struct DeltaComputer {
@@ -144,8 +147,8 @@ fn extract_fast_field_required_columns(schema: &Schema) -> Vec<(String, ColumnTy
.collect()
}
impl IndexMerger {
pub fn open(schema: Schema, segments: &[Segment]) -> crate::Result<IndexMerger> {
impl<C: Codec> IndexMerger<C> {
pub fn open(schema: Schema, segments: &[Segment<C>]) -> crate::Result<IndexMerger<C>> {
let alive_bitset = segments.iter().map(|_| None).collect_vec();
Self::open_with_custom_alive_set(schema, segments, alive_bitset)
}
@@ -162,11 +165,15 @@ impl IndexMerger {
// This can be used to merge but also apply an additional filter.
// One use case is demux, which is basically taking a list of
// segments and partitions them e.g. by a value in a field.
//
// # Panics if segments is empty.
pub fn open_with_custom_alive_set(
schema: Schema,
segments: &[Segment],
segments: &[Segment<C>],
alive_bitset_opt: Vec<Option<AliveBitSet>>,
) -> crate::Result<IndexMerger> {
) -> crate::Result<IndexMerger<C>> {
assert!(!segments.is_empty());
let codec = segments[0].index().codec().clone();
let mut readers = vec![];
for (segment, new_alive_bitset_opt) in segments.iter().zip(alive_bitset_opt) {
if segment.meta().num_docs() > 0 {
@@ -189,6 +196,7 @@ impl IndexMerger {
schema,
readers,
max_doc,
codec,
})
}
@@ -297,7 +305,7 @@ impl IndexMerger {
let mut max_term_ords: Vec<TermOrdinal> = Vec::new();
let field_readers: Vec<Arc<InvertedIndexReader>> = self
let field_readers: Vec<Arc<dyn InvertedIndexReader>> = self
.readers
.iter()
.map(|reader| reader.inverted_index(indexed_field))
@@ -355,7 +363,10 @@ impl IndexMerger {
indexed. Have you modified the schema?",
);
let mut segment_postings_containing_the_term: Vec<(usize, SegmentPostings)> = vec![];
let mut segment_postings_containing_the_term: Vec<(
usize,
<C::PostingsCodec as PostingsCodec>::Postings,
)> = Vec::with_capacity(self.readers.len());
while merged_terms.advance() {
segment_postings_containing_the_term.clear();
@@ -366,18 +377,29 @@ impl IndexMerger {
// Let's compute the list of non-empty posting lists
for (segment_ord, term_info) in merged_terms.current_segment_ords_and_term_infos() {
let segment_reader = &self.readers[segment_ord];
let inverted_index: &InvertedIndexReader = &field_readers[segment_ord];
let segment_postings = inverted_index
.read_postings_from_terminfo(&term_info, segment_postings_option)?;
let inverted_index = &field_readers[segment_ord];
let postings_data =
inverted_index.read_postings_data(&term_info, segment_postings_option)?;
let postings = self.codec.postings_codec().load_postings(
term_info.doc_freq,
postings_data.postings_data,
postings_data.record_option,
postings_data.effective_option,
postings_data.positions_data,
)?;
let alive_bitset_opt = segment_reader.alive_bitset();
let doc_freq = if let Some(alive_bitset) = alive_bitset_opt {
segment_postings.doc_freq_given_deletes(alive_bitset)
doc_freq_given_deletes(&postings, alive_bitset)
} else {
segment_postings.doc_freq()
// We do not an exact document frequency here.
match postings.doc_freq() {
crate::postings::DocFreq::Approximate(_) => exact_doc_freq(&postings),
crate::postings::DocFreq::Exact(doc_freq) => doc_freq,
}
};
if doc_freq > 0u32 {
total_doc_freq += doc_freq;
segment_postings_containing_the_term.push((segment_ord, segment_postings));
segment_postings_containing_the_term.push((segment_ord, postings));
}
}
@@ -395,11 +417,7 @@ impl IndexMerger {
assert!(!segment_postings_containing_the_term.is_empty());
let has_term_freq = {
let has_term_freq = !segment_postings_containing_the_term[0]
.1
.block_cursor
.freqs()
.is_empty();
let has_term_freq = segment_postings_containing_the_term[0].1.has_freq();
for (_, postings) in &segment_postings_containing_the_term[1..] {
// This may look at a strange way to test whether we have term freq or not.
// With JSON object, the schema is not sufficient to know whether a term
@@ -415,7 +433,7 @@ impl IndexMerger {
//
// Overall the reliable way to know if we have actual frequencies loaded or not
// is to check whether the actual decoded array is empty or not.
if has_term_freq == postings.block_cursor.freqs().is_empty() {
if postings.has_freq() != has_term_freq {
return Err(DataCorruption::comment_only(
"Term freqs are inconsistent across segments",
)
@@ -525,7 +543,7 @@ impl IndexMerger {
///
/// # Returns
/// The number of documents in the resulting segment.
pub fn write(&self, mut serializer: SegmentSerializer) -> crate::Result<u32> {
pub fn write(&self, mut serializer: SegmentSerializer<C>) -> crate::Result<u32> {
let doc_id_mapping = self.get_doc_id_from_concatenated_data()?;
debug!("write-fieldnorms");
if let Some(fieldnorms_serializer) = serializer.extract_fieldnorms_serializer() {
@@ -553,6 +571,43 @@ impl IndexMerger {
}
}
/// Compute the number of non-deleted documents.
///
/// This method will clone and scan through the posting lists.
/// (this is a rather expensive operation).
pub(crate) fn doc_freq_given_deletes<P: Postings + Clone>(
postings: &P,
alive_bitset: &AliveBitSet,
) -> u32 {
let mut docset = postings.clone();
let mut doc_freq = 0;
loop {
let doc = docset.doc();
if doc == TERMINATED {
return doc_freq;
}
if alive_bitset.is_alive(doc) {
doc_freq += 1u32;
}
docset.advance();
}
}
/// If the postings is not able to inform us of the document frequency,
/// we just scan through it.
pub(crate) fn exact_doc_freq<P: Postings + Clone>(postings: &P) -> u32 {
let mut docset = postings.clone();
let mut doc_freq = 0;
loop {
let doc = docset.doc();
if doc == TERMINATED {
return doc_freq;
}
doc_freq += 1u32;
docset.advance();
}
}
#[cfg(test)]
mod tests {
@@ -561,12 +616,16 @@ mod tests {
use proptest::strategy::Strategy;
use schema::FAST;
use crate::codec::postings::PostingsCodec;
use crate::codec::standard::postings::StandardPostingsCodec;
use crate::collector::tests::{
BytesFastFieldTestCollector, FastFieldTestCollector, TEST_COLLECTOR_WITH_SCORE,
};
use crate::collector::{Count, FacetCollector};
use crate::fastfield::AliveBitSet;
use crate::index::{Index, SegmentId};
use crate::indexer::NoMergePolicy;
use crate::postings::{DocFreq, Postings as _};
use crate::query::{AllQuery, BooleanQuery, EnableScoring, Scorer, TermQuery};
use crate::schema::{
Facet, FacetOptions, IndexRecordOption, NumericOptions, TantivyDocument, Term,
@@ -1518,10 +1577,10 @@ mod tests {
let searcher = reader.searcher();
let mut term_scorer = term_query
.specialized_weight(EnableScoring::enabled_from_searcher(&searcher))?
.term_scorer_for_test(searcher.segment_reader(0u32), 1.0)?
.term_scorer_for_test(searcher.segment_reader(0u32), 1.0)
.unwrap();
assert_eq!(term_scorer.doc(), 0);
assert_nearly_equals!(term_scorer.block_max_score(), 0.0079681855);
assert_nearly_equals!(term_scorer.seek_block_max(0), 0.0079681855);
assert_nearly_equals!(term_scorer.score(), 0.0079681855);
for _ in 0..81 {
writer.add_document(doc!(text=>"hello happy tax payer"))?;
@@ -1534,13 +1593,13 @@ mod tests {
for segment_reader in searcher.segment_readers() {
let mut term_scorer = term_query
.specialized_weight(EnableScoring::enabled_from_searcher(&searcher))?
.term_scorer_for_test(segment_reader, 1.0)?
.term_scorer_for_test(segment_reader, 1.0)
.unwrap();
// the difference compared to before is intrinsic to the bm25 formula. no worries
// there.
for doc in segment_reader.doc_ids_alive() {
assert_eq!(term_scorer.doc(), doc);
assert_nearly_equals!(term_scorer.block_max_score(), 0.003478312);
assert_nearly_equals!(term_scorer.seek_block_max(doc), 0.003478312);
assert_nearly_equals!(term_scorer.score(), 0.003478312);
term_scorer.advance();
}
@@ -1560,12 +1619,12 @@ mod tests {
let segment_reader = searcher.segment_reader(0u32);
let mut term_scorer = term_query
.specialized_weight(EnableScoring::enabled_from_searcher(&searcher))?
.term_scorer_for_test(segment_reader, 1.0)?
.term_scorer_for_test(segment_reader, 1.0)
.unwrap();
// the difference compared to before is intrinsic to the bm25 formula. no worries there.
for doc in segment_reader.doc_ids_alive() {
assert_eq!(term_scorer.doc(), doc);
assert_nearly_equals!(term_scorer.block_max_score(), 0.003478312);
assert_nearly_equals!(term_scorer.seek_block_max(doc), 0.003478312);
assert_nearly_equals!(term_scorer.score(), 0.003478312);
term_scorer.advance();
}
@@ -1579,4 +1638,16 @@ mod tests {
assert!(((super::MAX_DOC_LIMIT - 1) as i32) >= 0);
assert!((super::MAX_DOC_LIMIT as i32) < 0);
}
#[test]
fn test_doc_freq_given_delete() {
let docs =
<StandardPostingsCodec as PostingsCodec>::Postings::create_from_docs(&[0, 2, 10]);
assert_eq!(docs.doc_freq(), DocFreq::Exact(3));
let alive_bitset = AliveBitSet::for_test_from_deleted_docs(&[2], 12);
assert_eq!(super::doc_freq_given_deletes(&docs, &alive_bitset), 2);
let all_deleted =
AliveBitSet::for_test_from_deleted_docs(&[0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11], 12);
assert_eq!(super::doc_freq_given_deletes(&docs, &all_deleted), 0);
}
}

View File

@@ -1,16 +1,17 @@
use super::IndexWriter;
use crate::codec::Codec;
use crate::schema::document::Document;
use crate::{FutureResult, Opstamp, TantivyDocument};
/// A prepared commit
pub struct PreparedCommit<'a, D: Document = TantivyDocument> {
index_writer: &'a mut IndexWriter<D>,
pub struct PreparedCommit<'a, C: Codec, D: Document = TantivyDocument> {
index_writer: &'a mut IndexWriter<C, D>,
payload: Option<String>,
opstamp: Opstamp,
}
impl<'a, D: Document> PreparedCommit<'a, D> {
pub(crate) fn new(index_writer: &'a mut IndexWriter<D>, opstamp: Opstamp) -> Self {
impl<'a, C: Codec, D: Document> PreparedCommit<'a, C, D> {
pub(crate) fn new(index_writer: &'a mut IndexWriter<C, D>, opstamp: Opstamp) -> Self {
Self {
index_writer,
payload: None,

View File

@@ -8,17 +8,17 @@ use crate::store::StoreWriter;
/// Segment serializer is in charge of laying out on disk
/// the data accumulated and sorted by the `SegmentWriter`.
pub struct SegmentSerializer {
segment: Segment,
pub struct SegmentSerializer<C: crate::codec::Codec> {
segment: Segment<C>,
pub(crate) store_writer: StoreWriter,
fast_field_write: WritePtr,
fieldnorms_serializer: Option<FieldNormsSerializer>,
postings_serializer: InvertedIndexSerializer,
}
impl SegmentSerializer {
impl<C: crate::codec::Codec> SegmentSerializer<C> {
/// Creates a new `SegmentSerializer`.
pub fn for_segment(mut segment: Segment) -> crate::Result<SegmentSerializer> {
pub fn for_segment(mut segment: Segment<C>) -> crate::Result<SegmentSerializer<C>> {
let settings = segment.index().settings().clone();
let store_writer = {
let store_write = segment.open_write(SegmentComponent::Store)?;
@@ -50,7 +50,7 @@ impl SegmentSerializer {
self.store_writer.mem_usage()
}
pub fn segment(&self) -> &Segment {
pub fn segment(&self) -> &Segment<C> {
&self.segment
}

View File

@@ -10,10 +10,13 @@ use std::sync::{Arc, RwLock};
use rayon::{ThreadPool, ThreadPoolBuilder};
use super::segment_manager::SegmentManager;
use crate::codec::Codec;
use crate::core::META_FILEPATH;
use crate::directory::{Directory, DirectoryClone, GarbageCollectionResult};
use crate::fastfield::AliveBitSet;
use crate::index::{Index, IndexMeta, IndexSettings, Segment, SegmentId, SegmentMeta};
use crate::index::{
CodecConfiguration, Index, IndexMeta, IndexSettings, Segment, SegmentId, SegmentMeta,
};
use crate::indexer::delete_queue::DeleteCursor;
use crate::indexer::index_writer::advance_deletes;
use crate::indexer::merge_operation::MergeOperationInventory;
@@ -61,10 +64,10 @@ pub(crate) fn save_metas(metas: &IndexMeta, directory: &dyn Directory) -> crate:
// We voluntarily pass a merge_operation ref to guarantee that
// the merge_operation is alive during the process
#[derive(Clone)]
pub(crate) struct SegmentUpdater(Arc<InnerSegmentUpdater>);
pub(crate) struct SegmentUpdater<C: Codec>(Arc<InnerSegmentUpdater<C>>);
impl Deref for SegmentUpdater {
type Target = InnerSegmentUpdater;
impl<C: Codec> Deref for SegmentUpdater<C> {
type Target = InnerSegmentUpdater<C>;
#[inline]
fn deref(&self) -> &Self::Target {
@@ -72,8 +75,8 @@ impl Deref for SegmentUpdater {
}
}
fn garbage_collect_files(
segment_updater: SegmentUpdater,
fn garbage_collect_files<C: Codec>(
segment_updater: SegmentUpdater<C>,
) -> crate::Result<GarbageCollectionResult> {
info!("Running garbage collection");
let mut index = segment_updater.index.clone();
@@ -84,8 +87,8 @@ fn garbage_collect_files(
/// Merges a list of segments the list of segment givens in the `segment_entries`.
/// This function happens in the calling thread and is computationally expensive.
fn merge(
index: &Index,
fn merge<Codec: crate::codec::Codec>(
index: &Index<Codec>,
mut segment_entries: Vec<SegmentEntry>,
target_opstamp: Opstamp,
) -> crate::Result<Option<SegmentEntry>> {
@@ -108,13 +111,13 @@ fn merge(
let delete_cursor = segment_entries[0].delete_cursor().clone();
let segments: Vec<Segment> = segment_entries
let segments: Vec<Segment<Codec>> = segment_entries
.iter()
.map(|segment_entry| index.segment(segment_entry.meta().clone()))
.collect();
// An IndexMerger is like a "view" of our merged segments.
let merger: IndexMerger = IndexMerger::open(index.schema(), &segments[..])?;
let merger: IndexMerger<Codec> = IndexMerger::open(index.schema(), &segments[..])?;
// ... we just serialize this index merger in our new segment to merge the segments.
let segment_serializer = SegmentSerializer::for_segment(merged_segment.clone())?;
@@ -139,10 +142,10 @@ fn merge(
/// meant to work if you have an `IndexWriter` running for the origin indices, or
/// the destination `Index`.
#[doc(hidden)]
pub fn merge_indices<T: Into<Box<dyn Directory>>>(
indices: &[Index],
output_directory: T,
) -> crate::Result<Index> {
pub fn merge_indices<Codec: crate::codec::Codec>(
indices: &[Index<Codec>],
output_directory: Box<dyn Directory>,
) -> crate::Result<Index<Codec>> {
if indices.is_empty() {
// If there are no indices to merge, there is no need to do anything.
return Err(crate::TantivyError::InvalidArgument(
@@ -163,7 +166,7 @@ pub fn merge_indices<T: Into<Box<dyn Directory>>>(
));
}
let mut segments: Vec<Segment> = Vec::new();
let mut segments: Vec<Segment<Codec>> = Vec::new();
for index in indices {
segments.extend(index.searchable_segments()?);
}
@@ -185,12 +188,12 @@ pub fn merge_indices<T: Into<Box<dyn Directory>>>(
/// meant to work if you have an `IndexWriter` running for the origin indices, or
/// the destination `Index`.
#[doc(hidden)]
pub fn merge_filtered_segments<T: Into<Box<dyn Directory>>>(
segments: &[Segment],
pub fn merge_filtered_segments<C: crate::codec::Codec, T: Into<Box<dyn Directory>>>(
segments: &[Segment<C>],
target_settings: IndexSettings,
filter_doc_ids: Vec<Option<AliveBitSet>>,
output_directory: T,
) -> crate::Result<Index> {
) -> crate::Result<Index<C>> {
if segments.is_empty() {
// If there are no indices to merge, there is no need to do anything.
return Err(crate::TantivyError::InvalidArgument(
@@ -211,14 +214,15 @@ pub fn merge_filtered_segments<T: Into<Box<dyn Directory>>>(
));
}
let mut merged_index = Index::create(
output_directory,
target_schema.clone(),
target_settings.clone(),
)?;
let mut merged_index: Index<C> = Index::builder()
.schema(target_schema.clone())
.codec(segments[0].index().codec().clone())
.settings(target_settings.clone())
.create(output_directory.into())?;
let merged_segment = merged_index.new_segment();
let merged_segment_id = merged_segment.id();
let merger: IndexMerger =
let merger: IndexMerger<C> =
IndexMerger::open_with_custom_alive_set(merged_index.schema(), segments, filter_doc_ids)?;
let segment_serializer = SegmentSerializer::for_segment(merged_segment)?;
let num_docs = merger.write(segment_serializer)?;
@@ -235,6 +239,7 @@ pub fn merge_filtered_segments<T: Into<Box<dyn Directory>>>(
))
.trim_end()
);
let codec_configuration = CodecConfiguration::from(segments[0].index().codec());
let index_meta = IndexMeta {
index_settings: target_settings, // index_settings of all segments should be the same
@@ -242,6 +247,7 @@ pub fn merge_filtered_segments<T: Into<Box<dyn Directory>>>(
schema: target_schema,
opstamp: 0u64,
payload: Some(stats),
codec: codec_configuration,
};
// save the meta.json
@@ -250,7 +256,7 @@ pub fn merge_filtered_segments<T: Into<Box<dyn Directory>>>(
Ok(merged_index)
}
pub(crate) struct InnerSegmentUpdater {
pub(crate) struct InnerSegmentUpdater<C: Codec> {
// we keep a copy of the current active IndexMeta to
// avoid loading the file every time we need it in the
// `SegmentUpdater`.
@@ -261,7 +267,7 @@ pub(crate) struct InnerSegmentUpdater {
pool: ThreadPool,
merge_thread_pool: ThreadPool,
index: Index,
index: Index<C>,
segment_manager: SegmentManager,
merge_policy: RwLock<Arc<dyn MergePolicy>>,
killed: AtomicBool,
@@ -269,13 +275,13 @@ pub(crate) struct InnerSegmentUpdater {
merge_operations: MergeOperationInventory,
}
impl SegmentUpdater {
impl<Codec: crate::codec::Codec> SegmentUpdater<Codec> {
pub fn create(
index: Index,
index: Index<Codec>,
stamper: Stamper,
delete_cursor: &DeleteCursor,
num_merge_threads: usize,
) -> crate::Result<SegmentUpdater> {
) -> crate::Result<Self> {
let segments = index.searchable_segment_metas()?;
let segment_manager = SegmentManager::from_segments(segments, delete_cursor);
let pool = ThreadPoolBuilder::new()
@@ -404,12 +410,14 @@ impl SegmentUpdater {
//
// Segment 1 from disk 1, Segment 1 from disk 2, etc.
committed_segment_metas.sort_by_key(|segment_meta| -(segment_meta.max_doc() as i32));
let codec = CodecConfiguration::from(index.codec());
let index_meta = IndexMeta {
index_settings: index.settings().clone(),
segments: committed_segment_metas,
schema: index.schema(),
opstamp,
payload: commit_message,
codec,
};
// TODO add context to the error.
save_metas(&index_meta, directory.box_clone().borrow_mut())?;
@@ -443,7 +451,7 @@ impl SegmentUpdater {
opstamp: Opstamp,
payload: Option<String>,
) -> FutureResult<Opstamp> {
let segment_updater: SegmentUpdater = self.clone();
let segment_updater: SegmentUpdater<Codec> = self.clone();
self.schedule_task(move || {
let segment_entries = segment_updater.purge_deletes(opstamp)?;
segment_updater.segment_manager.commit(segment_entries);
@@ -702,6 +710,7 @@ impl SegmentUpdater {
#[cfg(test)]
mod tests {
use super::merge_indices;
use crate::codec::StandardCodec;
use crate::collector::TopDocs;
use crate::directory::RamDirectory;
use crate::fastfield::AliveBitSet;
@@ -915,7 +924,7 @@ mod tests {
#[test]
fn test_merge_empty_indices_array() {
let merge_result = merge_indices(&[], RamDirectory::default());
let merge_result = merge_indices::<StandardCodec>(&[], Box::new(RamDirectory::default()));
assert!(merge_result.is_err());
}
@@ -942,7 +951,10 @@ mod tests {
};
// mismatched schema index list
let result = merge_indices(&[first_index, second_index], RamDirectory::default());
let result = merge_indices(
&[first_index, second_index],
Box::new(RamDirectory::default()),
);
assert!(result.is_err());
Ok(())

View File

@@ -4,6 +4,7 @@ use itertools::Itertools;
use tokenizer_api::BoxTokenStream;
use super::operation::AddOperation;
use crate::codec::Codec;
use crate::fastfield::FastFieldsWriter;
use crate::fieldnorm::{FieldNormReaders, FieldNormsWriter};
use crate::index::{Segment, SegmentComponent};
@@ -12,7 +13,7 @@ use crate::indexer::segment_serializer::SegmentSerializer;
use crate::json_utils::{index_json_value, IndexingPositionsPerPath};
use crate::postings::{
compute_table_memory_size, serialize_postings, IndexingContext, IndexingPosition,
PerFieldPostingsWriter, PostingsWriter,
PerFieldPostingsWriter, PostingsWriter, PostingsWriterEnum,
};
use crate::schema::document::{Document, Value};
use crate::schema::{FieldEntry, FieldType, Schema, DATE_TIME_PRECISION_INDEXED};
@@ -45,11 +46,11 @@ fn compute_initial_table_size(per_thread_memory_budget: usize) -> crate::Result<
///
/// They creates the postings list in anonymous memory.
/// The segment is laid on disk when the segment gets `finalized`.
pub struct SegmentWriter {
pub struct SegmentWriter<Codec: crate::codec::Codec> {
pub(crate) max_doc: DocId,
pub(crate) ctx: IndexingContext,
pub(crate) per_field_postings_writers: PerFieldPostingsWriter,
pub(crate) segment_serializer: SegmentSerializer,
pub(crate) segment_serializer: SegmentSerializer<Codec>,
pub(crate) fast_field_writers: FastFieldsWriter,
pub(crate) fieldnorms_writer: FieldNormsWriter,
pub(crate) json_path_writer: JsonPathWriter,
@@ -60,7 +61,7 @@ pub struct SegmentWriter {
schema: Schema,
}
impl SegmentWriter {
impl<Codec: crate::codec::Codec> SegmentWriter<Codec> {
/// Creates a new `SegmentWriter`
///
/// The arguments are defined as follows
@@ -70,7 +71,10 @@ impl SegmentWriter {
/// behavior as a memory limit.
/// - segment: The segment being written
/// - schema
pub fn for_segment(memory_budget_in_bytes: usize, segment: Segment) -> crate::Result<Self> {
pub fn for_segment(
memory_budget_in_bytes: usize,
segment: Segment<Codec>,
) -> crate::Result<Self> {
let schema = segment.schema();
let tokenizer_manager = segment.index().tokenizers().clone();
let tokenizer_manager_fast_field = segment.index().fast_field_tokenizer().clone();
@@ -169,7 +173,7 @@ impl SegmentWriter {
}
let (term_buffer, ctx) = (&mut self.term_buffer, &mut self.ctx);
let postings_writer: &mut dyn PostingsWriter =
let postings_writer: &mut PostingsWriterEnum =
self.per_field_postings_writers.get_for_field_mut(field);
term_buffer.clear_with_field(field);
@@ -386,13 +390,13 @@ impl SegmentWriter {
/// to the `SegmentSerializer`.
///
/// `doc_id_map` is used to map to the new doc_id order.
fn remap_and_write(
fn remap_and_write<C: Codec>(
schema: Schema,
per_field_postings_writers: &PerFieldPostingsWriter,
ctx: IndexingContext,
fast_field_writers: FastFieldsWriter,
fieldnorms_writer: &FieldNormsWriter,
mut serializer: SegmentSerializer,
mut serializer: SegmentSerializer<C>,
) -> crate::Result<()> {
debug!("remap-and-write");
if let Some(fieldnorms_serializer) = serializer.extract_fieldnorms_serializer() {

View File

@@ -1,5 +1,7 @@
use std::marker::PhantomData;
use crate::codec::StandardCodec;
use crate::index::CodecConfiguration;
use crate::indexer::operation::AddOperation;
use crate::indexer::segment_updater::save_metas;
use crate::indexer::SegmentWriter;
@@ -7,22 +9,25 @@ use crate::schema::document::Document;
use crate::{Directory, Index, IndexMeta, Opstamp, Segment, TantivyDocument};
#[doc(hidden)]
pub struct SingleSegmentIndexWriter<D: Document = TantivyDocument> {
segment_writer: SegmentWriter,
segment: Segment,
pub struct SingleSegmentIndexWriter<
Codec: crate::codec::Codec = StandardCodec,
D: Document = TantivyDocument,
> {
segment_writer: SegmentWriter<Codec>,
segment: Segment<Codec>,
opstamp: Opstamp,
_phantom: PhantomData<D>,
_doc: PhantomData<D>,
}
impl<D: Document> SingleSegmentIndexWriter<D> {
pub fn new(index: Index, mem_budget: usize) -> crate::Result<Self> {
impl<Codec: crate::codec::Codec, D: Document> SingleSegmentIndexWriter<Codec, D> {
pub fn new(index: Index<Codec>, mem_budget: usize) -> crate::Result<Self> {
let segment = index.new_segment();
let segment_writer = SegmentWriter::for_segment(mem_budget, segment.clone())?;
Ok(Self {
segment_writer,
segment,
opstamp: 0,
_phantom: PhantomData,
_doc: PhantomData,
})
}
@@ -37,10 +42,10 @@ impl<D: Document> SingleSegmentIndexWriter<D> {
.add_document(AddOperation { opstamp, document })
}
pub fn finalize(self) -> crate::Result<Index> {
pub fn finalize(self) -> crate::Result<Index<Codec>> {
let max_doc = self.segment_writer.max_doc();
self.segment_writer.finalize()?;
let segment: Segment = self.segment.with_max_doc(max_doc);
let segment: Segment<Codec> = self.segment.with_max_doc(max_doc);
let index = segment.index();
let index_meta = IndexMeta {
index_settings: index.settings().clone(),
@@ -48,6 +53,7 @@ impl<D: Document> SingleSegmentIndexWriter<D> {
schema: index.schema(),
opstamp: 0,
payload: None,
codec: CodecConfiguration::from(index.codec()),
};
save_metas(&index_meta, index.directory())?;
index.directory().sync_directory()?;

View File

@@ -166,6 +166,9 @@ mod functional_test;
#[macro_use]
mod macros;
/// Tantivy codecs describes how data is layed out on disk.
pub mod codec;
mod future_result;
// Re-exports
@@ -225,7 +228,7 @@ pub use crate::core::{json_utils, Executor, Searcher, SearcherGeneration};
pub use crate::directory::Directory;
pub use crate::index::{
Index, IndexBuilder, IndexMeta, IndexSettings, InvertedIndexReader, Order, Segment,
SegmentMeta, SegmentReader,
SegmentMeta, SegmentReader, TantivyInvertedIndexReader,
};
pub use crate::indexer::{IndexWriter, SingleSegmentIndexWriter};
pub use crate::schema::{Document, TantivyDocument, Term};

View File

@@ -22,12 +22,6 @@ pub(crate) struct JsonPostingsWriter<Rec: Recorder> {
non_str_posting_writer: SpecializedPostingsWriter<DocIdRecorder>,
}
impl<Rec: Recorder> From<JsonPostingsWriter<Rec>> for Box<dyn PostingsWriter> {
fn from(json_postings_writer: JsonPostingsWriter<Rec>) -> Box<dyn PostingsWriter> {
Box::new(json_postings_writer)
}
}
impl<Rec: Recorder> PostingsWriter for JsonPostingsWriter<Rec> {
#[inline]
fn subscribe(

View File

@@ -1,5 +1,5 @@
use crate::docset::{DocSet, TERMINATED};
use crate::postings::{Postings, SegmentPostings};
use crate::postings::{DocFreq, Postings};
use crate::DocId;
/// `LoadedPostings` is a `DocSet` and `Postings` implementation.
@@ -25,16 +25,16 @@ impl LoadedPostings {
/// Creates a new `LoadedPostings` from a `SegmentPostings`.
///
/// It will also preload positions, if positions are available in the SegmentPostings.
pub fn load(segment_postings: &mut SegmentPostings) -> LoadedPostings {
let num_docs = segment_postings.doc_freq() as usize;
pub fn load(postings: &mut Box<dyn Postings>) -> LoadedPostings {
let num_docs: usize = u32::from(postings.doc_freq()) as usize;
let mut doc_ids = Vec::with_capacity(num_docs);
let mut positions = Vec::with_capacity(num_docs);
let mut position_offsets = Vec::with_capacity(num_docs);
while segment_postings.doc() != TERMINATED {
while postings.doc() != TERMINATED {
position_offsets.push(positions.len() as u32);
doc_ids.push(segment_postings.doc());
segment_postings.append_positions_with_offset(0, &mut positions);
segment_postings.advance();
doc_ids.push(postings.doc());
postings.append_positions_with_offset(0, &mut positions);
postings.advance();
}
position_offsets.push(positions.len() as u32);
LoadedPostings {
@@ -101,6 +101,14 @@ impl Postings for LoadedPostings {
output.push(*pos + offset);
}
}
fn has_freq(&self) -> bool {
true
}
fn doc_freq(&self) -> DocFreq {
DocFreq::Exact(self.doc_ids.len() as u32)
}
}
#[cfg(test)]

View File

@@ -4,7 +4,6 @@ mod block_search;
pub(crate) use self::block_search::branchless_binary_search;
mod block_segment_postings;
pub(crate) mod compression;
mod indexing_context;
mod json_postings_writer;
@@ -13,32 +12,23 @@ mod per_field_postings_writer;
mod postings;
mod postings_writer;
mod recorder;
mod segment_postings;
mod serializer;
mod skip;
pub(crate) mod serializer;
pub(crate) mod skip;
mod term_info;
pub(crate) use loaded_postings::LoadedPostings;
pub use postings::DocFreq;
pub(crate) use stacker::compute_table_memory_size;
pub use self::block_segment_postings::BlockSegmentPostings;
pub(crate) use self::indexing_context::IndexingContext;
pub(crate) use self::per_field_postings_writer::PerFieldPostingsWriter;
pub use self::postings::Postings;
pub(crate) use self::postings_writer::{serialize_postings, IndexingPosition, PostingsWriter};
pub use self::segment_postings::SegmentPostings;
pub(crate) use self::postings_writer::{
serialize_postings, IndexingPosition, PostingsWriter, PostingsWriterEnum,
};
pub use self::serializer::{FieldSerializer, InvertedIndexSerializer};
pub(crate) use self::skip::{BlockInfo, SkipReader};
pub use self::term_info::TermInfo;
#[expect(clippy::enum_variant_names)]
#[derive(Debug, PartialEq, Clone, Copy, Eq)]
pub(crate) enum FreqReadingOption {
NoFreq,
SkipFreq,
ReadFreq,
}
#[cfg(test)]
pub(crate) mod tests {
use std::mem;
@@ -49,6 +39,7 @@ pub(crate) mod tests {
use crate::index::{Index, SegmentComponent, SegmentReader};
use crate::indexer::operation::AddOperation;
use crate::indexer::SegmentWriter;
use crate::postings::DocFreq;
use crate::query::Scorer;
use crate::schema::{
Field, IndexRecordOption, Schema, Term, TextFieldIndexing, TextOptions, INDEXED, TEXT,
@@ -279,11 +270,11 @@ pub(crate) mod tests {
}
{
let term_a = Term::from_field_text(text_field, "a");
let mut postings_a = segment_reader
let mut postings_a: Box<dyn Postings> = segment_reader
.inverted_index(term_a.field())?
.read_postings(&term_a, IndexRecordOption::WithFreqsAndPositions)?
.unwrap();
assert_eq!(postings_a.len(), 1000);
assert_eq!(postings_a.doc_freq(), DocFreq::Exact(1000));
assert_eq!(postings_a.doc(), 0);
assert_eq!(postings_a.term_freq(), 6);
postings_a.positions(&mut positions);
@@ -306,7 +297,7 @@ pub(crate) mod tests {
.inverted_index(term_e.field())?
.read_postings(&term_e, IndexRecordOption::WithFreqsAndPositions)?
.unwrap();
assert_eq!(postings_e.len(), 1000 - 2);
assert_eq!(postings_e.doc_freq(), DocFreq::Exact(1000 - 2));
for i in 2u32..1000u32 {
assert_eq!(postings_e.term_freq(), i);
postings_e.positions(&mut positions);

View File

@@ -1,16 +1,15 @@
use crate::postings::json_postings_writer::JsonPostingsWriter;
use crate::postings::postings_writer::SpecializedPostingsWriter;
use crate::postings::postings_writer::{PostingsWriterEnum, SpecializedPostingsWriter};
use crate::postings::recorder::{DocIdRecorder, TermFrequencyRecorder, TfAndPositionRecorder};
use crate::postings::PostingsWriter;
use crate::schema::{Field, FieldEntry, FieldType, IndexRecordOption, Schema};
pub(crate) struct PerFieldPostingsWriter {
per_field_postings_writers: Vec<Box<dyn PostingsWriter>>,
per_field_postings_writers: Vec<PostingsWriterEnum>,
}
impl PerFieldPostingsWriter {
pub fn for_schema(schema: &Schema) -> Self {
let per_field_postings_writers = schema
let per_field_postings_writers: Vec<PostingsWriterEnum> = schema
.fields()
.map(|(_, field_entry)| posting_writer_from_field_entry(field_entry))
.collect();
@@ -19,16 +18,16 @@ impl PerFieldPostingsWriter {
}
}
pub(crate) fn get_for_field(&self, field: Field) -> &dyn PostingsWriter {
self.per_field_postings_writers[field.field_id() as usize].as_ref()
pub(crate) fn get_for_field(&self, field: Field) -> &PostingsWriterEnum {
&self.per_field_postings_writers[field.field_id() as usize]
}
pub(crate) fn get_for_field_mut(&mut self, field: Field) -> &mut dyn PostingsWriter {
self.per_field_postings_writers[field.field_id() as usize].as_mut()
pub(crate) fn get_for_field_mut(&mut self, field: Field) -> &mut PostingsWriterEnum {
&mut self.per_field_postings_writers[field.field_id() as usize]
}
}
fn posting_writer_from_field_entry(field_entry: &FieldEntry) -> Box<dyn PostingsWriter> {
fn posting_writer_from_field_entry(field_entry: &FieldEntry) -> PostingsWriterEnum {
match *field_entry.field_type() {
FieldType::Str(ref text_options) => text_options
.get_indexing_options()
@@ -51,7 +50,7 @@ fn posting_writer_from_field_entry(field_entry: &FieldEntry) -> Box<dyn Postings
| FieldType::Date(_)
| FieldType::Bytes(_)
| FieldType::IpAddr(_)
| FieldType::Facet(_) => Box::<SpecializedPostingsWriter<DocIdRecorder>>::default(),
| FieldType::Facet(_) => <SpecializedPostingsWriter<DocIdRecorder>>::default().into(),
FieldType::JsonObject(ref json_object_options) => {
if let Some(text_indexing_option) = json_object_options.get_text_indexing_options() {
match text_indexing_option.index_option() {

View File

@@ -1,5 +1,25 @@
use crate::docset::DocSet;
/// Result of the doc_freq method.
///
/// Postings can inform us that the document frequency is approximate.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub enum DocFreq {
/// The document frequency is approximate.
Approximate(u32),
/// The document frequency is exact.
Exact(u32),
}
impl From<DocFreq> for u32 {
fn from(doc_freq: DocFreq) -> Self {
match doc_freq {
DocFreq::Approximate(approximate_doc_freq) => approximate_doc_freq,
DocFreq::Exact(doc_freq) => doc_freq,
}
}
}
/// Postings (also called inverted list)
///
/// For a given term, it is the list of doc ids of the doc
@@ -14,6 +34,9 @@ pub trait Postings: DocSet + 'static {
/// The number of times the term appears in the document.
fn term_freq(&self) -> u32;
/// Returns the number of documents containing the term in the segment.
fn doc_freq(&self) -> DocFreq;
/// Returns the positions offsetted with a given value.
/// It is not necessary to clear the `output` before calling this method.
/// The output vector will be resized to the `term_freq`.
@@ -31,6 +54,16 @@ pub trait Postings: DocSet + 'static {
fn positions(&mut self, output: &mut Vec<u32>) {
self.positions_with_offset(0u32, output);
}
/// Returns true if the term_frequency is available.
///
/// This is a tricky question, because on JSON fields, it is possible
/// for a text term to have term freq, whereas a number term in the field has none.
///
/// This function returns whether the actual term has term frequencies or not.
/// In this above JSON field example, `has_freq` should return true for the
/// earlier and false for the latter.
fn has_freq(&self) -> bool;
}
impl Postings for Box<dyn Postings> {
@@ -41,4 +74,12 @@ impl Postings for Box<dyn Postings> {
fn append_positions_with_offset(&mut self, offset: u32, output: &mut Vec<u32>) {
(**self).append_positions_with_offset(offset, output);
}
fn has_freq(&self) -> bool {
(**self).has_freq()
}
fn doc_freq(&self) -> DocFreq {
(**self).doc_freq()
}
}

View File

@@ -7,7 +7,10 @@ use stacker::Addr;
use crate::fieldnorm::FieldNormReaders;
use crate::indexer::indexing_term::IndexingTerm;
use crate::indexer::path_to_unordered_id::OrderedPathId;
use crate::postings::recorder::{BufferLender, Recorder};
use crate::postings::json_postings_writer::JsonPostingsWriter;
use crate::postings::recorder::{
BufferLender, DocIdRecorder, Recorder, TermFrequencyRecorder, TfAndPositionRecorder,
};
use crate::postings::{
FieldSerializer, IndexingContext, InvertedIndexSerializer, PerFieldPostingsWriter,
};
@@ -100,6 +103,141 @@ pub(crate) struct IndexingPosition {
pub end_position: u32,
}
pub enum PostingsWriterEnum {
DocId(SpecializedPostingsWriter<DocIdRecorder>),
DocIdTf(SpecializedPostingsWriter<TermFrequencyRecorder>),
DocTfAndPosition(SpecializedPostingsWriter<TfAndPositionRecorder>),
JsonDocId(JsonPostingsWriter<DocIdRecorder>),
JsonDocIdTf(JsonPostingsWriter<TermFrequencyRecorder>),
JsonDocTfAndPosition(JsonPostingsWriter<TfAndPositionRecorder>),
}
impl From<SpecializedPostingsWriter<DocIdRecorder>> for PostingsWriterEnum {
fn from(doc_id_recorder_writer: SpecializedPostingsWriter<DocIdRecorder>) -> Self {
PostingsWriterEnum::DocId(doc_id_recorder_writer)
}
}
impl From<SpecializedPostingsWriter<TermFrequencyRecorder>> for PostingsWriterEnum {
fn from(doc_id_tf_recorder_writer: SpecializedPostingsWriter<TermFrequencyRecorder>) -> Self {
PostingsWriterEnum::DocIdTf(doc_id_tf_recorder_writer)
}
}
impl From<SpecializedPostingsWriter<TfAndPositionRecorder>> for PostingsWriterEnum {
fn from(
doc_id_tf_and_positions_recorder_writer: SpecializedPostingsWriter<TfAndPositionRecorder>,
) -> Self {
PostingsWriterEnum::DocTfAndPosition(doc_id_tf_and_positions_recorder_writer)
}
}
impl From<JsonPostingsWriter<DocIdRecorder>> for PostingsWriterEnum {
fn from(doc_id_recorder_writer: JsonPostingsWriter<DocIdRecorder>) -> Self {
PostingsWriterEnum::JsonDocId(doc_id_recorder_writer)
}
}
impl From<JsonPostingsWriter<TermFrequencyRecorder>> for PostingsWriterEnum {
fn from(doc_id_tf_recorder_writer: JsonPostingsWriter<TermFrequencyRecorder>) -> Self {
PostingsWriterEnum::JsonDocIdTf(doc_id_tf_recorder_writer)
}
}
impl From<JsonPostingsWriter<TfAndPositionRecorder>> for PostingsWriterEnum {
fn from(
doc_id_tf_and_positions_recorder_writer: JsonPostingsWriter<TfAndPositionRecorder>,
) -> Self {
PostingsWriterEnum::JsonDocTfAndPosition(doc_id_tf_and_positions_recorder_writer)
}
}
impl PostingsWriter for PostingsWriterEnum {
fn subscribe(&mut self, doc: DocId, pos: u32, term: &IndexingTerm, ctx: &mut IndexingContext) {
match self {
PostingsWriterEnum::DocId(writer) => writer.subscribe(doc, pos, term, ctx),
PostingsWriterEnum::DocIdTf(writer) => writer.subscribe(doc, pos, term, ctx),
PostingsWriterEnum::DocTfAndPosition(writer) => writer.subscribe(doc, pos, term, ctx),
PostingsWriterEnum::JsonDocId(writer) => writer.subscribe(doc, pos, term, ctx),
PostingsWriterEnum::JsonDocIdTf(writer) => writer.subscribe(doc, pos, term, ctx),
PostingsWriterEnum::JsonDocTfAndPosition(writer) => {
writer.subscribe(doc, pos, term, ctx)
}
}
}
fn serialize(
&self,
term_addrs: &[(Field, OrderedPathId, &[u8], Addr)],
ordered_id_to_path: &[&str],
ctx: &IndexingContext,
serializer: &mut FieldSerializer,
) -> io::Result<()> {
match self {
PostingsWriterEnum::DocId(writer) => {
writer.serialize(term_addrs, ordered_id_to_path, ctx, serializer)
}
PostingsWriterEnum::DocIdTf(writer) => {
writer.serialize(term_addrs, ordered_id_to_path, ctx, serializer)
}
PostingsWriterEnum::DocTfAndPosition(writer) => {
writer.serialize(term_addrs, ordered_id_to_path, ctx, serializer)
}
PostingsWriterEnum::JsonDocId(writer) => {
writer.serialize(term_addrs, ordered_id_to_path, ctx, serializer)
}
PostingsWriterEnum::JsonDocIdTf(writer) => {
writer.serialize(term_addrs, ordered_id_to_path, ctx, serializer)
}
PostingsWriterEnum::JsonDocTfAndPosition(writer) => {
writer.serialize(term_addrs, ordered_id_to_path, ctx, serializer)
}
}
}
/// Tokenize a text and subscribe all of its token.
fn index_text(
&mut self,
doc_id: DocId,
token_stream: &mut dyn TokenStream,
term_buffer: &mut IndexingTerm,
ctx: &mut IndexingContext,
indexing_position: &mut IndexingPosition,
) {
match self {
PostingsWriterEnum::DocId(writer) => {
writer.index_text(doc_id, token_stream, term_buffer, ctx, indexing_position)
}
PostingsWriterEnum::DocIdTf(writer) => {
writer.index_text(doc_id, token_stream, term_buffer, ctx, indexing_position)
}
PostingsWriterEnum::DocTfAndPosition(writer) => {
writer.index_text(doc_id, token_stream, term_buffer, ctx, indexing_position)
}
PostingsWriterEnum::JsonDocId(writer) => {
writer.index_text(doc_id, token_stream, term_buffer, ctx, indexing_position)
}
PostingsWriterEnum::JsonDocIdTf(writer) => {
writer.index_text(doc_id, token_stream, term_buffer, ctx, indexing_position)
}
PostingsWriterEnum::JsonDocTfAndPosition(writer) => {
writer.index_text(doc_id, token_stream, term_buffer, ctx, indexing_position)
}
}
}
fn total_num_tokens(&self) -> u64 {
match self {
PostingsWriterEnum::DocId(writer) => writer.total_num_tokens(),
PostingsWriterEnum::DocIdTf(writer) => writer.total_num_tokens(),
PostingsWriterEnum::DocTfAndPosition(writer) => writer.total_num_tokens(),
PostingsWriterEnum::JsonDocId(writer) => writer.total_num_tokens(),
PostingsWriterEnum::JsonDocIdTf(writer) => writer.total_num_tokens(),
PostingsWriterEnum::JsonDocTfAndPosition(writer) => writer.total_num_tokens(),
}
}
}
/// The `PostingsWriter` is in charge of receiving documenting
/// and building a `Segment` in anonymous memory.
///
@@ -171,14 +309,6 @@ pub(crate) struct SpecializedPostingsWriter<Rec: Recorder> {
_recorder_type: PhantomData<Rec>,
}
impl<Rec: Recorder> From<SpecializedPostingsWriter<Rec>> for Box<dyn PostingsWriter> {
fn from(
specialized_postings_writer: SpecializedPostingsWriter<Rec>,
) -> Box<dyn PostingsWriter> {
Box::new(specialized_postings_writer)
}
}
impl<Rec: Recorder> SpecializedPostingsWriter<Rec> {
#[inline]
pub(crate) fn serialize_one_term(

View File

@@ -70,7 +70,7 @@ pub(crate) trait Recorder: Copy + Default + Send + Sync + 'static {
fn serialize(
&self,
arena: &MemoryArena,
serializer: &mut FieldSerializer<'_>,
serializer: &mut FieldSerializer,
buffer_lender: &mut BufferLender,
);
/// Returns the number of document containing this term.
@@ -113,7 +113,7 @@ impl Recorder for DocIdRecorder {
fn serialize(
&self,
arena: &MemoryArena,
serializer: &mut FieldSerializer<'_>,
serializer: &mut FieldSerializer,
buffer_lender: &mut BufferLender,
) {
let buffer = buffer_lender.lend_u8();
@@ -181,7 +181,7 @@ impl Recorder for TermFrequencyRecorder {
fn serialize(
&self,
arena: &MemoryArena,
serializer: &mut FieldSerializer<'_>,
serializer: &mut FieldSerializer,
buffer_lender: &mut BufferLender,
) {
let buffer = buffer_lender.lend_u8();
@@ -238,7 +238,7 @@ impl Recorder for TfAndPositionRecorder {
fn serialize(
&self,
arena: &MemoryArena,
serializer: &mut FieldSerializer<'_>,
serializer: &mut FieldSerializer,
buffer_lender: &mut BufferLender,
) {
let (buffer_u8, buffer_positions) = buffer_lender.lend_all();

View File

@@ -8,7 +8,7 @@ use crate::directory::{CompositeWrite, WritePtr};
use crate::fieldnorm::FieldNormReader;
use crate::index::Segment;
use crate::positions::PositionSerializer;
use crate::postings::compression::{BlockEncoder, VIntEncoder, COMPRESSION_BLOCK_SIZE};
use crate::postings::compression::{BlockEncoder, VIntEncoder as _, COMPRESSION_BLOCK_SIZE};
use crate::postings::skip::SkipSerializer;
use crate::query::Bm25Weight;
use crate::schema::{Field, FieldEntry, FieldType, IndexRecordOption, Schema};
@@ -55,7 +55,9 @@ pub struct InvertedIndexSerializer {
impl InvertedIndexSerializer {
/// Open a new `InvertedIndexSerializer` for the given segment
pub fn open(segment: &mut Segment) -> crate::Result<InvertedIndexSerializer> {
pub fn open<C: crate::codec::Codec>(
segment: &mut Segment<C>,
) -> crate::Result<InvertedIndexSerializer> {
use crate::index::SegmentComponent::{Positions, Postings, Terms};
let inv_index_serializer = InvertedIndexSerializer {
terms_write: CompositeWrite::wrap(segment.open_write(Terms)?),

View File

@@ -142,23 +142,6 @@ impl SkipReader {
skip_reader
}
pub fn reset(&mut self, data: OwnedBytes, doc_freq: u32) {
self.last_doc_in_block = if doc_freq >= COMPRESSION_BLOCK_SIZE as u32 {
0
} else {
TERMINATED
};
self.last_doc_in_previous_block = 0u32;
self.owned_read = data;
self.block_info = BlockInfo::VInt { num_docs: doc_freq };
self.byte_offset = 0;
self.remaining_docs = doc_freq;
self.position_offset = 0u64;
if doc_freq >= COMPRESSION_BLOCK_SIZE as u32 {
self.read_block_info();
}
}
// Returns the block max score for this block if available.
//
// The block max score is available for all full bitpacked block,

View File

@@ -2,7 +2,7 @@ use crate::docset::{DocSet, COLLECT_BLOCK_BUFFER_LEN, TERMINATED};
use crate::index::SegmentReader;
use crate::query::boost_query::BoostScorer;
use crate::query::explanation::does_not_match;
use crate::query::{EnableScoring, Explanation, Query, Scorer, Weight};
use crate::query::{box_scorer, EnableScoring, Explanation, Query, Scorer, Weight};
use crate::{DocId, Score};
/// Query that matches all of the documents.
@@ -24,9 +24,9 @@ impl Weight for AllWeight {
fn scorer(&self, reader: &SegmentReader, boost: Score) -> crate::Result<Box<dyn Scorer>> {
let all_scorer = AllScorer::new(reader.max_doc());
if boost != 1.0 {
Ok(Box::new(BoostScorer::new(all_scorer, boost)))
Ok(box_scorer(BoostScorer::new(all_scorer, boost)))
} else {
Ok(Box::new(all_scorer))
Ok(box_scorer(all_scorer))
}
}

View File

@@ -10,7 +10,7 @@ use crate::postings::TermInfo;
use crate::query::{BitSetDocSet, ConstScorer, Explanation, Scorer, Weight};
use crate::schema::{Field, IndexRecordOption};
use crate::termdict::{TermDictionary, TermStreamer};
use crate::{DocId, Score, TantivyError};
use crate::{DocId, DocSet, Score, TantivyError};
/// A weight struct for Fuzzy Term and Regex Queries
pub struct AutomatonWeight<A> {
@@ -92,18 +92,9 @@ where
let mut term_stream = self.automaton_stream(term_dict)?;
while term_stream.advance() {
let term_info = term_stream.value();
let mut block_segment_postings = inverted_index
.read_block_postings_from_terminfo(term_info, IndexRecordOption::Basic)?;
loop {
let docs = block_segment_postings.docs();
if docs.is_empty() {
break;
}
for &doc in docs {
doc_bitset.insert(doc);
}
block_segment_postings.advance();
}
let mut block_segment_postings =
inverted_index.read_postings_from_terminfo(term_info, IndexRecordOption::Basic)?;
block_segment_postings.fill_bitset(&mut doc_bitset);
}
let doc_bitset = BitSetDocSet::from(doc_bitset);
let const_scorer = ConstScorer::new(doc_bitset, boost);

View File

@@ -24,6 +24,13 @@ impl BitSetDocSet {
self.cursor_bucket = bucket_addr;
self.cursor_tinybitset = self.docs.tinyset(bucket_addr);
}
/// Returns the number of documents in the bitset.
///
/// This call is not free: it will bitcount the number of bits in the bitset.
pub fn doc_freq(&self) -> u32 {
self.docs.len() as u32
}
}
impl From<BitSet> for BitSetDocSet {

View File

@@ -1,24 +1,18 @@
use std::collections::HashMap;
use crate::codec::{ObjectSafeCodec, SumOrDoNothingCombiner};
use crate::docset::COLLECT_BLOCK_BUFFER_LEN;
use crate::index::SegmentReader;
use crate::postings::FreqReadingOption;
use crate::query::disjunction::Disjunction;
use crate::query::explanation::does_not_match;
use crate::query::score_combiner::{DoNothingCombiner, ScoreCombiner};
use crate::query::term_query::TermScorer;
use crate::query::weight::{for_each_docset_buffered, for_each_pruning_scorer, for_each_scorer};
use crate::query::weight::for_each_docset_buffered;
use crate::query::{
intersect_scorers, AllScorer, BufferedUnionScorer, EmptyScorer, Exclude, Explanation, Occur,
RequiredOptionalScorer, Scorer, Weight,
box_scorer, intersect_scorers, AllScorer, BufferedUnionScorer, EmptyScorer, Exclude,
Explanation, Occur, RequiredOptionalScorer, Scorer, SumCombiner, Weight,
};
use crate::{DocId, Score};
enum SpecializedScorer {
TermUnion(Vec<TermScorer>),
Other(Box<dyn Scorer>),
}
fn scorer_disjunction<TScoreCombiner>(
scorers: Vec<Box<dyn Scorer>>,
score_combiner: TScoreCombiner,
@@ -32,7 +26,7 @@ where
if scorers.len() == 1 {
return scorers.into_iter().next().unwrap(); // Safe unwrap.
}
Box::new(Disjunction::new(
box_scorer(Disjunction::new(
scorers,
score_combiner,
minimum_match_required,
@@ -44,57 +38,39 @@ fn scorer_union<TScoreCombiner>(
scorers: Vec<Box<dyn Scorer>>,
score_combiner_fn: impl Fn() -> TScoreCombiner,
num_docs: u32,
) -> SpecializedScorer
codec: &dyn ObjectSafeCodec,
) -> Box<dyn Scorer>
where
TScoreCombiner: ScoreCombiner,
{
assert!(!scorers.is_empty());
if scorers.len() == 1 {
return SpecializedScorer::Other(scorers.into_iter().next().unwrap()); //< we checked the size beforehand
}
{
let is_all_term_queries = scorers.iter().all(|scorer| scorer.is::<TermScorer>());
if is_all_term_queries {
let scorers: Vec<TermScorer> = scorers
.into_iter()
.map(|scorer| *(scorer.downcast::<TermScorer>().map_err(|_| ()).unwrap()))
.collect();
if scorers
.iter()
.all(|scorer| scorer.freq_reading_option() == FreqReadingOption::ReadFreq)
match scorers.len() {
0 => box_scorer(EmptyScorer),
1 => scorers.into_iter().next().unwrap(),
_ => {
let combiner_opt: Option<SumOrDoNothingCombiner> = if std::any::TypeId::of::<
TScoreCombiner,
>() == std::any::TypeId::of::<
SumCombiner,
>() {
Some(SumOrDoNothingCombiner::Sum)
} else if std::any::TypeId::of::<TScoreCombiner>()
== std::any::TypeId::of::<DoNothingCombiner>()
{
// Block wand is only available if we read frequencies.
return SpecializedScorer::TermUnion(scorers);
Some(SumOrDoNothingCombiner::DoNothing)
} else {
return SpecializedScorer::Other(Box::new(BufferedUnionScorer::build(
None
};
if let Some(combiner) = combiner_opt {
codec.build_union_scorer_with_sum_combiner(scorers, num_docs, combiner)
} else {
box_scorer(BufferedUnionScorer::build(
scorers,
score_combiner_fn,
num_docs,
)));
))
}
}
}
SpecializedScorer::Other(Box::new(BufferedUnionScorer::build(
scorers,
score_combiner_fn,
num_docs,
)))
}
fn into_box_scorer<TScoreCombiner: ScoreCombiner>(
scorer: SpecializedScorer,
score_combiner_fn: impl Fn() -> TScoreCombiner,
num_docs: u32,
) -> Box<dyn Scorer> {
match scorer {
SpecializedScorer::TermUnion(term_scorers) => {
let union_scorer =
BufferedUnionScorer::build(term_scorers, score_combiner_fn, num_docs);
Box::new(union_scorer)
}
SpecializedScorer::Other(scorer) => scorer,
}
}
/// Returns the effective MUST scorer, accounting for removed AllScorers.
@@ -110,7 +86,7 @@ fn effective_must_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)))
Some(box_scorer(AllScorer::new(max_doc)))
} else {
// No MUST constraint at all
None
@@ -128,28 +104,26 @@ fn effective_must_scorer(
/// 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,
should_scorer: Box<dyn Scorer>,
removed_all_scorer_count: usize,
max_doc: DocId,
num_docs: u32,
score_combiner_fn: impl Fn() -> TScoreCombiner,
scoring_enabled: bool,
) -> SpecializedScorer {
) -> Box<dyn Scorer> {
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(
let all_scorers: Vec<Box<dyn Scorer>> =
vec![should_scorer, box_scorer(AllScorer::new(max_doc))];
box_scorer(BufferedUnionScorer::build(
all_scorers,
score_combiner_fn,
num_docs,
)))
))
} else {
// Scoring disabled - AllScorer alone is sufficient
SpecializedScorer::Other(Box::new(AllScorer::new(max_doc)))
box_scorer(AllScorer::new(max_doc))
}
} else {
should_scorer
@@ -160,9 +134,9 @@ enum ShouldScorersCombinationMethod {
// Should scorers are irrelevant.
Ignored,
// Only contributes to final score.
Optional(SpecializedScorer),
Optional(Box<dyn Scorer>),
// Regardless of score, the should scorers may impact whether a document is matching or not.
Required(SpecializedScorer),
Required(Box<dyn Scorer>),
}
/// Weight associated to the `BoolQuery`.
@@ -224,7 +198,7 @@ impl<TScoreCombiner: ScoreCombiner> BooleanWeight<TScoreCombiner> {
reader: &SegmentReader,
boost: Score,
score_combiner_fn: impl Fn() -> TComplexScoreCombiner,
) -> crate::Result<SpecializedScorer> {
) -> crate::Result<Box<dyn Scorer>> {
let num_docs = reader.num_docs();
let mut per_occur_scorers = self.per_occur_scorers(reader, boost)?;
@@ -234,7 +208,7 @@ impl<TScoreCombiner: ScoreCombiner> BooleanWeight<TScoreCombiner> {
let must_special_scorer_counts = remove_and_count_all_and_empty_scorers(&mut must_scorers);
if must_special_scorer_counts.num_empty_scorers > 0 {
return Ok(SpecializedScorer::Other(Box::new(EmptyScorer)));
return Ok(box_scorer(EmptyScorer));
}
let mut should_scorers = per_occur_scorers.remove(&Occur::Should).unwrap_or_default();
@@ -249,7 +223,7 @@ impl<TScoreCombiner: ScoreCombiner> BooleanWeight<TScoreCombiner> {
if exclude_special_scorer_counts.num_all_scorers > 0 {
// We exclude all documents at one point.
return Ok(SpecializedScorer::Other(Box::new(EmptyScorer)));
return Ok(box_scorer(EmptyScorer));
}
let effective_minimum_number_should_match = self
@@ -261,7 +235,7 @@ impl<TScoreCombiner: ScoreCombiner> BooleanWeight<TScoreCombiner> {
if effective_minimum_number_should_match > num_of_should_scorers {
// We don't have enough scorers to satisfy the minimum number of should matches.
// The request will match no documents.
return Ok(SpecializedScorer::Other(Box::new(EmptyScorer)));
return Ok(box_scorer(EmptyScorer));
}
match effective_minimum_number_should_match {
0 if num_of_should_scorers == 0 => ShouldScorersCombinationMethod::Ignored,
@@ -269,11 +243,13 @@ impl<TScoreCombiner: ScoreCombiner> BooleanWeight<TScoreCombiner> {
should_scorers,
&score_combiner_fn,
num_docs,
reader.codec(),
)),
1 => ShouldScorersCombinationMethod::Required(scorer_union(
should_scorers,
&score_combiner_fn,
num_docs,
reader.codec(),
)),
n if num_of_should_scorers == n => {
// When num_of_should_scorers equals the number of should clauses,
@@ -281,12 +257,10 @@ impl<TScoreCombiner: ScoreCombiner> BooleanWeight<TScoreCombiner> {
must_scorers.append(&mut should_scorers);
ShouldScorersCombinationMethod::Ignored
}
_ => ShouldScorersCombinationMethod::Required(SpecializedScorer::Other(
scorer_disjunction(
should_scorers,
score_combiner_fn(),
effective_minimum_number_should_match,
),
_ => ShouldScorersCombinationMethod::Required(scorer_disjunction(
should_scorers,
score_combiner_fn(),
effective_minimum_number_should_match,
)),
}
};
@@ -294,13 +268,13 @@ impl<TScoreCombiner: ScoreCombiner> BooleanWeight<TScoreCombiner> {
let exclude_scorer_opt: Option<Box<dyn Scorer>> = if exclude_scorers.is_empty() {
None
} else {
let exclude_specialized_scorer: SpecializedScorer =
scorer_union(exclude_scorers, DoNothingCombiner::default, num_docs);
Some(into_box_scorer(
exclude_specialized_scorer,
let exclude_scorers_union: Box<dyn Scorer> = scorer_union(
exclude_scorers,
DoNothingCombiner::default,
num_docs,
))
reader.codec(),
);
Some(exclude_scorers_union)
};
let include_scorer = match (should_scorers, must_scorers) {
@@ -315,8 +289,8 @@ impl<TScoreCombiner: ScoreCombiner> BooleanWeight<TScoreCombiner> {
reader.max_doc(),
num_docs,
)
.unwrap_or_else(|| Box::new(EmptyScorer));
SpecializedScorer::Other(boxed_scorer)
.unwrap_or_else(|| box_scorer(EmptyScorer));
boxed_scorer
}
(ShouldScorersCombinationMethod::Optional(should_scorer), must_scorers) => {
// Optional SHOULD: contributes to scoring but not required for matching.
@@ -341,16 +315,12 @@ impl<TScoreCombiner: ScoreCombiner> BooleanWeight<TScoreCombiner> {
Some(must_scorer) => {
// Has MUST constraint: SHOULD only affects scoring.
if self.scoring_enabled {
SpecializedScorer::Other(Box::new(RequiredOptionalScorer::<
_,
_,
TScoreCombiner,
>::new(
box_scorer(RequiredOptionalScorer::<_, _, TScoreCombiner>::new(
must_scorer,
into_box_scorer(should_scorer, &score_combiner_fn, num_docs),
)))
should_scorer,
))
} else {
SpecializedScorer::Other(must_scorer)
must_scorer
}
}
}
@@ -370,23 +340,13 @@ impl<TScoreCombiner: ScoreCombiner> BooleanWeight<TScoreCombiner> {
}
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,
))
intersect_scorers(vec![must_scorer, should_scorer], num_docs)
}
}
}
};
if let Some(exclude_scorer) = exclude_scorer_opt {
let include_scorer_boxed =
into_box_scorer(include_scorer, &score_combiner_fn, num_docs);
Ok(SpecializedScorer::Other(Box::new(Exclude::new(
include_scorer_boxed,
exclude_scorer,
))))
Ok(box_scorer(Exclude::new(include_scorer, exclude_scorer)))
} else {
Ok(include_scorer)
}
@@ -419,7 +379,6 @@ fn remove_and_count_all_and_empty_scorers(
impl<TScoreCombiner: ScoreCombiner + Sync> Weight for BooleanWeight<TScoreCombiner> {
fn scorer(&self, reader: &SegmentReader, boost: Score) -> crate::Result<Box<dyn Scorer>> {
let num_docs = reader.num_docs();
if self.weights.is_empty() {
Ok(Box::new(EmptyScorer))
} else if self.weights.len() == 1 {
@@ -431,14 +390,8 @@ impl<TScoreCombiner: ScoreCombiner + Sync> Weight for BooleanWeight<TScoreCombin
}
} else if self.scoring_enabled {
self.complex_scorer(reader, boost, &self.score_combiner_fn)
.map(|specialized_scorer| {
into_box_scorer(specialized_scorer, &self.score_combiner_fn, num_docs)
})
} else {
self.complex_scorer(reader, boost, DoNothingCombiner::default)
.map(|specialized_scorer| {
into_box_scorer(specialized_scorer, DoNothingCombiner::default, num_docs)
})
}
}
@@ -467,20 +420,8 @@ impl<TScoreCombiner: ScoreCombiner + Sync> Weight for BooleanWeight<TScoreCombin
reader: &SegmentReader,
callback: &mut dyn FnMut(DocId, Score),
) -> crate::Result<()> {
let scorer = self.complex_scorer(reader, 1.0, &self.score_combiner_fn)?;
match scorer {
SpecializedScorer::TermUnion(term_scorers) => {
let mut union_scorer = BufferedUnionScorer::build(
term_scorers,
&self.score_combiner_fn,
reader.num_docs(),
);
for_each_scorer(&mut union_scorer, callback);
}
SpecializedScorer::Other(mut scorer) => {
for_each_scorer(scorer.as_mut(), callback);
}
}
let mut scorer = self.complex_scorer(reader, 1.0, &self.score_combiner_fn)?;
scorer.for_each(callback);
Ok(())
}
@@ -489,22 +430,9 @@ impl<TScoreCombiner: ScoreCombiner + Sync> Weight for BooleanWeight<TScoreCombin
reader: &SegmentReader,
callback: &mut dyn FnMut(&[DocId]),
) -> crate::Result<()> {
let scorer = self.complex_scorer(reader, 1.0, || DoNothingCombiner)?;
let mut scorer = self.complex_scorer(reader, 1.0, || DoNothingCombiner)?;
let mut buffer = [0u32; COLLECT_BLOCK_BUFFER_LEN];
match scorer {
SpecializedScorer::TermUnion(term_scorers) => {
let mut union_scorer = BufferedUnionScorer::build(
term_scorers,
&self.score_combiner_fn,
reader.num_docs(),
);
for_each_docset_buffered(&mut union_scorer, &mut buffer, callback);
}
SpecializedScorer::Other(mut scorer) => {
for_each_docset_buffered(scorer.as_mut(), &mut buffer, callback);
}
}
for_each_docset_buffered(scorer.as_mut(), &mut buffer, callback);
Ok(())
}
@@ -525,14 +453,7 @@ impl<TScoreCombiner: ScoreCombiner + Sync> Weight for BooleanWeight<TScoreCombin
callback: &mut dyn FnMut(DocId, Score) -> Score,
) -> crate::Result<()> {
let scorer = self.complex_scorer(reader, 1.0, &self.score_combiner_fn)?;
match scorer {
SpecializedScorer::TermUnion(term_scorers) => {
super::block_wand(term_scorers, threshold, callback);
}
SpecializedScorer::Other(mut scorer) => {
for_each_pruning_scorer(scorer.as_mut(), threshold, callback);
}
}
reader.codec().for_each_pruning(threshold, scorer, callback);
Ok(())
}
}

View File

@@ -1,8 +1,6 @@
mod block_wand;
mod boolean_query;
mod boolean_weight;
pub(crate) use self::block_wand::{block_wand, block_wand_single_scorer};
pub use self::boolean_query::BooleanQuery;
pub use self::boolean_weight::BooleanWeight;

View File

@@ -1,7 +1,7 @@
use std::fmt;
use crate::docset::COLLECT_BLOCK_BUFFER_LEN;
use crate::query::{EnableScoring, Explanation, Query, Scorer, Weight};
use crate::query::{box_scorer, EnableScoring, Explanation, Query, Scorer, Weight};
use crate::{DocId, DocSet, Score, SegmentReader, TantivyError, Term};
/// `ConstScoreQuery` is a wrapper over a query to provide a constant score.
@@ -65,7 +65,10 @@ impl ConstWeight {
impl Weight for ConstWeight {
fn scorer(&self, reader: &SegmentReader, boost: Score) -> crate::Result<Box<dyn Scorer>> {
let inner_scorer = self.weight.scorer(reader, boost)?;
Ok(Box::new(ConstScorer::new(inner_scorer, boost * self.score)))
Ok(box_scorer(ConstScorer::new(
inner_scorer,
boost * self.score,
)))
}
fn explain(&self, reader: &SegmentReader, doc: u32) -> crate::Result<Explanation> {

View File

@@ -2,7 +2,7 @@ use super::Scorer;
use crate::docset::TERMINATED;
use crate::index::SegmentReader;
use crate::query::explanation::does_not_match;
use crate::query::{EnableScoring, Explanation, Query, Weight};
use crate::query::{box_scorer, EnableScoring, Explanation, Query, Weight};
use crate::{DocId, DocSet, Score, Searcher};
/// `EmptyQuery` is a dummy `Query` in which no document matches.
@@ -27,7 +27,7 @@ impl Query for EmptyQuery {
pub struct EmptyWeight;
impl Weight for EmptyWeight {
fn scorer(&self, _reader: &SegmentReader, _boost: Score) -> crate::Result<Box<dyn Scorer>> {
Ok(Box::new(EmptyScorer))
Ok(box_scorer(EmptyScorer))
}
fn explain(&self, _reader: &SegmentReader, doc: DocId) -> crate::Result<Explanation> {

View File

@@ -3,7 +3,7 @@ use core::fmt::Debug;
use columnar::{ColumnIndex, DynamicColumn};
use common::BitSet;
use super::{ConstScorer, EmptyScorer};
use super::{box_scorer, ConstScorer, EmptyScorer};
use crate::docset::{DocSet, TERMINATED};
use crate::index::SegmentReader;
use crate::query::all_query::AllScorer;
@@ -117,7 +117,7 @@ impl Weight for ExistsWeight {
}
}
if non_empty_columns.is_empty() {
return Ok(Box::new(EmptyScorer));
return Ok(box_scorer(EmptyScorer));
}
// If any column is full, all docs match.
@@ -128,9 +128,9 @@ impl Weight for ExistsWeight {
{
let all_scorer = AllScorer::new(max_doc);
if boost != 1.0f32 {
return Ok(Box::new(BoostScorer::new(all_scorer, boost)));
return Ok(box_scorer(BoostScorer::new(all_scorer, boost)));
} else {
return Ok(Box::new(all_scorer));
return Ok(box_scorer(all_scorer));
}
}
@@ -138,7 +138,7 @@ impl Weight for ExistsWeight {
// NOTE: A lower number may be better for very sparse columns
if non_empty_columns.len() < 4 {
let docset = ExistsDocSet::new(non_empty_columns, reader.max_doc());
return Ok(Box::new(ConstScorer::new(docset, boost)));
return Ok(box_scorer(ConstScorer::new(docset, boost)));
}
// If we have many dynamic columns, precompute a bitset of matching docs
@@ -162,7 +162,7 @@ impl Weight for ExistsWeight {
}
}
let docset = BitSetDocSet::from(doc_bitset);
Ok(Box::new(ConstScorer::new(docset, boost)))
Ok(box_scorer(ConstScorer::new(docset, boost)))
}
fn explain(&self, reader: &SegmentReader, doc: DocId) -> crate::Result<Explanation> {

View File

@@ -1,7 +1,7 @@
use super::size_hint::estimate_intersection;
use crate::docset::{DocSet, SeekDangerResult, TERMINATED};
use crate::query::term_query::TermScorer;
use crate::query::{EmptyScorer, Scorer};
use crate::query::{box_scorer, EmptyScorer, Scorer};
use crate::{DocId, Score};
/// Returns the intersection scorer.
@@ -20,7 +20,7 @@ pub fn intersect_scorers(
num_docs_segment: u32,
) -> Box<dyn Scorer> {
if scorers.is_empty() {
return Box::new(EmptyScorer);
return box_scorer(EmptyScorer);
}
if scorers.len() == 1 {
return scorers.pop().unwrap();
@@ -29,7 +29,7 @@ pub fn intersect_scorers(
scorers.sort_by_key(|scorer| scorer.cost());
let doc = go_to_first_doc(&mut scorers[..]);
if doc == TERMINATED {
return Box::new(EmptyScorer);
return box_scorer(EmptyScorer);
}
// We know that we have at least 2 elements.
let left = scorers.remove(0);
@@ -38,14 +38,14 @@ pub fn intersect_scorers(
.iter()
.all(|&scorer| scorer.is::<TermScorer>());
if all_term_scorers {
return Box::new(Intersection {
return box_scorer(Intersection {
left: *(left.downcast::<TermScorer>().map_err(|_| ()).unwrap()),
right: *(right.downcast::<TermScorer>().map_err(|_| ()).unwrap()),
others: scorers,
num_docs: num_docs_segment,
});
}
Box::new(Intersection {
box_scorer(Intersection {
left,
right,
others: scorers,
@@ -84,6 +84,14 @@ impl<TDocSet: DocSet> Intersection<TDocSet, TDocSet> {
docsets.sort_by_key(|docset| docset.cost());
go_to_first_doc(&mut docsets);
let left = docsets.remove(0);
debug_assert!({
let doc = left.doc();
if doc == TERMINATED {
true
} else {
docsets.iter().all(|docset| docset.doc() == doc)
}
});
let right = docsets.remove(0);
Intersection {
left,
@@ -112,30 +120,24 @@ impl<TDocSet: DocSet, TOtherDocSet: DocSet> DocSet for Intersection<TDocSet, TOt
// Invariant:
// - candidate is always <= to the next document in the intersection.
// - candidate strictly increases at every occurence of the loop.
let mut candidate = 0;
let mut candidate = left.doc() + 1;
// Termination: candidate strictly increases.
'outer: while candidate < TERMINATED {
// As we enter the loop, we should always have candidate < next_doc.
// This step always increases candidate.
//
// TODO: Think about which value would make sense here
// It depends on the DocSet implementation, when a seek would outweigh an advance.
candidate = if candidate > left.doc().wrapping_add(100) {
left.seek(candidate)
} else {
left.advance()
};
candidate = left.seek(candidate);
// Left is positionned on `candidate`.
debug_assert_eq!(left.doc(), candidate);
if let SeekDangerResult::SeekLowerBound(seek_lower_bound) = right.seek_danger(candidate)
{
// The max is technically useless but it makes the invariant
// easier to proofread.
debug_assert!(seek_lower_bound >= candidate);
debug_assert!(
seek_lower_bound == TERMINATED || seek_lower_bound > candidate,
"seek_lower_bound {seek_lower_bound} must be greater than candidate \
{candidate}"
);
candidate = seek_lower_bound;
continue;
}
@@ -148,7 +150,11 @@ impl<TDocSet: DocSet, TOtherDocSet: DocSet> DocSet for Intersection<TDocSet, TOt
other.seek_danger(candidate)
{
// One of the scorer does not match, let's restart at the top of the loop.
debug_assert!(seek_lower_bound >= candidate);
debug_assert!(
seek_lower_bound == TERMINATED || seek_lower_bound > candidate,
"seek_lower_bound {seek_lower_bound} must be greater than candidate \
{candidate}"
);
candidate = seek_lower_bound;
continue 'outer;
}
@@ -238,9 +244,12 @@ mod tests {
use proptest::prelude::*;
use super::Intersection;
use crate::collector::Count;
use crate::docset::{DocSet, TERMINATED};
use crate::postings::tests::test_skip_against_unoptimized;
use crate::query::VecDocSet;
use crate::query::{QueryParser, VecDocSet};
use crate::schema::{Schema, TEXT};
use crate::Index;
#[test]
fn test_intersection() {
@@ -411,4 +420,29 @@ mod tests {
assert_eq!(intersection.doc(), TERMINATED);
}
}
#[test]
fn test_bug_2811_intersection_candidate_should_increase() {
let mut schema_builder = Schema::builder();
let text_field = schema_builder.add_text_field("text", TEXT);
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
let mut writer = index.writer_for_tests().unwrap();
writer
.add_document(doc!(text_field=>"hello happy tax"))
.unwrap();
writer.add_document(doc!(text_field=>"hello")).unwrap();
writer.add_document(doc!(text_field=>"hello")).unwrap();
writer.add_document(doc!(text_field=>"happy tax")).unwrap();
writer.commit().unwrap();
let query_parser = QueryParser::for_index(&index, Vec::new());
let query = query_parser
.parse_query(r#"+text:hello +text:"happy tax""#)
.unwrap();
let searcher = index.reader().unwrap().searcher();
let c = searcher.search(&*query, &Count).unwrap();
assert_eq!(c, 1);
}
}

View File

@@ -24,7 +24,7 @@ mod reqopt_scorer;
mod scorer;
mod set_query;
mod size_hint;
mod term_query;
pub(crate) mod term_query;
mod union;
mod weight;
@@ -54,13 +54,14 @@ pub use self::more_like_this::{MoreLikeThisQuery, MoreLikeThisQueryBuilder};
pub use self::phrase_prefix_query::PhrasePrefixQuery;
pub use self::phrase_query::regex_phrase_query::{wildcard_query_to_regex_str, RegexPhraseQuery};
pub use self::phrase_query::PhraseQuery;
pub(crate) use self::phrase_query::PhraseScorer;
pub use self::query::{EnableScoring, Query, QueryClone};
pub use self::query_parser::{QueryParser, QueryParserError};
pub use self::range_query::*;
pub use self::regex_query::RegexQuery;
pub use self::reqopt_scorer::RequiredOptionalScorer;
pub use self::score_combiner::{DisjunctionMaxCombiner, ScoreCombiner, SumCombiner};
pub use self::scorer::Scorer;
pub use self::scorer::{box_scorer, Scorer};
pub use self::set_query::TermSetQuery;
pub use self::term_query::TermQuery;
pub use self::union::BufferedUnionScorer;

View File

@@ -2,7 +2,7 @@ use crate::docset::{DocSet, SeekDangerResult, TERMINATED};
use crate::fieldnorm::FieldNormReader;
use crate::postings::Postings;
use crate::query::bm25::Bm25Weight;
use crate::query::phrase_query::{intersection_count, PhraseScorer};
use crate::query::phrase_query::{intersection_exists, PhraseScorer};
use crate::query::Scorer;
use crate::{DocId, Score};
@@ -100,7 +100,6 @@ pub struct PhrasePrefixScorer<TPostings: Postings> {
phrase_scorer: PhraseKind<TPostings>,
suffixes: Vec<TPostings>,
suffix_offset: u32,
phrase_count: u32,
suffix_position_buffer: Vec<u32>,
}
@@ -144,7 +143,6 @@ impl<TPostings: Postings> PhrasePrefixScorer<TPostings> {
phrase_scorer,
suffixes,
suffix_offset: (max_offset - suffix_pos) as u32,
phrase_count: 0,
suffix_position_buffer: Vec::with_capacity(100),
};
if phrase_prefix_scorer.doc() != TERMINATED && !phrase_prefix_scorer.matches_prefix() {
@@ -153,12 +151,7 @@ impl<TPostings: Postings> PhrasePrefixScorer<TPostings> {
phrase_prefix_scorer
}
pub fn phrase_count(&self) -> u32 {
self.phrase_count
}
fn matches_prefix(&mut self) -> bool {
let mut count = 0;
let current_doc = self.doc();
let pos_matching = self.phrase_scorer.get_intersection();
for suffix in &mut self.suffixes {
@@ -168,11 +161,12 @@ impl<TPostings: Postings> PhrasePrefixScorer<TPostings> {
let doc = suffix.seek(current_doc);
if doc == current_doc {
suffix.positions_with_offset(self.suffix_offset, &mut self.suffix_position_buffer);
count += intersection_count(pos_matching, &self.suffix_position_buffer);
if intersection_exists(pos_matching, &self.suffix_position_buffer) {
return true;
}
}
}
self.phrase_count = count as u32;
count != 0
false
}
}

View File

@@ -1,12 +1,11 @@
use super::{prefix_end, PhrasePrefixScorer};
use crate::fieldnorm::FieldNormReader;
use crate::index::SegmentReader;
use crate::postings::SegmentPostings;
use crate::postings::Postings;
use crate::query::bm25::Bm25Weight;
use crate::query::explanation::does_not_match;
use crate::query::{EmptyScorer, Explanation, Scorer, Weight};
use crate::query::{box_scorer, EmptyScorer, Scorer, Weight};
use crate::schema::{IndexRecordOption, Term};
use crate::{DocId, DocSet, Score};
use crate::Score;
pub struct PhrasePrefixWeight {
phrase_terms: Vec<(usize, Term)>,
@@ -46,13 +45,13 @@ impl PhrasePrefixWeight {
&self,
reader: &SegmentReader,
boost: Score,
) -> crate::Result<Option<PhrasePrefixScorer<SegmentPostings>>> {
) -> crate::Result<Option<Box<dyn Scorer>>> {
let similarity_weight_opt = self
.similarity_weight_opt
.as_ref()
.map(|similarity_weight| similarity_weight.boost_by(boost));
let fieldnorm_reader = self.fieldnorm_reader(reader)?;
let mut term_postings_list = Vec::new();
let mut term_postings_list: Vec<(usize, Box<dyn Postings>)> = Vec::new();
for &(offset, ref term) in &self.phrase_terms {
if let Some(postings) = reader
.inverted_index(term.field())?
@@ -103,49 +102,32 @@ impl PhrasePrefixWeight {
}
}
Ok(Some(PhrasePrefixScorer::new(
Ok(Some(box_scorer(PhrasePrefixScorer::new(
term_postings_list,
similarity_weight_opt,
fieldnorm_reader,
suffixes,
self.prefix.0,
)))
))))
}
}
impl Weight for PhrasePrefixWeight {
fn scorer(&self, reader: &SegmentReader, boost: Score) -> crate::Result<Box<dyn Scorer>> {
if let Some(scorer) = self.phrase_scorer(reader, boost)? {
Ok(Box::new(scorer))
Ok(scorer)
} else {
Ok(Box::new(EmptyScorer))
Ok(box_scorer(EmptyScorer))
}
}
fn explain(&self, reader: &SegmentReader, doc: DocId) -> crate::Result<Explanation> {
let scorer_opt = self.phrase_scorer(reader, 1.0)?;
if scorer_opt.is_none() {
return Err(does_not_match(doc));
}
let mut scorer = scorer_opt.unwrap();
if scorer.seek(doc) != doc {
return Err(does_not_match(doc));
}
let fieldnorm_reader = self.fieldnorm_reader(reader)?;
let fieldnorm_id = fieldnorm_reader.fieldnorm_id(doc);
let phrase_count = scorer.phrase_count();
let mut explanation = Explanation::new("Phrase Prefix Scorer", scorer.score());
if let Some(similarity_weight) = self.similarity_weight_opt.as_ref() {
explanation.add_detail(similarity_weight.explain(fieldnorm_id, phrase_count));
}
Ok(explanation)
}
}
#[cfg(test)]
mod tests {
use crate::docset::TERMINATED;
use crate::index::Index;
use crate::postings::Postings;
use crate::query::phrase_prefix_query::PhrasePrefixScorer;
use crate::query::{EnableScoring, PhrasePrefixQuery, Query};
use crate::schema::{Schema, TEXT};
use crate::{DocSet, IndexWriter, Term};
@@ -186,14 +168,14 @@ mod tests {
.phrase_prefix_query_weight(enable_scoring)
.unwrap()
.unwrap();
let mut phrase_scorer = phrase_weight
let mut phrase_scorer_boxed = phrase_weight
.phrase_scorer(searcher.segment_reader(0u32), 1.0)?
.unwrap();
let phrase_scorer: &mut PhrasePrefixScorer<Box<dyn Postings>> =
phrase_scorer_boxed.as_any_mut().downcast_mut().unwrap();
assert_eq!(phrase_scorer.doc(), 1);
assert_eq!(phrase_scorer.phrase_count(), 2);
assert_eq!(phrase_scorer.advance(), 2);
assert_eq!(phrase_scorer.doc(), 2);
assert_eq!(phrase_scorer.phrase_count(), 1);
assert_eq!(phrase_scorer.advance(), TERMINATED);
Ok(())
}
@@ -213,14 +195,15 @@ mod tests {
.phrase_prefix_query_weight(enable_scoring)
.unwrap()
.unwrap();
let mut phrase_scorer = phrase_weight
let mut phrase_scorer_boxed = phrase_weight
.phrase_scorer(searcher.segment_reader(0u32), 1.0)?
.unwrap();
let phrase_scorer = phrase_scorer_boxed
.downcast_mut::<PhrasePrefixScorer<Box<dyn Postings>>>()
.unwrap();
assert_eq!(phrase_scorer.doc(), 1);
assert_eq!(phrase_scorer.phrase_count(), 2);
assert_eq!(phrase_scorer.advance(), 2);
assert_eq!(phrase_scorer.doc(), 2);
assert_eq!(phrase_scorer.phrase_count(), 1);
assert_eq!(phrase_scorer.advance(), TERMINATED);
Ok(())
}

View File

@@ -5,7 +5,7 @@ pub mod regex_phrase_query;
mod regex_phrase_weight;
pub use self::phrase_query::PhraseQuery;
pub(crate) use self::phrase_scorer::intersection_count;
pub(crate) use self::phrase_scorer::intersection_exists;
pub use self::phrase_scorer::PhraseScorer;
pub use self::phrase_weight::PhraseWeight;

View File

@@ -126,7 +126,7 @@ impl PhraseQuery {
};
let mut weight = PhraseWeight::new(self.phrase_terms.clone(), bm25_weight_opt);
if self.slop > 0 {
weight.slop(self.slop);
weight.set_slop(self.slop);
}
Ok(weight)
}

View File

@@ -1,10 +1,11 @@
use std::cmp::Ordering;
use crate::codec::standard::postings::StandardPostings;
use crate::docset::{DocSet, SeekDangerResult, TERMINATED};
use crate::fieldnorm::FieldNormReader;
use crate::postings::Postings;
use crate::query::bm25::Bm25Weight;
use crate::query::{Intersection, Scorer};
use crate::query::{Explanation, Intersection, Scorer};
use crate::{DocId, Score};
struct PostingsWithOffset<TPostings> {
@@ -43,7 +44,7 @@ impl<TPostings: Postings> DocSet for PostingsWithOffset<TPostings> {
}
}
pub struct PhraseScorer<TPostings: Postings> {
pub struct PhraseScorer<TPostings: Postings = StandardPostings> {
intersection_docset: Intersection<PostingsWithOffset<TPostings>, PostingsWithOffset<TPostings>>,
num_terms: usize,
left_positions: Vec<u32>,
@@ -58,7 +59,7 @@ pub struct PhraseScorer<TPostings: Postings> {
}
/// Returns true if and only if the two sorted arrays contain a common element
fn intersection_exists(left: &[u32], right: &[u32]) -> bool {
pub(crate) fn intersection_exists(left: &[u32], right: &[u32]) -> bool {
let mut left_index = 0;
let mut right_index = 0;
while left_index < left.len() && right_index < right.len() {
@@ -79,7 +80,7 @@ fn intersection_exists(left: &[u32], right: &[u32]) -> bool {
false
}
pub(crate) fn intersection_count(left: &[u32], right: &[u32]) -> usize {
fn intersection_count(left: &[u32], right: &[u32]) -> usize {
let mut left_index = 0;
let mut right_index = 0;
let mut count = 0;
@@ -402,6 +403,7 @@ impl<TPostings: Postings> PhraseScorer<TPostings> {
scorer
}
/// Returns the number of phrases identified in the current matching doc.
pub fn phrase_count(&self) -> u32 {
self.phrase_count
}
@@ -531,7 +533,12 @@ impl<TPostings: Postings> DocSet for PhraseScorer<TPostings> {
}
fn seek_danger(&mut self, target: DocId) -> SeekDangerResult {
debug_assert!(target >= self.doc());
debug_assert!(
target >= self.doc(),
"target ({}) should be greater than or equal to doc ({})",
target,
self.doc()
);
let seek_res = self.intersection_docset.seek_danger(target);
if seek_res != SeekDangerResult::Found {
return seek_res;
@@ -579,6 +586,17 @@ impl<TPostings: Postings> Scorer for PhraseScorer<TPostings> {
1.0f32
}
}
fn explain(&mut self) -> Explanation {
let doc = self.doc();
let phrase_count = self.phrase_count();
let fieldnorm_id = self.fieldnorm_reader.fieldnorm_id(doc);
let mut explanation = Explanation::new("Phrase Scorer", self.score());
if let Some(similarity_weight) = self.similarity_weight_opt.as_ref() {
explanation.add_detail(similarity_weight.explain(fieldnorm_id, phrase_count));
}
explanation
}
}
#[cfg(test)]

View File

@@ -1,11 +1,10 @@
use super::PhraseScorer;
use crate::fieldnorm::FieldNormReader;
use crate::index::SegmentReader;
use crate::postings::SegmentPostings;
use crate::postings::TermInfo;
use crate::query::bm25::Bm25Weight;
use crate::query::explanation::does_not_match;
use crate::query::{EmptyScorer, Explanation, Scorer, Weight};
use crate::schema::{IndexRecordOption, Term};
use crate::query::{box_scorer, EmptyScorer, Explanation, Scorer, Weight};
use crate::schema::Term;
use crate::{DocId, DocSet, Score};
pub struct PhraseWeight {
@@ -21,11 +20,10 @@ impl PhraseWeight {
phrase_terms: Vec<(usize, Term)>,
similarity_weight_opt: Option<Bm25Weight>,
) -> PhraseWeight {
let slop = 0;
PhraseWeight {
phrase_terms,
similarity_weight_opt,
slop,
slop: 0,
}
}
@@ -43,32 +41,52 @@ impl PhraseWeight {
&self,
reader: &SegmentReader,
boost: Score,
) -> crate::Result<Option<PhraseScorer<SegmentPostings>>> {
) -> crate::Result<Option<Box<dyn Scorer>>> {
let similarity_weight_opt = self
.similarity_weight_opt
.as_ref()
.map(|similarity_weight| similarity_weight.boost_by(boost));
let fieldnorm_reader = self.fieldnorm_reader(reader)?;
let mut term_postings_list = Vec::new();
for &(offset, ref term) in &self.phrase_terms {
if let Some(postings) = reader
.inverted_index(term.field())?
.read_postings(term, IndexRecordOption::WithFreqsAndPositions)?
{
term_postings_list.push((offset, postings));
} else {
return Ok(None);
}
if self.phrase_terms.is_empty() {
return Ok(None);
}
Ok(Some(PhraseScorer::new(
term_postings_list,
let field = self.phrase_terms[0].1.field();
if !self
.phrase_terms
.iter()
.all(|(_offset, term)| term.field() == field)
{
return Err(crate::TantivyError::InvalidArgument(
"All terms in a phrase query must belong to the same field".to_string(),
));
}
let inverted_index_reader = reader.inverted_index(field)?;
let mut term_infos: Vec<(usize, TermInfo)> = Vec::with_capacity(self.phrase_terms.len());
for &(offset, ref term) in &self.phrase_terms {
let Some(term_info) = inverted_index_reader.get_term_info(term)? else {
return Ok(None);
};
term_infos.push((offset, term_info));
}
let scorer = reader.codec().new_phrase_scorer_type_erased(
&term_infos[..],
similarity_weight_opt,
fieldnorm_reader,
self.slop,
)))
inverted_index_reader.as_ref(),
)?;
Ok(Some(scorer))
}
pub fn slop(&mut self, slop: u32) {
/// Sets the slop for the given PhraseWeight.
pub fn set_slop(&mut self, slop: u32) {
self.slop = slop;
}
}
@@ -76,9 +94,9 @@ impl PhraseWeight {
impl Weight for PhraseWeight {
fn scorer(&self, reader: &SegmentReader, boost: Score) -> crate::Result<Box<dyn Scorer>> {
if let Some(scorer) = self.phrase_scorer(reader, boost)? {
Ok(Box::new(scorer))
Ok(scorer)
} else {
Ok(Box::new(EmptyScorer))
Ok(box_scorer(EmptyScorer))
}
}
@@ -91,14 +109,7 @@ impl Weight for PhraseWeight {
if scorer.seek(doc) != doc {
return Err(does_not_match(doc));
}
let fieldnorm_reader = self.fieldnorm_reader(reader)?;
let fieldnorm_id = fieldnorm_reader.fieldnorm_id(doc);
let phrase_count = scorer.phrase_count();
let mut explanation = Explanation::new("Phrase Scorer", scorer.score());
if let Some(similarity_weight) = self.similarity_weight_opt.as_ref() {
explanation.add_detail(similarity_weight.explain(fieldnorm_id, phrase_count));
}
Ok(explanation)
Ok(scorer.explain())
}
}
@@ -106,7 +117,8 @@ impl Weight for PhraseWeight {
mod tests {
use super::super::tests::create_index;
use crate::docset::TERMINATED;
use crate::query::{EnableScoring, PhraseQuery};
use crate::query::phrase_query::PhraseScorer;
use crate::query::{EnableScoring, PhraseQuery, Scorer};
use crate::{DocSet, Term};
#[test]
@@ -121,9 +133,11 @@ mod tests {
]);
let enable_scoring = EnableScoring::enabled_from_searcher(&searcher);
let phrase_weight = phrase_query.phrase_weight(enable_scoring).unwrap();
let mut phrase_scorer = phrase_weight
let phrase_scorer_boxed: Box<dyn Scorer> = phrase_weight
.phrase_scorer(searcher.segment_reader(0u32), 1.0)?
.unwrap();
let mut phrase_scorer: Box<PhraseScorer> =
phrase_scorer_boxed.downcast::<PhraseScorer>().ok().unwrap();
assert_eq!(phrase_scorer.doc(), 1);
assert_eq!(phrase_scorer.phrase_count(), 2);
assert_eq!(phrase_scorer.advance(), 2);

View File

@@ -6,11 +6,13 @@ use tantivy_fst::Regex;
use super::PhraseScorer;
use crate::fieldnorm::FieldNormReader;
use crate::index::SegmentReader;
use crate::postings::{LoadedPostings, Postings, SegmentPostings, TermInfo};
use crate::postings::{LoadedPostings, Postings, TermInfo};
use crate::query::bm25::Bm25Weight;
use crate::query::explanation::does_not_match;
use crate::query::union::{BitSetPostingUnion, SimpleUnion};
use crate::query::{AutomatonWeight, BitSetDocSet, EmptyScorer, Explanation, Scorer, Weight};
use crate::query::{
box_scorer, AutomatonWeight, BitSetDocSet, EmptyScorer, Explanation, Scorer, Weight,
};
use crate::schema::{Field, IndexRecordOption};
use crate::{DocId, DocSet, InvertedIndexReader, Score};
@@ -84,7 +86,8 @@ impl RegexPhraseWeight {
"Phrase query exceeded max expansions {num_terms}"
)));
}
let union = Self::get_union_from_term_infos(&term_infos, reader, &inverted_index)?;
let union =
Self::get_union_from_term_infos(&term_infos, reader, inverted_index.as_ref())?;
posting_lists.push((offset, union));
}
@@ -99,22 +102,13 @@ impl RegexPhraseWeight {
/// Add all docs of the term to the docset
fn add_to_bitset(
inverted_index: &InvertedIndexReader,
inverted_index: &dyn InvertedIndexReader,
term_info: &TermInfo,
doc_bitset: &mut BitSet,
) -> crate::Result<()> {
let mut block_segment_postings = inverted_index
.read_block_postings_from_terminfo(term_info, IndexRecordOption::Basic)?;
loop {
let docs = block_segment_postings.docs();
if docs.is_empty() {
break;
}
for &doc in docs {
doc_bitset.insert(doc);
}
block_segment_postings.advance();
}
let mut segment_postings =
inverted_index.read_postings_from_terminfo(term_info, IndexRecordOption::Basic)?;
segment_postings.fill_bitset(doc_bitset);
Ok(())
}
@@ -175,7 +169,7 @@ impl RegexPhraseWeight {
pub(crate) fn get_union_from_term_infos(
term_infos: &[TermInfo],
reader: &SegmentReader,
inverted_index: &InvertedIndexReader,
inverted_index: &dyn InvertedIndexReader,
) -> crate::Result<UnionType> {
let max_doc = reader.max_doc();
@@ -188,7 +182,7 @@ impl RegexPhraseWeight {
// - Bucket 1: Terms appearing in 0.1% to 1% of documents
// - Bucket 2: Terms appearing in 1% to 10% of documents
// - Bucket 3: Terms appearing in more than 10% of documents
let mut buckets: Vec<(BitSet, Vec<SegmentPostings>)> = (0..4)
let mut buckets: Vec<(BitSet, Vec<Box<dyn Postings>>)> = (0..4)
.map(|_| (BitSet::with_max_value(max_doc), Vec::new()))
.collect();
@@ -197,7 +191,7 @@ impl RegexPhraseWeight {
for term_info in term_infos {
let mut term_posting = inverted_index
.read_postings_from_terminfo(term_info, IndexRecordOption::WithFreqsAndPositions)?;
let num_docs = term_posting.doc_freq();
let num_docs = u32::from(term_posting.doc_freq());
if num_docs < SPARSE_TERM_DOC_THRESHOLD {
let current_bucket = &mut sparse_buckets[0];
@@ -271,9 +265,9 @@ impl RegexPhraseWeight {
impl Weight for RegexPhraseWeight {
fn scorer(&self, reader: &SegmentReader, boost: Score) -> crate::Result<Box<dyn Scorer>> {
if let Some(scorer) = self.phrase_scorer(reader, boost)? {
Ok(Box::new(scorer))
Ok(box_scorer(scorer))
} else {
Ok(Box::new(EmptyScorer))
Ok(box_scorer(EmptyScorer))
}
}

View File

@@ -8,10 +8,12 @@ use super::range_query_fastfield::FastFieldRangeWeight;
use crate::index::SegmentReader;
use crate::query::explanation::does_not_match;
use crate::query::range_query::is_type_valid_for_fastfield_range_query;
use crate::query::{BitSetDocSet, ConstScorer, EnableScoring, Explanation, Query, Scorer, Weight};
use crate::query::{
box_scorer, BitSetDocSet, ConstScorer, EnableScoring, Explanation, Query, Scorer, Weight,
};
use crate::schema::{Field, IndexRecordOption, Term, Type};
use crate::termdict::{TermDictionary, TermStreamer};
use crate::{DocId, Score};
use crate::{DocId, DocSet, Score};
/// `RangeQuery` matches all documents that have at least one term within a defined range.
///
@@ -228,21 +230,12 @@ impl Weight for InvertedIndexRangeWeight {
}
processed_count += 1;
let term_info = term_range.value();
let mut block_segment_postings = inverted_index
.read_block_postings_from_terminfo(term_info, IndexRecordOption::Basic)?;
loop {
let docs = block_segment_postings.docs();
if docs.is_empty() {
break;
}
for &doc in block_segment_postings.docs() {
doc_bitset.insert(doc);
}
block_segment_postings.advance();
}
let mut postings =
inverted_index.read_postings_from_terminfo(term_info, IndexRecordOption::Basic)?;
postings.fill_bitset(&mut doc_bitset);
}
let doc_bitset = BitSetDocSet::from(doc_bitset);
Ok(Box::new(ConstScorer::new(doc_bitset, boost)))
Ok(box_scorer(ConstScorer::new(doc_bitset, boost)))
}
fn explain(&self, reader: &SegmentReader, doc: DocId) -> crate::Result<Explanation> {

View File

@@ -13,7 +13,8 @@ use common::bounds::{BoundsRange, TransformBound};
use super::fast_field_range_doc_set::RangeDocSet;
use crate::query::{
AllScorer, ConstScorer, EmptyScorer, EnableScoring, Explanation, Query, Scorer, Weight,
box_scorer, AllScorer, ConstScorer, EmptyScorer, EnableScoring, Explanation, Query, Scorer,
Weight,
};
use crate::schema::{Type, ValueBytes};
use crate::{DocId, DocSet, Score, SegmentReader, TantivyError, Term};
@@ -55,7 +56,7 @@ impl Weight for FastFieldRangeWeight {
fn scorer(&self, reader: &SegmentReader, boost: Score) -> crate::Result<Box<dyn Scorer>> {
// Check if both bounds are Bound::Unbounded
if self.bounds.is_unbounded() {
return Ok(Box::new(AllScorer::new(reader.max_doc())));
return Ok(box_scorer(AllScorer::new(reader.max_doc())));
}
let term = self
@@ -95,7 +96,7 @@ impl Weight for FastFieldRangeWeight {
let Some(str_dict_column): Option<StrColumn> =
reader.fast_fields().str(&field_name)?
else {
return Ok(Box::new(EmptyScorer));
return Ok(box_scorer(EmptyScorer));
};
let dict = str_dict_column.dictionary();
@@ -107,7 +108,7 @@ impl Weight for FastFieldRangeWeight {
let Some((column, _col_type)) = fast_field_reader
.u64_lenient_for_type(Some(&[ColumnType::Str]), &field_name)?
else {
return Ok(Box::new(EmptyScorer));
return Ok(box_scorer(EmptyScorer));
};
search_on_u64_ff(column, boost, BoundsRange::new(lower_bound, upper_bound))
}
@@ -119,7 +120,7 @@ impl Weight for FastFieldRangeWeight {
let Some((column, _col_type)) = fast_field_reader
.u64_lenient_for_type(Some(&[ColumnType::DateTime]), &field_name)?
else {
return Ok(Box::new(EmptyScorer));
return Ok(box_scorer(EmptyScorer));
};
let bounds = bounds.map_bound(|term| term.as_date().unwrap().to_u64());
search_on_u64_ff(
@@ -146,7 +147,7 @@ impl Weight for FastFieldRangeWeight {
let Some(ip_addr_column): Option<Column<Ipv6Addr>> =
reader.fast_fields().column_opt(&field_name)?
else {
return Ok(Box::new(EmptyScorer));
return Ok(box_scorer(EmptyScorer));
};
let value_range = bound_range_inclusive_ip(
&bounds.lower_bound,
@@ -155,11 +156,11 @@ impl Weight for FastFieldRangeWeight {
ip_addr_column.max_value(),
);
let docset = RangeDocSet::new(value_range, ip_addr_column);
Ok(Box::new(ConstScorer::new(docset, boost)))
Ok(box_scorer(ConstScorer::new(docset, boost)))
} else if field_type.is_str() {
let Some(str_dict_column): Option<StrColumn> = reader.fast_fields().str(&field_name)?
else {
return Ok(Box::new(EmptyScorer));
return Ok(box_scorer(EmptyScorer));
};
let dict = str_dict_column.dictionary();
@@ -171,7 +172,7 @@ impl Weight for FastFieldRangeWeight {
let Some((column, _col_type)) =
fast_field_reader.u64_lenient_for_type(None, &field_name)?
else {
return Ok(Box::new(EmptyScorer));
return Ok(box_scorer(EmptyScorer));
};
search_on_u64_ff(column, boost, BoundsRange::new(lower_bound, upper_bound))
} else {
@@ -209,7 +210,7 @@ impl Weight for FastFieldRangeWeight {
&field_name,
)?
else {
return Ok(Box::new(EmptyScorer));
return Ok(box_scorer(EmptyScorer));
};
search_on_u64_ff(
column,
@@ -250,7 +251,7 @@ fn search_on_json_numerical_field(
let Some((column, col_type)) =
fast_field_reader.u64_lenient_for_type(allowed_column_types, field_name)?
else {
return Ok(Box::new(EmptyScorer));
return Ok(box_scorer(EmptyScorer));
};
let actual_column_type: NumericalType = col_type
.numerical_type()
@@ -408,18 +409,18 @@ fn search_on_u64_ff(
)
.unwrap_or(1..=0); // empty range
if value_range.is_empty() {
return Ok(Box::new(EmptyScorer));
return Ok(box_scorer(EmptyScorer));
}
if col_min_value >= *value_range.start() && col_max_value <= *value_range.end() {
// all values in the column are within the range.
if column.index.get_cardinality() == Cardinality::Full {
if boost != 1.0f32 {
return Ok(Box::new(ConstScorer::new(
return Ok(box_scorer(ConstScorer::new(
AllScorer::new(column.num_docs()),
boost,
)));
} else {
return Ok(Box::new(AllScorer::new(column.num_docs())));
return Ok(box_scorer(AllScorer::new(column.num_docs())));
}
} else {
// TODO Make it a field presence request for that specific column
@@ -427,7 +428,7 @@ fn search_on_u64_ff(
}
let docset = RangeDocSet::new(value_range, column);
Ok(Box::new(ConstScorer::new(docset, boost)))
Ok(box_scorer(ConstScorer::new(docset, boost)))
}
/// Returns true if the type maps to a u64 fast field

View File

@@ -1,9 +1,11 @@
use std::mem::{transmute_copy, ManuallyDrop};
use std::ops::DerefMut;
use downcast_rs::impl_downcast;
use crate::docset::DocSet;
use crate::Score;
use crate::query::Explanation;
use crate::{DocId, Score, TERMINATED};
/// Scored set of documents matching a query within a specific segment.
///
@@ -13,6 +15,53 @@ pub trait Scorer: downcast_rs::Downcast + DocSet + 'static {
///
/// This method will perform a bit of computation and is not cached.
fn score(&mut self) -> Score;
/// Calls `callback` with all of the `(doc, score)` for which score
/// is exceeding a given threshold.
///
/// This method is useful for the TopDocs collector.
/// For all docsets, the blanket implementation has the benefit
/// of prefiltering (doc, score) pairs, avoiding the
/// virtual dispatch cost.
///
/// More importantly, it makes it possible for scorers to implement
/// important optimization (e.g. BlockWAND for union).
fn for_each_pruning(
&mut self,
threshold: Score,
callback: &mut dyn FnMut(DocId, Score) -> Score,
) {
for_each_pruning_scorer_default_impl(self, threshold, callback);
}
/// Calls `callback` with all of the `(doc, score)` in the scorer.
fn for_each(&mut self, callback: &mut dyn FnMut(DocId, Score)) {
let mut doc = self.doc();
while doc != TERMINATED {
callback(doc, self.score());
doc = self.advance();
}
}
/// Returns an explanation for the score of the current document.
fn explain(&mut self) -> Explanation {
let score = self.score();
let name = std::any::type_name_of_val(self);
Explanation::new(name, score)
}
}
/// Boxes a scorer. Prefer this to Box::new as it avoids double boxing
/// when TScorer is already a Box<dyn Scorer>.
pub fn box_scorer<TScorer: Scorer>(scorer: TScorer) -> Box<dyn Scorer> {
if std::any::TypeId::of::<TScorer>() == std::any::TypeId::of::<Box<dyn Scorer>>() {
unsafe {
let forget_me = ManuallyDrop::new(scorer);
transmute_copy::<TScorer, Box<dyn Scorer>>(&forget_me)
}
} else {
Box::new(scorer)
}
}
impl_downcast!(Scorer);
@@ -22,4 +71,41 @@ impl Scorer for Box<dyn Scorer> {
fn score(&mut self) -> Score {
self.deref_mut().score()
}
fn for_each_pruning(
&mut self,
threshold: Score,
callback: &mut dyn FnMut(DocId, Score) -> Score,
) {
self.deref_mut().for_each_pruning(threshold, callback);
}
fn for_each(&mut self, callback: &mut dyn FnMut(DocId, Score)) {
self.deref_mut().for_each(callback);
}
}
/// Calls `callback` with all of the `(doc, score)` for which score
/// is exceeding a given threshold.
///
/// This method is useful for the [`TopDocs`](crate::collector::TopDocs) collector.
/// For all docsets, the blanket implementation has the benefit
/// of prefiltering (doc, score) pairs, avoiding the
/// virtual dispatch cost.
///
/// More importantly, it makes it possible for scorers to implement
/// important optimization (e.g. BlockWAND for union).
pub(crate) fn for_each_pruning_scorer_default_impl<TScorer: Scorer + ?Sized>(
scorer: &mut TScorer,
mut threshold: Score,
callback: &mut dyn FnMut(DocId, Score) -> Score,
) {
let mut doc = scorer.doc();
while doc != TERMINATED {
let score = scorer.score();
if score > threshold {
threshold = callback(doc, score);
}
doc = scorer.advance();
}
}

View File

@@ -4,9 +4,9 @@ mod term_weight;
pub use self::term_query::TermQuery;
pub use self::term_scorer::TermScorer;
#[cfg(test)]
mod tests {
use crate::collector::TopDocs;
use crate::docset::DocSet;
use crate::postings::compression::COMPRESSION_BLOCK_SIZE;

View File

@@ -1,23 +1,27 @@
use crate::codec::postings::{PostingsCodec, PostingsWithBlockMax};
use crate::codec::{Codec, StandardCodec};
use crate::docset::DocSet;
use crate::fieldnorm::FieldNormReader;
use crate::postings::{FreqReadingOption, Postings, SegmentPostings};
use crate::postings::Postings;
use crate::query::bm25::Bm25Weight;
use crate::query::{Explanation, Scorer};
use crate::{DocId, Score};
#[derive(Clone)]
pub struct TermScorer {
postings: SegmentPostings,
pub struct TermScorer<
TPostings: Postings = <<StandardCodec as Codec>::PostingsCodec as PostingsCodec>::Postings,
> {
postings: TPostings,
fieldnorm_reader: FieldNormReader,
similarity_weight: Bm25Weight,
}
impl TermScorer {
impl<TPostings: Postings> TermScorer<TPostings> {
pub fn new(
postings: SegmentPostings,
postings: TPostings,
fieldnorm_reader: FieldNormReader,
similarity_weight: Bm25Weight,
) -> TermScorer {
) -> TermScorer<TPostings> {
TermScorer {
postings,
fieldnorm_reader,
@@ -25,10 +29,35 @@ impl TermScorer {
}
}
pub(crate) fn seek_block(&mut self, target_doc: DocId) {
self.postings.block_cursor.seek_block(target_doc);
pub fn term_freq(&self) -> u32 {
self.postings.term_freq()
}
pub fn fieldnorm_id(&self) -> u8 {
self.fieldnorm_reader.fieldnorm_id(self.doc())
}
pub fn max_score(&self) -> Score {
self.similarity_weight.max_score()
}
}
impl<TPostingsWithBlockMax: PostingsWithBlockMax> TermScorer<TPostingsWithBlockMax> {
pub(crate) fn last_doc_in_block(&self) -> DocId {
self.postings.last_doc_in_block()
}
/// Advances the term scorer to the block containing target_doc and returns
/// an upperbound for the score all of the documents in the block.
/// (BlockMax). This score is not guaranteed to be the
/// effective maximum score of the block.
pub(crate) fn seek_block_max(&mut self, target_doc: DocId) -> Score {
self.postings
.seek_block_max(target_doc, &self.fieldnorm_reader, &self.similarity_weight)
}
}
impl TermScorer {
#[cfg(test)]
pub fn create_for_test(
doc_and_tfs: &[(DocId, u32)],
@@ -44,60 +73,15 @@ impl TermScorer {
.unwrap_or(0u32)
< fieldnorms.len() as u32
);
let segment_postings =
type SegmentPostings = <<StandardCodec as Codec>::PostingsCodec as PostingsCodec>::Postings;
let segment_postings: SegmentPostings =
SegmentPostings::create_from_docs_and_tfs(doc_and_tfs, Some(fieldnorms));
let fieldnorm_reader = FieldNormReader::for_test(fieldnorms);
TermScorer::new(segment_postings, fieldnorm_reader, similarity_weight)
}
/// See `FreqReadingOption`.
pub(crate) fn freq_reading_option(&self) -> FreqReadingOption {
self.postings.block_cursor.freq_reading_option()
}
/// Returns the maximum score for the current block.
///
/// In some rare case, the result may not be exact. In this case a lower value is returned,
/// (and may lead us to return a lesser document).
///
/// At index time, we store the (fieldnorm_id, term frequency) pair that maximizes the
/// score assuming the average fieldnorm computed on this segment.
///
/// Though extremely rare, it is theoretically possible that the actual average fieldnorm
/// is different enough from the current segment average fieldnorm that the maximum over a
/// specific is achieved on a different document.
///
/// (The result is on the other hand guaranteed to be correct if there is only one segment).
pub fn block_max_score(&mut self) -> Score {
self.postings
.block_cursor
.block_max_score(&self.fieldnorm_reader, &self.similarity_weight)
}
pub fn term_freq(&self) -> u32 {
self.postings.term_freq()
}
pub fn fieldnorm_id(&self) -> u8 {
self.fieldnorm_reader.fieldnorm_id(self.doc())
}
pub fn explain(&self) -> Explanation {
let fieldnorm_id = self.fieldnorm_id();
let term_freq = self.term_freq();
self.similarity_weight.explain(fieldnorm_id, term_freq)
}
pub fn max_score(&self) -> Score {
self.similarity_weight.max_score()
}
pub fn last_doc_in_block(&self) -> DocId {
self.postings.block_cursor.skip_reader().last_doc_in_block()
}
}
impl DocSet for TermScorer {
impl<TPostings: Postings> DocSet for TermScorer<TPostings> {
#[inline]
fn advance(&mut self) -> DocId {
self.postings.advance()
@@ -105,6 +89,7 @@ impl DocSet for TermScorer {
#[inline]
fn seek(&mut self, target: DocId) -> DocId {
debug_assert!(target >= self.doc());
self.postings.seek(target)
}
@@ -118,13 +103,19 @@ impl DocSet for TermScorer {
}
}
impl Scorer for TermScorer {
impl<TPostings: Postings> Scorer for TermScorer<TPostings> {
#[inline]
fn score(&mut self) -> Score {
let fieldnorm_id = self.fieldnorm_id();
let term_freq = self.term_freq();
self.similarity_weight.score(fieldnorm_id, term_freq)
}
fn explain(&mut self) -> Explanation {
let fieldnorm_id = self.fieldnorm_id();
let term_freq = self.term_freq();
self.similarity_weight.explain(fieldnorm_id, term_freq)
}
}
#[cfg(test)]
@@ -133,7 +124,7 @@ mod tests {
use crate::index::SegmentId;
use crate::indexer::index_writer::MEMORY_BUDGET_NUM_BYTES_MIN;
use crate::merge_policy::NoMergePolicy;
use crate::indexer::NoMergePolicy;
use crate::postings::compression::COMPRESSION_BLOCK_SIZE;
use crate::query::term_query::TermScorer;
use crate::query::{Bm25Weight, EnableScoring, Scorer, TermQuery};
@@ -154,7 +145,7 @@ mod tests {
crate::assert_nearly_equals!(max_scorer, 1.3990127);
assert_eq!(term_scorer.doc(), 2);
assert_eq!(term_scorer.term_freq(), 3);
assert_nearly_equals!(term_scorer.block_max_score(), 1.3676447);
assert_nearly_equals!(term_scorer.seek_block_max(2), 1.3676447);
assert_nearly_equals!(term_scorer.score(), 1.0892314);
assert_eq!(term_scorer.advance(), 3);
assert_eq!(term_scorer.doc(), 3);
@@ -169,9 +160,9 @@ mod tests {
}
#[test]
fn test_term_scorer_shallow_advance() -> crate::Result<()> {
fn test_term_scorer_shallow_advance() {
let bm25_weight = Bm25Weight::for_one_term(300, 1024, 10.0);
let mut doc_and_tfs = vec![];
let mut doc_and_tfs = Vec::new();
for i in 0u32..300u32 {
let doc = i * 10;
doc_and_tfs.push((doc, 1u32 + doc % 3u32));
@@ -179,11 +170,10 @@ mod tests {
let fieldnorms: Vec<u32> = std::iter::repeat_n(10u32, 3_000).collect();
let mut term_scorer = TermScorer::create_for_test(&doc_and_tfs, &fieldnorms, bm25_weight);
assert_eq!(term_scorer.doc(), 0u32);
term_scorer.seek_block(1289);
term_scorer.seek_block_max(1289);
assert_eq!(term_scorer.doc(), 0u32);
term_scorer.seek(1289);
assert_eq!(term_scorer.doc(), 1290);
Ok(())
}
proptest! {
@@ -217,7 +207,7 @@ mod tests {
let docs: Vec<DocId> = (0..term_doc_freq).map(|doc| doc as DocId).collect();
for block in docs.chunks(COMPRESSION_BLOCK_SIZE) {
let block_max_score: Score = term_scorer.block_max_score();
let block_max_score: Score = term_scorer.seek_block_max(0);
let mut block_max_score_computed: Score = 0.0;
for &doc in block {
assert_eq!(term_scorer.doc(), doc);
@@ -245,25 +235,24 @@ mod tests {
let fieldnorms: Vec<u32> = std::iter::repeat_n(20u32, 300).collect();
let bm25_weight = Bm25Weight::for_one_term(10, 129, 20.0);
let mut docs = TermScorer::create_for_test(&doc_tfs[..], &fieldnorms[..], bm25_weight);
assert_nearly_equals!(docs.block_max_score(), 2.5161593);
docs.seek_block(135);
assert_nearly_equals!(docs.block_max_score(), 3.4597192);
docs.seek_block(256);
assert_nearly_equals!(docs.seek_block_max(0), 2.5161593);
assert_nearly_equals!(docs.seek_block_max(135), 3.4597192);
// the block is not loaded yet.
assert_nearly_equals!(docs.block_max_score(), 5.2971773);
assert_nearly_equals!(docs.seek_block_max(256), 5.2971773);
assert_eq!(256, docs.seek(256));
assert_nearly_equals!(docs.block_max_score(), 3.9539647);
assert_nearly_equals!(docs.seek_block_max(256), 3.9539647);
}
fn test_block_wand_aux(term_query: &TermQuery, searcher: &Searcher) -> crate::Result<()> {
let term_weight =
term_query.specialized_weight(EnableScoring::enabled_from_searcher(searcher))?;
fn test_block_wand_aux(term_query: &TermQuery, searcher: &Searcher) {
let term_weight = term_query
.specialized_weight(EnableScoring::enabled_from_searcher(searcher))
.unwrap();
for reader in searcher.segment_readers() {
let mut block_max_scores = vec![];
let mut block_max_scores_b = vec![];
let mut docs = vec![];
{
let mut term_scorer = term_weight.term_scorer_for_test(reader, 1.0)?.unwrap();
let mut term_scorer = term_weight.term_scorer_for_test(reader, 1.0).unwrap();
while term_scorer.doc() != TERMINATED {
let mut score = term_scorer.score();
docs.push(term_scorer.doc());
@@ -277,10 +266,10 @@ mod tests {
}
}
{
let mut term_scorer = term_weight.term_scorer_for_test(reader, 1.0)?.unwrap();
let mut term_scorer = term_weight.term_scorer_for_test(reader, 1.0).unwrap();
for d in docs {
term_scorer.seek_block(d);
block_max_scores_b.push(term_scorer.block_max_score());
let block_max_score = term_scorer.seek_block_max(d);
block_max_scores_b.push(block_max_score);
}
}
for (l, r) in block_max_scores
@@ -291,18 +280,18 @@ mod tests {
assert_nearly_equals!(l, r);
}
}
Ok(())
}
#[ignore]
#[test]
fn test_block_wand_long_test() -> crate::Result<()> {
fn test_block_wand_long_test() {
let mut schema_builder = Schema::builder();
let text_field = schema_builder.add_text_field("text", TEXT);
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
let mut writer: IndexWriter =
index.writer_with_num_threads(3, 3 * MEMORY_BUDGET_NUM_BYTES_MIN)?;
let mut writer: IndexWriter = index
.writer_with_num_threads(3, 3 * MEMORY_BUDGET_NUM_BYTES_MIN)
.unwrap();
use rand::Rng;
let mut rng = rand::rng();
writer.set_merge_policy(Box::new(NoMergePolicy));
@@ -310,15 +299,15 @@ mod tests {
let term_freq = rng.random_range(1..10000);
let words: Vec<&str> = std::iter::repeat_n("bbbb", term_freq).collect();
let text = words.join(" ");
writer.add_document(doc!(text_field=>text))?;
writer.add_document(doc!(text_field=>text)).unwrap();
}
writer.commit()?;
writer.commit().unwrap();
let term_query = TermQuery::new(
Term::from_field_text(text_field, "bbbb"),
IndexRecordOption::WithFreqs,
);
let segment_ids: Vec<SegmentId>;
let reader = index.reader()?;
let reader = index.reader().unwrap();
{
let searcher = reader.searcher();
segment_ids = searcher
@@ -326,15 +315,14 @@ mod tests {
.iter()
.map(|segment| segment.segment_id())
.collect();
test_block_wand_aux(&term_query, &searcher)?;
test_block_wand_aux(&term_query, &searcher);
}
writer.merge(&segment_ids[..]).wait().unwrap();
{
reader.reload()?;
reader.reload().unwrap();
let searcher = reader.searcher();
assert_eq!(searcher.segment_readers().len(), 1);
test_block_wand_aux(&term_query, &searcher)?;
test_block_wand_aux(&term_query, &searcher);
}
Ok(())
}
}

View File

@@ -1,12 +1,11 @@
use super::term_scorer::TermScorer;
use crate::docset::{DocSet, COLLECT_BLOCK_BUFFER_LEN};
use crate::fieldnorm::FieldNormReader;
use crate::index::SegmentReader;
use crate::postings::SegmentPostings;
use crate::query::bm25::Bm25Weight;
use crate::query::explanation::does_not_match;
use crate::query::weight::{for_each_docset_buffered, for_each_scorer};
use crate::query::{AllScorer, AllWeight, EmptyScorer, Explanation, Scorer, Weight};
use crate::query::term_query::TermScorer;
use crate::query::weight::for_each_docset_buffered;
use crate::query::{box_scorer, AllScorer, AllWeight, EmptyScorer, Explanation, Scorer, Weight};
use crate::schema::IndexRecordOption;
use crate::{DocId, Score, TantivyError, Term};
@@ -18,7 +17,7 @@ pub struct TermWeight {
}
enum TermOrEmptyOrAllScorer {
TermScorer(Box<TermScorer>),
TermScorer(Box<dyn Scorer>),
Empty,
AllMatch(AllScorer),
}
@@ -27,8 +26,8 @@ impl TermOrEmptyOrAllScorer {
pub fn into_boxed_scorer(self) -> Box<dyn Scorer> {
match self {
TermOrEmptyOrAllScorer::TermScorer(scorer) => scorer,
TermOrEmptyOrAllScorer::Empty => Box::new(EmptyScorer),
TermOrEmptyOrAllScorer::AllMatch(scorer) => Box::new(scorer),
TermOrEmptyOrAllScorer::Empty => box_scorer(EmptyScorer),
TermOrEmptyOrAllScorer::AllMatch(scorer) => box_scorer(scorer),
}
}
}
@@ -44,6 +43,7 @@ impl Weight for TermWeight {
if term_scorer.doc() > doc || term_scorer.seek(doc) != doc {
return Err(does_not_match(doc));
}
let mut term_scorer = term_scorer.downcast::<TermScorer>().ok().unwrap();
let mut explanation = term_scorer.explain();
explanation.add_context(format!("Term={:?}", self.term,));
Ok(explanation)
@@ -73,11 +73,11 @@ impl Weight for TermWeight {
) -> crate::Result<()> {
match self.specialized_scorer(reader, 1.0)? {
TermOrEmptyOrAllScorer::TermScorer(mut term_scorer) => {
for_each_scorer(&mut *term_scorer, callback);
term_scorer.for_each(callback);
}
TermOrEmptyOrAllScorer::Empty => {}
TermOrEmptyOrAllScorer::AllMatch(mut all_scorer) => {
for_each_scorer(&mut all_scorer, callback);
all_scorer.for_each(callback);
}
}
Ok(())
@@ -124,11 +124,9 @@ impl Weight for TermWeight {
let specialized_scorer = self.specialized_scorer(reader, 1.0)?;
match specialized_scorer {
TermOrEmptyOrAllScorer::TermScorer(term_scorer) => {
crate::query::boolean_query::block_wand_single_scorer(
*term_scorer,
threshold,
callback,
);
reader
.codec()
.for_each_pruning(threshold, term_scorer, callback);
}
TermOrEmptyOrAllScorer::Empty => {}
TermOrEmptyOrAllScorer::AllMatch(_) => {
@@ -168,12 +166,15 @@ impl TermWeight {
&self,
reader: &SegmentReader,
boost: Score,
) -> crate::Result<Option<TermScorer>> {
let scorer = self.specialized_scorer(reader, boost)?;
Ok(match scorer {
TermOrEmptyOrAllScorer::TermScorer(scorer) => Some(*scorer),
) -> Option<super::TermScorer> {
let scorer = self.specialized_scorer(reader, boost).unwrap();
match scorer {
TermOrEmptyOrAllScorer::TermScorer(scorer) => {
let term_scorer = scorer.downcast::<super::TermScorer>().ok()?;
Some(*term_scorer)
}
_ => None,
})
}
}
fn specialized_scorer(
@@ -196,14 +197,16 @@ impl TermWeight {
)));
}
let segment_postings: SegmentPostings =
inverted_index.read_postings_from_terminfo(&term_info, self.index_record_option)?;
let fieldnorm_reader = self.fieldnorm_reader(reader)?;
let similarity_weight = self.similarity_weight.boost_by(boost);
Ok(TermOrEmptyOrAllScorer::TermScorer(Box::new(
TermScorer::new(segment_postings, fieldnorm_reader, similarity_weight),
)))
let term_scorer = inverted_index.new_term_scorer(
&term_info,
self.index_record_option,
fieldnorm_reader,
similarity_weight,
)?;
Ok(TermOrEmptyOrAllScorer::TermScorer(term_scorer))
}
fn fieldnorm_reader(&self, segment_reader: &SegmentReader) -> crate::Result<FieldNormReader> {

View File

@@ -1,7 +1,7 @@
use std::cell::RefCell;
use crate::docset::DocSet;
use crate::postings::Postings;
use crate::postings::{DocFreq, Postings};
use crate::query::BitSetDocSet;
use crate::DocId;
@@ -16,6 +16,9 @@ pub struct BitSetPostingUnion<TDocSet> {
docsets: RefCell<Vec<TDocSet>>,
/// The already unionized BitSet of the docsets
bitset: BitSetDocSet,
/// The total number of documents in the union (regardless of the position we are in the
/// bitset).
doc_freq: u32,
}
impl<TDocSet: DocSet> BitSetPostingUnion<TDocSet> {
@@ -23,9 +26,11 @@ impl<TDocSet: DocSet> BitSetPostingUnion<TDocSet> {
docsets: Vec<TDocSet>,
bitset: BitSetDocSet,
) -> BitSetPostingUnion<TDocSet> {
let doc_freq = bitset.doc_freq();
BitSetPostingUnion {
docsets: RefCell::new(docsets),
bitset,
doc_freq,
}
}
}
@@ -46,6 +51,10 @@ impl<TDocSet: Postings> Postings for BitSetPostingUnion<TDocSet> {
term_freq
}
fn has_freq(&self) -> bool {
true
}
fn append_positions_with_offset(&mut self, offset: u32, output: &mut Vec<u32>) {
let curr_doc = self.bitset.doc();
let mut docsets = self.docsets.borrow_mut();
@@ -64,6 +73,10 @@ impl<TDocSet: Postings> Postings for BitSetPostingUnion<TDocSet> {
output.sort_unstable();
output.dedup();
}
fn doc_freq(&self) -> DocFreq {
DocFreq::Exact(self.doc_freq)
}
}
impl<TDocSet: DocSet> DocSet for BitSetPostingUnion<TDocSet> {

View File

@@ -31,7 +31,7 @@ where P: FnMut(&mut T) -> bool {
/// Creates a `DocSet` that iterate through the union of two or more `DocSet`s.
pub struct BufferedUnionScorer<TScorer, TScoreCombiner = DoNothingCombiner> {
/// Active scorers (already filtered of `TERMINATED`).
docsets: Vec<TScorer>,
scorers: Vec<TScorer>,
/// Sliding window presence map for upcoming docs.
///
/// There are `HORIZON_NUM_TINYBITSETS` buckets, each covering
@@ -46,6 +46,8 @@ pub struct BufferedUnionScorer<TScorer, TScoreCombiner = DoNothingCombiner> {
/// hit the same doc within the buffered window.
scores: Box<[TScoreCombiner; HORIZON as usize]>,
/// Start doc ID (inclusive) of the current sliding window.
/// None if the window is not loaded yet. This is true for a freshly created
/// BufferedUnionScorer.
window_start_doc: DocId,
/// Current doc ID of the union.
doc: DocId,
@@ -81,51 +83,81 @@ fn refill<TScorer: Scorer, TScoreCombiner: ScoreCombiner>(
}
impl<TScorer: Scorer, TScoreCombiner: ScoreCombiner> BufferedUnionScorer<TScorer, TScoreCombiner> {
/// Returns the underlying scorers in the union.
pub fn into_scorers(self) -> Vec<TScorer> {
self.scorers
}
/// Accessor for the underlying scorers in the union.
pub fn scorers(&self) -> &[TScorer] {
&self.scorers[..]
}
/// num_docs is the number of documents in the segment.
pub(crate) fn build(
docsets: Vec<TScorer>,
score_combiner_fn: impl FnOnce() -> TScoreCombiner,
num_docs: u32,
) -> BufferedUnionScorer<TScorer, TScoreCombiner> {
let non_empty_docsets: Vec<TScorer> = docsets
let score_combiner = score_combiner_fn();
let mut non_empty_docsets: Vec<TScorer> = docsets
.into_iter()
.filter(|docset| docset.doc() != TERMINATED)
.collect();
let mut union = BufferedUnionScorer {
docsets: non_empty_docsets,
bitsets: Box::new([TinySet::empty(); HORIZON_NUM_TINYBITSETS]),
scores: Box::new([score_combiner_fn(); HORIZON as usize]),
bucket_idx: HORIZON_NUM_TINYBITSETS,
window_start_doc: 0,
doc: 0,
score: 0.0,
num_docs,
};
if union.refill() {
union.advance();
} else {
union.doc = TERMINATED;
let first_doc: DocId = non_empty_docsets
.iter()
.map(|docset| docset.doc())
.min()
.unwrap_or(TERMINATED);
let mut score_combiner_cloned = score_combiner;
let mut i = 0;
while i < non_empty_docsets.len() {
let should_remove_docset: bool = {
let non_empty_docset = &mut non_empty_docsets[i];
if non_empty_docset.doc() != first_doc {
false
} else {
score_combiner_cloned.update(non_empty_docset);
non_empty_docsets[i].advance() == TERMINATED
}
};
if should_remove_docset {
non_empty_docsets.swap_remove(i);
} else {
i += 1;
}
}
let first_score: Score = score_combiner_cloned.score();
BufferedUnionScorer {
scorers: non_empty_docsets,
bitsets: Box::new([TinySet::empty(); HORIZON_NUM_TINYBITSETS]),
scores: Box::new([score_combiner; HORIZON as usize]),
bucket_idx: HORIZON_NUM_TINYBITSETS,
// That way we will be detected as outside the window,
window_start_doc: u32::MAX - HORIZON,
doc: first_doc,
score: first_score,
num_docs,
}
union
}
fn refill(&mut self) -> bool {
if let Some(min_doc) = self.docsets.iter().map(DocSet::doc).min() {
// Reset the sliding window to start at the smallest doc
// across all scorers and prebuffer within the horizon.
self.window_start_doc = min_doc;
self.bucket_idx = 0;
self.doc = min_doc;
refill(
&mut self.docsets,
&mut self.bitsets,
&mut self.scores,
min_doc,
);
true
} else {
false
}
let Some(min_doc) = self.scorers.iter().map(DocSet::doc).min() else {
return false;
};
// Reset the sliding window to start at the smallest doc
// across all scorers and prebuffer within the horizon.
self.window_start_doc = min_doc;
self.bucket_idx = 0;
self.doc = min_doc;
refill(
&mut self.scorers,
&mut self.bitsets,
&mut self.scores,
min_doc,
);
true
}
#[inline]
@@ -147,6 +179,7 @@ impl<TScorer: Scorer, TScoreCombiner: ScoreCombiner> BufferedUnionScorer<TScorer
fn is_in_horizon(&self, target: DocId) -> bool {
// wrapping_sub, because target may be < window_start_doc
// in particular during initialization.
let gap = target.wrapping_sub(self.window_start_doc);
gap < HORIZON
}
@@ -176,11 +209,10 @@ where
if self.doc >= target {
return self.doc;
}
let gap = target - self.window_start_doc;
if gap < HORIZON {
if self.is_in_horizon(target) {
// Our value is within the buffered horizon.
// Skipping to corresponding bucket.
let gap = target.wrapping_sub(self.window_start_doc);
let new_bucket_idx = gap as usize / 64;
for obsolete_tinyset in &mut self.bitsets[self.bucket_idx..new_bucket_idx] {
obsolete_tinyset.clear();
@@ -199,16 +231,14 @@ where
doc
} else {
// clear the buffered info.
for obsolete_tinyset in self.bitsets.iter_mut() {
*obsolete_tinyset = TinySet::empty();
}
self.bitsets.fill(TinySet::empty());
for score_combiner in self.scores.iter_mut() {
score_combiner.clear();
}
// The target is outside of the buffered horizon.
// advance all docsets to a doc >= to the target.
unordered_drain_filter(&mut self.docsets, |docset| {
unordered_drain_filter(&mut self.scorers, |docset| {
if docset.doc() < target {
docset.seek(target);
}
@@ -245,7 +275,7 @@ where
let mut is_hit = false;
let mut min_new_target = TERMINATED;
for docset in self.docsets.iter_mut() {
for docset in self.scorers.iter_mut() {
match docset.seek_danger(target) {
SeekDangerResult::Found => {
is_hit = true;
@@ -275,11 +305,11 @@ where
}
fn size_hint(&self) -> u32 {
estimate_union(self.docsets.iter().map(DocSet::size_hint), self.num_docs)
estimate_union(self.scorers.iter().map(DocSet::size_hint), self.num_docs)
}
fn cost(&self) -> u64 {
self.docsets.iter().map(|docset| docset.cost()).sum()
self.scorers.iter().map(|docset| docset.cost()).sum()
}
// TODO Also implement `count` with deletes efficiently.
@@ -287,21 +317,17 @@ where
if self.doc == TERMINATED {
return 0;
}
let mut count = self.bitsets[self.bucket_idx..HORIZON_NUM_TINYBITSETS]
let mut count = 1 + self.bitsets[self.bucket_idx..HORIZON_NUM_TINYBITSETS]
.iter()
.map(|bitset| bitset.len())
.sum::<u32>()
+ 1;
for bitset in self.bitsets.iter_mut() {
bitset.clear();
}
.copied()
.map(TinySet::len)
.sum::<u32>();
while self.refill() {
count += self.bitsets.iter().map(|bitset| bitset.len()).sum::<u32>();
for bitset in self.bitsets.iter_mut() {
bitset.clear();
}
count += self.bitsets.iter().copied().map(TinySet::len).sum::<u32>();
self.bitsets.fill(TinySet::empty());
}
self.bucket_idx = HORIZON_NUM_TINYBITSETS;
self.doc = TERMINATED;
count
}
}

View File

@@ -1,5 +1,5 @@
use crate::docset::{DocSet, TERMINATED};
use crate::postings::Postings;
use crate::postings::{DocFreq, Postings};
use crate::DocId;
/// A `SimpleUnion` is a `DocSet` that is the union of multiple `DocSet`.
@@ -56,6 +56,22 @@ impl<TDocSet: Postings> Postings for SimpleUnion<TDocSet> {
term_freq
}
fn has_freq(&self) -> bool {
true
}
/// We do not know the actual document frequency, so we return
/// the maximum document frequency of the docsets.
fn doc_freq(&self) -> DocFreq {
let approximate_doc_freq = self
.docsets
.iter()
.map(|docset| u32::from(docset.doc_freq()))
.max()
.unwrap_or(0u32);
DocFreq::Approximate(approximate_doc_freq)
}
fn append_positions_with_offset(&mut self, offset: u32, output: &mut Vec<u32>) {
for docset in &mut self.docsets {
let doc = docset.doc();

View File

@@ -1,21 +1,9 @@
use super::Scorer;
use crate::docset::COLLECT_BLOCK_BUFFER_LEN;
use crate::index::SegmentReader;
use crate::query::explanation::does_not_match;
use crate::query::Explanation;
use crate::{DocId, DocSet, Score, TERMINATED};
/// Iterates through all of the documents and scores matched by the DocSet
/// `DocSet`.
pub(crate) fn for_each_scorer<TScorer: Scorer + ?Sized>(
scorer: &mut TScorer,
callback: &mut dyn FnMut(DocId, Score),
) {
let mut doc = scorer.doc();
while doc != TERMINATED {
callback(doc, scorer.score());
doc = scorer.advance();
}
}
use crate::{DocId, DocSet, Score};
/// Iterates through all of the documents matched by the DocSet
/// `DocSet`.
@@ -34,31 +22,6 @@ pub(crate) fn for_each_docset_buffered<T: DocSet + ?Sized>(
}
}
/// Calls `callback` with all of the `(doc, score)` for which score
/// is exceeding a given threshold.
///
/// This method is useful for the [`TopDocs`](crate::collector::TopDocs) collector.
/// For all docsets, the blanket implementation has the benefit
/// of prefiltering (doc, score) pairs, avoiding the
/// virtual dispatch cost.
///
/// More importantly, it makes it possible for scorers to implement
/// important optimization (e.g. BlockWAND for union).
pub(crate) fn for_each_pruning_scorer<TScorer: Scorer + ?Sized>(
scorer: &mut TScorer,
mut threshold: Score,
callback: &mut dyn FnMut(DocId, Score) -> Score,
) {
let mut doc = scorer.doc();
while doc != TERMINATED {
let score = scorer.score();
if score > threshold {
threshold = callback(doc, score);
}
doc = scorer.advance();
}
}
/// A Weight is the specialization of a `Query`
/// for a given set of segments.
///
@@ -72,7 +35,13 @@ pub trait Weight: Send + Sync + 'static {
fn scorer(&self, reader: &SegmentReader, boost: Score) -> crate::Result<Box<dyn Scorer>>;
/// Returns an [`Explanation`] for the given document.
fn explain(&self, reader: &SegmentReader, doc: DocId) -> crate::Result<Explanation>;
fn explain(&self, reader: &SegmentReader, doc: DocId) -> crate::Result<Explanation> {
let mut scorer = self.scorer(reader, 1.0)?;
if scorer.doc() > doc || scorer.seek(doc) != doc {
return Err(does_not_match(doc));
}
Ok(scorer.explain())
}
/// Returns the number documents within the given [`SegmentReader`].
fn count(&self, reader: &SegmentReader) -> crate::Result<u32> {
@@ -92,7 +61,7 @@ pub trait Weight: Send + Sync + 'static {
callback: &mut dyn FnMut(DocId, Score),
) -> crate::Result<()> {
let mut scorer = self.scorer(reader, 1.0)?;
for_each_scorer(scorer.as_mut(), callback);
scorer.for_each(callback);
Ok(())
}
@@ -127,7 +96,7 @@ pub trait Weight: Send + Sync + 'static {
callback: &mut dyn FnMut(DocId, Score) -> Score,
) -> crate::Result<()> {
let mut scorer = self.scorer(reader, 1.0)?;
for_each_pruning_scorer(scorer.as_mut(), threshold, callback);
scorer.for_each_pruning(threshold, callback);
Ok(())
}
}

View File

@@ -7,6 +7,7 @@ use arc_swap::ArcSwap;
pub use warming::Warmer;
use self::warming::WarmingState;
use crate::codec::Codec;
use crate::core::searcher::{SearcherGeneration, SearcherInner};
use crate::directory::{Directory, WatchCallback, WatchHandle, META_LOCK};
use crate::store::DOCSTORE_CACHE_CAPACITY;
@@ -38,17 +39,17 @@ pub enum ReloadPolicy {
/// - number of warming threads, for parallelizing warming work
/// - The cache size of the underlying doc store readers.
#[derive(Clone)]
pub struct IndexReaderBuilder {
pub struct IndexReaderBuilder<C: Codec = crate::codec::StandardCodec> {
reload_policy: ReloadPolicy,
index: Index,
index: Index<C>,
warmers: Vec<Weak<dyn Warmer>>,
num_warming_threads: usize,
doc_store_cache_num_blocks: usize,
}
impl IndexReaderBuilder {
impl<C: Codec> IndexReaderBuilder<C> {
#[must_use]
pub(crate) fn new(index: Index) -> IndexReaderBuilder {
pub(crate) fn new(index: Index<C>) -> IndexReaderBuilder<C> {
IndexReaderBuilder {
reload_policy: ReloadPolicy::OnCommitWithDelay,
index,
@@ -63,7 +64,7 @@ impl IndexReaderBuilder {
/// Building the reader is a non-trivial operation that requires
/// to open different segment readers. It may take hundreds of milliseconds
/// of time and it may return an error.
pub fn try_into(self) -> crate::Result<IndexReader> {
pub fn try_into(self) -> crate::Result<IndexReader<C>> {
let searcher_generation_inventory = Inventory::default();
let warming_state = WarmingState::new(
self.num_warming_threads,
@@ -106,7 +107,7 @@ impl IndexReaderBuilder {
///
/// See [`ReloadPolicy`] for more details.
#[must_use]
pub fn reload_policy(mut self, reload_policy: ReloadPolicy) -> IndexReaderBuilder {
pub fn reload_policy(mut self, reload_policy: ReloadPolicy) -> IndexReaderBuilder<C> {
self.reload_policy = reload_policy;
self
}
@@ -118,14 +119,14 @@ impl IndexReaderBuilder {
pub fn doc_store_cache_num_blocks(
mut self,
doc_store_cache_num_blocks: usize,
) -> IndexReaderBuilder {
) -> IndexReaderBuilder<C> {
self.doc_store_cache_num_blocks = doc_store_cache_num_blocks;
self
}
/// Set the [`Warmer`]s that are invoked when reloading searchable segments.
#[must_use]
pub fn warmers(mut self, warmers: Vec<Weak<dyn Warmer>>) -> IndexReaderBuilder {
pub fn warmers(mut self, warmers: Vec<Weak<dyn Warmer>>) -> IndexReaderBuilder<C> {
self.warmers = warmers;
self
}
@@ -135,33 +136,33 @@ impl IndexReaderBuilder {
/// This allows parallelizing warming work when there are multiple [`Warmer`] registered with
/// the [`IndexReader`].
#[must_use]
pub fn num_warming_threads(mut self, num_warming_threads: usize) -> IndexReaderBuilder {
pub fn num_warming_threads(mut self, num_warming_threads: usize) -> IndexReaderBuilder<C> {
self.num_warming_threads = num_warming_threads;
self
}
}
impl TryInto<IndexReader> for IndexReaderBuilder {
impl<C: Codec> TryInto<IndexReader<C>> for IndexReaderBuilder<C> {
type Error = crate::TantivyError;
fn try_into(self) -> crate::Result<IndexReader> {
fn try_into(self) -> crate::Result<IndexReader<C>> {
IndexReaderBuilder::try_into(self)
}
}
struct InnerIndexReader {
struct InnerIndexReader<C: Codec> {
doc_store_cache_num_blocks: usize,
index: Index,
index: Index<C>,
warming_state: WarmingState,
searcher: arc_swap::ArcSwap<SearcherInner>,
searcher_generation_counter: Arc<AtomicU64>,
searcher_generation_inventory: Inventory<SearcherGeneration>,
}
impl InnerIndexReader {
impl<C: Codec> InnerIndexReader<C> {
fn new(
doc_store_cache_num_blocks: usize,
index: Index,
index: Index<C>,
warming_state: WarmingState,
// The searcher_generation_inventory is not used as source, but as target to track the
// loaded segments.
@@ -189,7 +190,7 @@ impl InnerIndexReader {
///
/// This function acquires a lock to prevent GC from removing files
/// as we are opening our index.
fn open_segment_readers(index: &Index) -> crate::Result<Vec<SegmentReader>> {
fn open_segment_readers(index: &Index<C>) -> crate::Result<Vec<SegmentReader>> {
// Prevents segment files from getting deleted while we are in the process of opening them
let _meta_lock = index.directory().acquire_lock(&META_LOCK)?;
let searchable_segments = index.searchable_segments()?;
@@ -212,7 +213,7 @@ impl InnerIndexReader {
}
fn create_searcher(
index: &Index,
index: &Index<C>,
doc_store_cache_num_blocks: usize,
warming_state: &WarmingState,
searcher_generation_counter: &Arc<AtomicU64>,
@@ -226,9 +227,10 @@ impl InnerIndexReader {
);
let schema = index.schema();
// SearcherInner uses Index<StandardCodec> since the codec doesn't affect reading
let searcher = Arc::new(SearcherInner::new(
schema,
index.clone(),
index.with_standard_codec(),
segment_readers,
searcher_generation,
doc_store_cache_num_blocks,
@@ -264,14 +266,14 @@ impl InnerIndexReader {
///
/// `IndexReader` just wraps an `Arc`.
#[derive(Clone)]
pub struct IndexReader {
inner: Arc<InnerIndexReader>,
pub struct IndexReader<C: Codec = crate::codec::StandardCodec> {
inner: Arc<InnerIndexReader<C>>,
_watch_handle_opt: Option<WatchHandle>,
}
impl IndexReader {
impl<C: Codec> IndexReader<C> {
#[cfg(test)]
pub(crate) fn index(&self) -> Index {
pub(crate) fn index(&self) -> Index<C> {
self.inner.index.clone()
}

View File

@@ -124,7 +124,6 @@ impl SegmentSpaceUsage {
FieldNorms => PerField(self.fieldnorms().clone()),
Terms => PerField(self.termdict().clone()),
SegmentComponent::Store => ComponentSpaceUsage::Store(self.store().clone()),
SegmentComponent::TempStore => ComponentSpaceUsage::Store(self.store().clone()),
Delete => Basic(self.deletes()),
}
}