From 1fc7afa90a69d47d836ea22bae2da785fb75dad7 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Mon, 5 Feb 2018 09:33:25 +0900 Subject: [PATCH] Issue/range query (#242) BitSet and RangeQuery --- .gitignore | 3 +- CHANGELOG.md | 10 + src/collector/count_collector.rs | 22 +- src/common/bitset.rs | 396 ++++++++++++++++++++++ src/common/mod.rs | 3 + src/lib.rs | 107 ++++++ src/postings/docset.rs | 35 +- src/postings/intersection.rs | 3 +- src/postings/postings.rs | 24 +- src/postings/segment_postings.rs | 20 +- src/postings/vec_postings.rs | 4 +- src/query/all_query.rs | 4 +- src/query/bitset/mod.rs | 268 +++++++++++++++ src/query/boolean_query/boolean_query.rs | 10 +- src/query/boolean_query/boolean_scorer.rs | 2 +- src/query/boolean_query/boolean_weight.rs | 38 ++- src/query/mod.rs | 6 + src/query/phrase_query/phrase_scorer.rs | 4 +- src/query/range_query.rs | 292 ++++++++++++++++ src/query/scorer.rs | 64 +++- src/query/term_query/term_scorer.rs | 2 +- src/termdict/fstdict/termdict.rs | 12 +- src/termdict/mod.rs | 8 +- 23 files changed, 1247 insertions(+), 90 deletions(-) create mode 100644 src/common/bitset.rs create mode 100644 src/query/bitset/mod.rs create mode 100644 src/query/range_query.rs diff --git a/.gitignore b/.gitignore index e2a04b58a..b6f5cc5b8 100644 --- a/.gitignore +++ b/.gitignore @@ -1,3 +1,4 @@ +*.swp target target/debug .vscode @@ -8,4 +9,4 @@ benchmark cpp/simdcomp/bitpackingbenchmark *.bk .idea -trace.dat \ No newline at end of file +trace.dat diff --git a/CHANGELOG.md b/CHANGELOG.md index b5f468f2d..5f425cbc2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,13 @@ +Tantivy 0.5 +========================== +- Faceting +- RangeQuery +- Configurable tokenization pipeline +- Allowing super large indexes + - 64 bits file address + - Smarter encoding of the `TermInfo` objects + + Tantivy 0.4.3 ========================== diff --git a/src/collector/count_collector.rs b/src/collector/count_collector.rs index 6707e687e..15363e33a 100644 --- a/src/collector/count_collector.rs +++ b/src/collector/count_collector.rs @@ -33,18 +33,16 @@ impl Collector for CountCollector { #[cfg(test)] mod tests { - use super::*; - use test::Bencher; - use collector::Collector; + use collector::{Collector, CountCollector}; - #[bench] - fn build_collector(b: &mut Bencher) { - b.iter(|| { - let mut count_collector = CountCollector::default(); - for doc in 0..1_000_000 { - count_collector.collect(doc, 1f32); - } - count_collector.count() - }); + #[test] + fn test_count_collector() { + let mut count_collector = CountCollector::default(); + assert_eq!(count_collector.count(), 0); + count_collector.collect(0u32, 1f32); + assert_eq!(count_collector.count(), 1); + assert_eq!(count_collector.count(), 1); + count_collector.collect(1u32, 1f32); + assert_eq!(count_collector.count(), 2); } } diff --git a/src/common/bitset.rs b/src/common/bitset.rs new file mode 100644 index 000000000..fb01e961e --- /dev/null +++ b/src/common/bitset.rs @@ -0,0 +1,396 @@ +use std::fmt; + +#[derive(Clone, Copy, Eq, PartialEq)] +pub(crate) struct TinySet(u64); + +impl fmt::Debug for TinySet { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + self.into_iter().collect::>().fmt(f) + } +} + +pub struct TinySetIterator(TinySet); +impl Iterator for TinySetIterator { + type Item = u32; + + fn next(&mut self) -> Option { + self.0.pop_lowest() + } +} + +impl IntoIterator for TinySet { + type Item = u32; + type IntoIter = TinySetIterator; + fn into_iter(self) -> Self::IntoIter { + TinySetIterator(self) + } +} + +impl TinySet { + + /// Returns an empty `TinySet`. + pub fn empty() -> TinySet { + TinySet(0u64) + } + + /// Returns the complement of the set in `[0, 64[`. + fn complement(&self) -> TinySet { + TinySet(!self.0) + } + + + /// Returns true iff the `TinySet` contains the element `el`. + pub fn contains(&self, el: u32) -> bool { + !self.intersect(TinySet::singleton(el)).is_empty() + } + + /// Returns the intersection of `self` and `other` + pub fn intersect(&self, other: TinySet) -> TinySet { + TinySet(self.0 & other.0) + } + + /// Creates a new `TinySet` containing only one element + /// within `[0; 64[` + #[inline(always)] + pub fn singleton(el: u32) -> TinySet { + TinySet(1u64 << (el as u64)) + } + + /// Insert a new element within [0..64[ + #[inline(always)] + pub fn insert(self, el: u32) -> TinySet { + self.union(TinySet::singleton(el)) + } + + /// Insert a new element within [0..64[ + #[inline(always)] + pub fn insert_mut(&mut self, el: u32) -> bool { + let old = *self; + *self = old.insert(el); + old != *self + } + + /// Returns the union of two tinysets + #[inline(always)] + pub fn union(self, other: TinySet) -> TinySet { + TinySet(self.0 | other.0) + } + + /// Returns true iff the `TinySet` is empty. + #[inline(always)] + pub fn is_empty(&self) -> bool { + self.0 == 0u64 + } + + /// Returns the lowest element in the `TinySet` + /// and removes it. + #[inline(always)] + pub fn pop_lowest(&mut self) -> Option { + if let Some(lowest) = self.lowest() { + self.0 ^= TinySet::singleton(lowest).0; + Some(lowest) + } else { + None + } + } + + /// Returns the lowest element in the `TinySet` + /// (or None if the set is empty). + #[inline(always)] + pub fn lowest(&mut self) -> Option { + if self.is_empty() { + None + } else { + let least_significant_bit = self.0.trailing_zeros() as u32; + Some(least_significant_bit) + } + } + + /// Returns a `TinySet` than contains all values up + /// to limit excluded. + /// + /// The limit is assumed to be strictly lower than 64. + pub fn range_lower(upper_bound: u32) -> TinySet { + TinySet((1u64 << ((upper_bound % 64u32) as u64)) - 1u64) + } + + /// Returns a `TinySet` that contains all values greater + /// or equal to the given limit, included. (and up to 63) + /// + /// The limit is assumed to be strictly lower than 64. + pub fn range_greater_or_equal(from_included: u32) -> TinySet { + TinySet::range_lower(from_included).complement() + } +} + +#[derive(Clone)] +pub struct BitSet { + tinysets: Box<[TinySet]>, + len: usize, //< Technically it should be u32, but we + // count multiple inserts. + // `usize` guards us from overflow. + max_value: u32, +} + +fn num_buckets(max_val: u32) -> u32 { + (max_val + 63u32) / 64u32 +} + +impl BitSet { + + /// Create a new `BitSet` that may contain elements + /// within `[0, max_val[`. + pub fn with_max_value(max_value: u32) -> BitSet { + let num_buckets = num_buckets(max_value); + let tinybisets = vec![TinySet::empty(); num_buckets as usize].into_boxed_slice(); + BitSet { + tinysets: tinybisets, + len: 0, + max_value + } + } + + /// Removes all elements from the `BitSet`. + pub fn clear(&mut self) { + for tinyset in self.tinysets.iter_mut() { + *tinyset = TinySet::empty(); + } + } + + /// Returns the number of elements in the `BitSet`. + pub fn len(&self) -> usize { + self.len + } + + /// Inserts an element in the `BitSet` + pub fn insert(&mut self, el: u32) { + // we do not check saturated els. + let higher = el / 64u32; + let lower = el % 64u32; + self.len += + if self.tinysets[higher as usize].insert_mut(lower) { + 1 + } else { + 0 + }; + } + + /// Returns true iff the elements is in the `BitSet`. + pub fn contains(&self, el: u32) -> bool { + self.tinyset(el / 64u32) + .contains(el % 64) + } + + /// Returns the first non-empty `TinySet` associated to a bucket lower + /// or greater than bucket. + /// + /// Reminder: the tiny set with the bucket `bucket`, represents the + /// elements from `bucket * 64` to `(bucket+1) * 64`. + pub(crate) fn first_non_empty_bucket(&self, bucket: u32) -> Option { + self.tinysets[bucket as usize..] + .iter() + .cloned() + .position(|tinyset| !tinyset.is_empty()) + .map(|delta_bucket| bucket + delta_bucket as u32) + } + + pub fn max_value(&self) -> u32 { + self.max_value + } + + /// Returns the tiny bitset representing the + /// the set restricted to the number range from + /// `bucket * 64` to `(bucket + 1) * 64`. + pub(crate) fn tinyset(&self, bucket: u32) -> TinySet { + self.tinysets[bucket as usize] + } +} + + +#[cfg(test)] +mod tests { + + extern crate test; + use tests; + use std::collections::HashSet; + use super::BitSet; + use super::TinySet; + use tests::generate_nonunique_unsorted; + use std::collections::BTreeSet; + use query::BitSetDocSet; + use DocSet; + + #[test] + fn test_tiny_set() { + assert!(TinySet::empty().is_empty()); + { + let mut u = TinySet::empty().insert(1u32); + assert_eq!(u.pop_lowest(), Some(1u32)); + assert!(u.pop_lowest().is_none()) + } + { + let mut u = TinySet::empty() + .insert(1u32) + .insert(1u32); + assert_eq!(u.pop_lowest(), Some(1u32)); + assert!(u.pop_lowest().is_none()) + } + { + let mut u = TinySet::empty().insert(2u32); + assert_eq!(u.pop_lowest(), Some(2u32)); + u.insert_mut(1u32); + assert_eq!(u.pop_lowest(), Some(1u32)); + assert!(u.pop_lowest().is_none()); + } + { + let mut u = TinySet::empty().insert(63u32); + assert_eq!(u.pop_lowest(), Some(63u32)); + assert!(u.pop_lowest().is_none()); + } + } + + #[test] + fn test_bitset() { + let test_against_hashset = |els: &[u32], max_value: u32| { + let mut hashset: HashSet = HashSet::new(); + let mut bitset = BitSet::with_max_value(max_value); + for &el in els { + assert!(el < max_value); + hashset.insert(el); + bitset.insert(el); + } + for el in 0..max_value { + assert_eq!(hashset.contains(&el), bitset.contains(el)); + } + assert_eq!(bitset.max_value(), max_value); + }; + + test_against_hashset(&[], 0); + test_against_hashset(&[], 1); + test_against_hashset(&[0u32], 1); + test_against_hashset(&[0u32], 100); + test_against_hashset(&[1u32, 2u32], 4); + test_against_hashset(&[99u32], 100); + test_against_hashset(&[63u32], 64); + test_against_hashset(&[62u32, 63u32], 64); + } + + + #[test] + fn test_bitset_large() { + let arr = generate_nonunique_unsorted(1_000_000, 50_000); + let mut btreeset: BTreeSet = BTreeSet::new(); + let mut bitset = BitSet::with_max_value(1_000_000); + for el in arr { + btreeset.insert(el); + bitset.insert(el); + } + for i in 0..1_000_000 { + assert_eq!(btreeset.contains(&i), bitset.contains(i)); + } + assert_eq!(btreeset.len(), bitset.len()); + let mut bitset_docset = BitSetDocSet::from(bitset); + for el in btreeset.into_iter() { + bitset_docset.advance(); + assert_eq!(bitset_docset.doc(), el); + } + assert!(!bitset_docset.advance()); + } + + #[test] + fn test_bitset_num_buckets() { + use super::num_buckets; + assert_eq!(num_buckets(0u32), 0); + assert_eq!(num_buckets(1u32), 1); + assert_eq!(num_buckets(64u32), 1); + assert_eq!(num_buckets(65u32), 2); + assert_eq!(num_buckets(128u32), 2); + assert_eq!(num_buckets(129u32), 3); + } + + #[test] + fn test_tinyset_range() { + assert_eq!(TinySet::range_lower(3).into_iter().collect::>(), [0, 1, 2]); + assert!(TinySet::range_lower(0).is_empty()); + assert_eq!( + TinySet::range_lower(63).into_iter().collect::>(), + (0u32..63u32).collect::>() + ); + assert_eq!(TinySet::range_lower(1).into_iter().collect::>(), [0]); + assert_eq!(TinySet::range_lower(2).into_iter().collect::>(), [0, 1]); + assert_eq!( + TinySet::range_greater_or_equal(3).into_iter().collect::>(), + (3u32..64u32).collect::>() + ); + } + + #[test] + fn test_bitset_len() { + let mut bitset = BitSet::with_max_value(1_000); + assert_eq!(bitset.len(), 0); + bitset.insert(3u32); + assert_eq!(bitset.len(), 1); + bitset.insert(103u32); + assert_eq!(bitset.len(), 2); + bitset.insert(3u32); + assert_eq!(bitset.len(), 2); + bitset.insert(103u32); + assert_eq!(bitset.len(), 2); + bitset.insert(104u32); + assert_eq!(bitset.len(), 3); + } + + #[test] + fn test_bitset_clear() { + let mut bitset = BitSet::with_max_value(1_000); + let els = tests::sample(1_000, 0.01f32); + for &el in &els { + bitset.insert(el); + } + assert!(els.iter().all(|el| bitset.contains(*el))); + bitset.clear(); + for el in 0u32..1000u32 { + assert!(!bitset.contains(el)); + } + } + + #[bench] + fn bench_tinyset_pop(b: &mut test::Bencher) { + b.iter(|| { + test::black_box(TinySet::singleton(31u32)) + .pop_lowest() + }); + } + + #[bench] + fn bench_tinyset_sum(b: &mut test::Bencher) { + let tiny_set = TinySet::empty() + .insert(10u32) + .insert(14u32) + .insert(21u32); + b.iter(|| { + assert_eq!( + test::black_box(tiny_set).into_iter().sum::(), + 45u32); + }); + } + + #[bench] + fn bench_tinyarr_sum(b: &mut test::Bencher) { + let v = [10u32, 14u32, 21u32] ; + b.iter(|| { + test::black_box(v) + .iter() + .cloned() + .sum::() + }); + } + + #[bench] + fn bench_bitset_initialize(b: &mut test::Bencher) { + b.iter(|| { + BitSet::with_max_value(1_000_000) + }); + } +} + diff --git a/src/common/mod.rs b/src/common/mod.rs index 39c86aa3f..aceea844d 100644 --- a/src/common/mod.rs +++ b/src/common/mod.rs @@ -4,6 +4,7 @@ mod vint; mod counting_writer; mod composite_file; pub mod bitpacker; +mod bitset; pub(crate) use self::composite_file::{CompositeFile, CompositeWrite}; pub use self::serialize::BinarySerializable; @@ -12,6 +13,8 @@ pub use self::timer::TimerTree; pub use self::timer::OpenTimer; pub use self::vint::VInt; pub use self::counting_writer::CountingWriter; +pub use self::bitset::BitSet; +pub(crate) use self::bitset::TinySet; use std::io; diff --git a/src/lib.rs b/src/lib.rs index ca6c0dfc1..ec38e0936 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -4,6 +4,7 @@ #![feature(box_syntax)] #![feature(optin_builtin_traits)] #![feature(conservative_impl_trait)] +#![feature(collections_range)] #![feature(integer_atomics)] #![cfg_attr(test, feature(test))] #![cfg_attr(test, feature(iterator_step_by))] @@ -17,11 +18,106 @@ //! Tantivy is a search engine library. //! Think `Lucene`, but in Rust. //! +//! ```rust + +//! # extern crate tempdir; +//! # +//! #[macro_use] +//! extern crate tantivy; +//! +//! // ... +//! +//! # use std::path::Path; +//! # use tempdir::TempDir; +//! # use tantivy::Index; +//! # use tantivy::schema::*; +//! # use tantivy::collector::TopCollector; +//! # use tantivy::query::QueryParser; +//! # +//! # fn main() { +//! # // Let's create a temporary directory for the +//! # // sake of this example +//! # if let Ok(dir) = TempDir::new("tantivy_example_dir") { +//! # run_example(dir.path()).unwrap(); +//! # dir.close().unwrap(); +//! # } +//! # } +//! # +//! # fn run_example(index_path: &Path) -> tantivy::Result<()> { +//! // First we need to define a schema ... +//! +//! // `TEXT` means the field should be tokenized and indexed, +//! // along with its term frequency and term positions. +//! // +//! // `STORED` means that the field will also be saved +//! // in a compressed, row-oriented key-value store. +//! // This store is useful to reconstruct the +//! // documents that were selected during the search phase. +//! let mut schema_builder = SchemaBuilder::default(); +//! let title = schema_builder.add_text_field("title", TEXT | STORED); +//! let body = schema_builder.add_text_field("body", TEXT); +//! let schema = schema_builder.build(); +//! +//! // Indexing documents +//! +//! let index = Index::create(index_path, schema.clone())?; +//! +//! // Here we use a buffer of 100MB that will be split +//! // between indexing threads. +//! let mut index_writer = index.writer(100_000_000)?; +//! +//! // Let's index one documents! +//! index_writer.add_document(doc!( +//! title => "The Old Man and the Sea", +//! body => "He was an old man who fished alone in a skiff in \ +//! the Gulf Stream and he had gone eighty-four days \ +//! now without taking a fish." +//! )); +//! +//! // We need to call .commit() explicitly to force the +//! // index_writer to finish processing the documents in the queue, +//! // flush the current index to the disk, and advertise +//! // the existence of new documents. +//! index_writer.commit()?; +//! +//! // # Searching +//! +//! index.load_searchers()?; +//! +//! let searcher = index.searcher(); +//! +//! let query_parser = QueryParser::for_index(&index, vec![title, body]); +//! +//! // QueryParser may fail if the query is not in the right +//! // format. For user facing applications, this can be a problem. +//! // A ticket has been opened regarding this problem. +//! let query = query_parser.parse_query("sea whale")?; +//! +//! let mut top_collector = TopCollector::with_limit(10); +//! searcher.search(&*query, &mut top_collector)?; +//! +//! // Our top collector now contains the 10 +//! // most relevant doc ids... +//! let doc_addresses = top_collector.docs(); +//! for doc_address in doc_addresses { +//! let retrieved_doc = searcher.doc(&doc_address)?; +//! println!("{}", schema.to_json(&retrieved_doc)); +//! } +//! +//! # Ok(()) +//! # } +//! ``` +//! +//! +//! //! A good place for you to get started is to check out //! the example code ( //! [literate programming](http://fulmicoton.com/tantivy-examples/simple_search.html) / //! [source code](https://github.com/fulmicoton/tantivy/blob/master/examples/simple_search.rs)) + + + #[macro_use] extern crate lazy_static; @@ -190,6 +286,7 @@ mod tests { use fastfield::{FastFieldReader, I64FastFieldReader, U64FastFieldReader}; use Postings; use rand::{Rng, SeedableRng, XorShiftRng}; + use rand::distributions::{Range, IndependentSample}; fn generate_array_with_seed(n: usize, ratio: f32, seed_val: u32) -> Vec { let seed: &[u32; 4] = &[1, 2, 3, seed_val]; @@ -200,6 +297,16 @@ mod tests { .collect() } + pub fn generate_nonunique_unsorted(max_value: u32, n_elems: usize) -> Vec { + let seed: &[u32; 4] = &[1, 2, 3, 4]; + let mut rng: XorShiftRng = XorShiftRng::from_seed(*seed); + let between = Range::new(0u32, max_value); + (0..n_elems) + .map(|_| between.ind_sample(&mut rng)) + .collect::>() + } + + pub fn generate_array(n: usize, ratio: f32) -> Vec { generate_array_with_seed(n, ratio, 4) } diff --git a/src/postings/docset.rs b/src/postings/docset.rs index 65c41f76b..c030b092b 100644 --- a/src/postings/docset.rs +++ b/src/postings/docset.rs @@ -2,6 +2,7 @@ use DocId; use std::borrow::Borrow; use std::borrow::BorrowMut; use std::cmp::Ordering; +use common::BitSet; /// Expresses the outcome of a call to `DocSet`'s `.skip_next(...)`. #[derive(PartialEq, Eq, Debug)] @@ -92,7 +93,14 @@ pub trait DocSet { /// Returns a best-effort hint of the /// length of the docset. - fn size_hint(&self) -> usize; + fn size_hint(&self) -> u32; + + /// Appends all docs to a `bitset`. + fn append_to_bitset(&mut self, bitset: &mut BitSet) { + while self.advance() { + bitset.insert(self.doc()); + } + } } impl DocSet for Box { @@ -111,30 +119,13 @@ impl DocSet for Box { unboxed.doc() } - fn size_hint(&self) -> usize { + fn size_hint(&self) -> u32 { let unboxed: &TDocSet = self.borrow(); unboxed.size_hint() } -} -impl<'a, TDocSet: DocSet> DocSet for &'a mut TDocSet { - fn advance(&mut self) -> bool { - let unref: &mut TDocSet = *self; - unref.advance() - } - - fn skip_next(&mut self, target: DocId) -> SkipResult { - let unref: &mut TDocSet = *self; - unref.skip_next(target) - } - - fn doc(&self) -> DocId { - let unref: &TDocSet = *self; - unref.doc() - } - - fn size_hint(&self) -> usize { - let unref: &TDocSet = *self; - unref.size_hint() + fn append_to_bitset(&mut self, bitset: &mut BitSet) { + let unboxed: &mut TDocSet = self.borrow_mut(); + unboxed.append_to_bitset(bitset); } } diff --git a/src/postings/intersection.rs b/src/postings/intersection.rs index b105405f6..5234f51c0 100644 --- a/src/postings/intersection.rs +++ b/src/postings/intersection.rs @@ -31,7 +31,8 @@ impl IntersectionDocSet { } impl DocSet for IntersectionDocSet { - fn size_hint(&self) -> usize { + /// Returns the minimum `.size_hint()` of the intersected docsets. + fn size_hint(&self) -> u32 { self.docsets .iter() .map(|docset| docset.size_hint()) diff --git a/src/postings/postings.rs b/src/postings/postings.rs index 52f16198a..ac5516e2e 100644 --- a/src/postings/postings.rs +++ b/src/postings/postings.rs @@ -30,15 +30,15 @@ impl Postings for Box { unboxed.positions() } } - -impl<'a, TPostings: Postings> Postings for &'a mut TPostings { - fn term_freq(&self) -> u32 { - let unref: &TPostings = *self; - unref.term_freq() - } - - fn positions(&self) -> &[u32] { - let unref: &TPostings = *self; - unref.positions() - } -} +// +//impl<'a, TPostings: Postings> Postings for &'a mut TPostings { +// fn term_freq(&self) -> u32 { +// let unref: &TPostings = *self; +// unref.term_freq() +// } +// +// fn positions(&self) -> &[u32] { +// let unref: &TPostings = *self; +// unref.positions() +// } +//} diff --git a/src/postings/segment_postings.rs b/src/postings/segment_postings.rs index 682fc82f6..9fbee7efa 100644 --- a/src/postings/segment_postings.rs +++ b/src/postings/segment_postings.rs @@ -1,5 +1,6 @@ use compression::{BlockDecoder, CompressedIntStream, VIntDecoder, COMPRESSION_BLOCK_SIZE}; use DocId; +use common::BitSet; use postings::{DocSet, HasLen, Postings, SkipResult}; use std::cmp; use fst::Streamer; @@ -235,8 +236,8 @@ impl DocSet for SegmentPostings { } } - fn size_hint(&self) -> usize { - self.len() + fn size_hint(&self) -> u32 { + self.len() as u32 } /// Return the current document's `DocId`. @@ -249,6 +250,21 @@ impl DocSet for SegmentPostings { ); docs[self.cur] } + + fn append_to_bitset(&mut self, bitset: &mut BitSet) { + // finish the current block + if self.advance() { + for &doc in &self.block_cursor.docs()[self.cur..] { + bitset.insert(doc); + } + // ... iterate through the remaining blocks. + while self.block_cursor.advance() { + for &doc in self.block_cursor.docs() { + bitset.insert(doc); + } + } + } + } } impl HasLen for SegmentPostings { diff --git a/src/postings/vec_postings.rs b/src/postings/vec_postings.rs index f6c5ae8d9..51c402cd6 100644 --- a/src/postings/vec_postings.rs +++ b/src/postings/vec_postings.rs @@ -35,8 +35,8 @@ impl DocSet for VecPostings { self.doc_ids[self.cursor.0] } - fn size_hint(&self) -> usize { - self.len() + fn size_hint(&self) -> u32 { + self.len() as u32 } } diff --git a/src/query/all_query.rs b/src/query/all_query.rs index da36bf38b..4d2c3eff2 100644 --- a/src/query/all_query.rs +++ b/src/query/all_query.rs @@ -59,8 +59,8 @@ impl DocSet for AllScorer { self.doc } - fn size_hint(&self) -> usize { - self.max_doc as usize + fn size_hint(&self) -> u32 { + self.max_doc } } diff --git a/src/query/bitset/mod.rs b/src/query/bitset/mod.rs new file mode 100644 index 000000000..cb8ecde7b --- /dev/null +++ b/src/query/bitset/mod.rs @@ -0,0 +1,268 @@ +use common::{BitSet, TinySet}; +use DocId; +use postings::DocSet; +use postings::SkipResult; +use std::cmp::Ordering; + +/// A `BitSetDocSet` makes it possible to iterate through a bitset as if it was a `DocSet`. +/// +/// # Implementation detail +/// +/// Skipping is relatively fast here as we can directly point to the +/// right tiny bitset bucket. +/// +/// TODO: Consider implementing a `BitTreeSet` in order to advance faster +/// when the bitset is sparse +pub struct BitSetDocSet { + docs: BitSet, + cursor_bucket: u32, //< index associated to the current tiny bitset + cursor_tinybitset: TinySet, + doc: u32, +} + +impl BitSetDocSet { + fn go_to_bucket(&mut self, bucket_addr: u32) { + self.cursor_bucket = bucket_addr; + self.cursor_tinybitset = self.docs.tinyset(bucket_addr); + } +} + +impl From for BitSetDocSet { + fn from(docs: BitSet) -> BitSetDocSet { + let first_tiny_bitset = if docs.max_value() == 0 { + TinySet::empty() + } else { + docs.tinyset(0) + }; + BitSetDocSet { + docs, + cursor_bucket: 0, + cursor_tinybitset: first_tiny_bitset, + doc: 0u32, + } + } +} + +impl DocSet for BitSetDocSet { + fn advance(&mut self) -> bool { + if let Some(lower) = self.cursor_tinybitset.pop_lowest() { + self.doc = (self.cursor_bucket as u32 * 64u32) | lower; + return true; + } + if let Some(cursor_bucket) = self.docs.first_non_empty_bucket(self.cursor_bucket + 1) { + self.go_to_bucket(cursor_bucket); + let lower = self.cursor_tinybitset.pop_lowest().unwrap(); + self.doc = (cursor_bucket * 64u32) | lower; + true + } else { + false + } +} + + fn skip_next(&mut self, target: DocId) -> SkipResult { + // skip is required to advance. + if !self.advance() { + return SkipResult::End; + } + let target_bucket = target / 64u32; + + // Mask for all of the bits greater or equal + // to our target document. + match target_bucket.cmp(&self.cursor_bucket) { + Ordering::Greater => { + self.go_to_bucket(target_bucket); + let greater_filter: TinySet = TinySet::range_greater_or_equal(target); + self.cursor_tinybitset = self.cursor_tinybitset.intersect(greater_filter); + if !self.advance() { + SkipResult::End + } else { + if self.doc() == target { + SkipResult::Reached + } else { + debug_assert!(self.doc() > target); + SkipResult::OverStep + } + } + } + Ordering::Equal => loop { + match self.doc().cmp(&target) { + Ordering::Less => { + if !self.advance() { + return SkipResult::End; + } + } + Ordering::Equal => { + return SkipResult::Reached; + } + Ordering::Greater => { + debug_assert!(self.doc() > target); + return SkipResult::OverStep; + } + } + }, + Ordering::Less => { + debug_assert!(self.doc() > target); + SkipResult::OverStep + } + } + } + + /// Returns the current document + fn doc(&self) -> DocId { + self.doc + } + + /// Advances the cursor to the next document + /// None is returned if the iterator has `DocSet` + /// has already been entirely consumed. + fn next(&mut self) -> Option { + if self.advance() { + Some(self.doc()) + } else { + None + } + } + + /// Returns half of the `max_doc` + /// This is quite a terrible heuristic, + /// but we don't have access to any better + /// value. + fn size_hint(&self) -> u32 { + self.docs.len() as u32 + } +} + +#[cfg(test)] +mod tests { + use DocId; + use common::BitSet; + use postings::{DocSet, SkipResult}; + use super::BitSetDocSet; + extern crate test; + + fn create_docbitset(docs: &[DocId], max_doc: DocId) -> BitSetDocSet { + let mut docset = BitSet::with_max_value(max_doc); + for &doc in docs { + docset.insert(doc); + } + BitSetDocSet::from(docset) + } + + fn test_go_through_sequential(docs: &[DocId]) { + let mut docset = create_docbitset(docs, 1_000u32); + for &doc in docs { + assert!(docset.advance()); + assert_eq!(doc, docset.doc()); + } + assert!(!docset.advance()); + assert!(!docset.advance()); + } + + #[test] + fn test_docbitset_sequential() { + test_go_through_sequential(&[]); + test_go_through_sequential(&[1, 2, 3]); + test_go_through_sequential(&[1, 2, 3, 4, 5, 63, 64, 65]); + test_go_through_sequential(&[63, 64, 65]); + test_go_through_sequential(&[1, 2, 3, 4, 95, 96, 97, 98, 99]); + } + + #[test] + fn test_docbitset_skip() { + { + let mut docset = create_docbitset(&[1, 5, 6, 7, 5112], 10_000); + assert_eq!(docset.skip_next(7), SkipResult::Reached); + assert_eq!(docset.doc(), 7); + assert!(docset.advance(), 7); + assert_eq!(docset.doc(), 5112); + assert!(!docset.advance()); + } + { + let mut docset = create_docbitset(&[1, 5, 6, 7, 5112], 10_000); + assert_eq!(docset.skip_next(3), SkipResult::OverStep); + assert_eq!(docset.doc(), 5); + assert!(docset.advance()); + } + { + let mut docset = create_docbitset(&[5112], 10_000); + assert_eq!(docset.skip_next(5112), SkipResult::Reached); + assert_eq!(docset.doc(), 5112); + assert!(!docset.advance()); + } + { + let mut docset = create_docbitset(&[5112], 10_000); + assert_eq!(docset.skip_next(5113), SkipResult::End); + assert!(!docset.advance()); + } + { + let mut docset = create_docbitset(&[5112], 10_000); + assert_eq!(docset.skip_next(5111), SkipResult::OverStep); + assert_eq!(docset.doc(), 5112); + assert!(!docset.advance()); + } + { + let mut docset = create_docbitset(&[1, 5, 6, 7, 5112, 5500, 6666], 10_000); + assert_eq!(docset.skip_next(5112), SkipResult::Reached); + assert_eq!(docset.doc(), 5112); + assert!(docset.advance()); + assert_eq!(docset.doc(), 5500); + assert!(docset.advance()); + assert_eq!(docset.doc(), 6666); + assert!(!docset.advance()); + } + { + let mut docset = create_docbitset(&[1, 5, 6, 7, 5112, 5500, 6666], 10_000); + assert_eq!(docset.skip_next(5111), SkipResult::OverStep); + assert_eq!(docset.doc(), 5112); + assert!(docset.advance()); + assert_eq!(docset.doc(), 5500); + assert!(docset.advance()); + assert_eq!(docset.doc(), 6666); + assert!(!docset.advance()); + } + { + let mut docset = create_docbitset(&[1, 5, 6, 7, 5112, 5513, 6666], 10_000); + assert_eq!(docset.skip_next(5111), SkipResult::OverStep); + assert_eq!(docset.doc(), 5112); + assert!(docset.advance()); + assert_eq!(docset.doc(), 5513); + assert!(docset.advance()); + assert_eq!(docset.doc(), 6666); + assert!(!docset.advance()); + } + } + + + #[bench] + fn bench_bitset_1pct_insert(b: &mut test::Bencher) { + use tests; + let els = tests::generate_nonunique_unsorted(1_000_000u32, 10_000); + b.iter(|| { + let mut bitset = BitSet::with_max_value(1_000_000); + for el in els.iter().cloned() { bitset.insert(el); } + }); + } + + #[bench] + fn bench_bitset_1pct_clone(b: &mut test::Bencher) { + use tests; + let els = tests::generate_nonunique_unsorted(1_000_000u32, 10_000); + let mut bitset = BitSet::with_max_value(1_000_000); + for el in els { bitset.insert(el); } + b.iter(|| { bitset.clone() }); + } + + #[bench] + fn bench_bitset_1pct_clone_iterate(b: &mut test::Bencher) { + use tests; + use DocSet; + let els = tests::generate_nonunique_unsorted(1_000_000u32, 10_000); + let mut bitset = BitSet::with_max_value(1_000_000); + for el in els { bitset.insert(el); } + b.iter(|| { + let mut docset = BitSetDocSet::from(bitset.clone()); + while docset.advance() {} + }); + } +} + diff --git a/src/query/boolean_query/boolean_query.rs b/src/query/boolean_query/boolean_query.rs index af4418d4e..112c2f519 100644 --- a/src/query/boolean_query/boolean_query.rs +++ b/src/query/boolean_query/boolean_query.rs @@ -8,7 +8,6 @@ use schema::Term; use query::TermQuery; use schema::IndexRecordOption; use query::Occur; -use query::OccurFilter; /// The boolean query combines a set of queries /// @@ -39,14 +38,9 @@ impl Query for BooleanQuery { fn weight(&self, searcher: &Searcher) -> Result> { let sub_weights = self.subqueries .iter() - .map(|&(ref _occur, ref subquery)| subquery.weight(searcher)) + .map(|&(ref occur, ref subquery)| Ok((*occur, subquery.weight(searcher)?))) .collect::>()?; - let occurs: Vec = self.subqueries - .iter() - .map(|&(ref occur, ref _subquery)| *occur) - .collect(); - let filter = OccurFilter::new(&occurs); - Ok(box BooleanWeight::new(sub_weights, filter)) + Ok(box BooleanWeight::new(sub_weights)) } } diff --git a/src/query/boolean_query/boolean_scorer.rs b/src/query/boolean_query/boolean_scorer.rs index 5bc574c68..12228850b 100644 --- a/src/query/boolean_query/boolean_scorer.rs +++ b/src/query/boolean_query/boolean_scorer.rs @@ -90,7 +90,7 @@ impl BooleanScorer { } impl DocSet for BooleanScorer { - fn size_hint(&self) -> usize { + fn size_hint(&self) -> u32 { // TODO fix this. it should be the min // of the MUST scorer // and the max of the SHOULD scorers. diff --git a/src/query/boolean_query/boolean_weight.rs b/src/query/boolean_query/boolean_weight.rs index ef5d36374..76f9e8a2e 100644 --- a/src/query/boolean_query/boolean_weight.rs +++ b/src/query/boolean_query/boolean_weight.rs @@ -1,31 +1,43 @@ use query::Weight; use core::SegmentReader; +use query::EmptyScorer; use query::Scorer; use super::BooleanScorer; use query::OccurFilter; +use query::Occur; use Result; pub struct BooleanWeight { - weights: Vec>, - occur_filter: OccurFilter, + weights: Vec<(Occur, Box)>, } impl BooleanWeight { - pub fn new(weights: Vec>, occur_filter: OccurFilter) -> BooleanWeight { - BooleanWeight { - weights, - occur_filter, - } + pub fn new(weights: Vec<(Occur, Box)>) -> BooleanWeight { + BooleanWeight { weights } } } impl Weight for BooleanWeight { fn scorer<'a>(&'a self, reader: &'a SegmentReader) -> Result> { - let sub_scorers: Vec> = self.weights - .iter() - .map(|weight| weight.scorer(reader)) - .collect::>()?; - let boolean_scorer = BooleanScorer::new(sub_scorers, self.occur_filter); - Ok(box boolean_scorer) + if self.weights.is_empty() { + Ok(box EmptyScorer) + } else if self.weights.len() == 1 { + let &(occur, ref weight) = &self.weights[0]; + if occur == Occur::MustNot { + Ok(box EmptyScorer) + } else { + weight.scorer(reader) + } + } else { + let sub_scorers: Vec> = self.weights + .iter() + .map(|&(_, ref weight)| weight) + .map(|weight| weight.scorer(reader)) + .collect::>()?; + let occurs: Vec = self.weights.iter().map(|&(ref occur, _)| *occur).collect(); + let occur_filter = OccurFilter::new(&occurs); + let boolean_scorer = BooleanScorer::new(sub_scorers, occur_filter); + Ok(box boolean_scorer) + } } } diff --git a/src/query/mod.rs b/src/query/mod.rs index 7303baebb..7541daf4c 100644 --- a/src/query/mod.rs +++ b/src/query/mod.rs @@ -12,7 +12,10 @@ mod term_query; mod query_parser; mod phrase_query; mod all_query; +mod bitset; +mod range_query; +pub use self::bitset::BitSetDocSet; pub use self::boolean_query::BooleanQuery; pub use self::occur_filter::OccurFilter; pub use self::occur::Occur; @@ -24,4 +27,7 @@ pub use self::scorer::EmptyScorer; pub use self::scorer::Scorer; pub use self::term_query::TermQuery; pub use self::weight::Weight; + pub use self::all_query::{AllQuery, AllScorer, AllWeight}; +pub use self::range_query::RangeQuery; +pub use self::scorer::ConstScorer; diff --git a/src/query/phrase_query/phrase_scorer.rs b/src/query/phrase_query/phrase_scorer.rs index f6775cb60..1c576cac0 100644 --- a/src/query/phrase_query/phrase_scorer.rs +++ b/src/query/phrase_query/phrase_scorer.rs @@ -35,7 +35,7 @@ impl DocSet for PostingsWithOffset { self.segment_postings.doc() } - fn size_hint(&self) -> usize { + fn size_hint(&self) -> u32 { self.segment_postings.size_hint() } @@ -122,7 +122,7 @@ impl DocSet for PhraseScorer { self.intersection_docset.doc() } - fn size_hint(&self) -> usize { + fn size_hint(&self) -> u32 { self.intersection_docset.size_hint() } } diff --git a/src/query/range_query.rs b/src/query/range_query.rs new file mode 100644 index 000000000..3b9e65994 --- /dev/null +++ b/src/query/range_query.rs @@ -0,0 +1,292 @@ +use schema::{Field, IndexRecordOption, Term}; +use query::{Query, Scorer, Weight}; +use termdict::{TermDictionary, TermStreamer, TermStreamerBuilder}; +use core::SegmentReader; +use common::BitSet; +use Result; +use std::any::Any; +use core::Searcher; +use query::BitSetDocSet; +use query::ConstScorer; +use std::collections::Bound; +use std::collections::range::RangeArgument; + + +fn map_boundVec >(bound: Bound, transform: &Transform) -> Bound> { + use self::Bound::*; + match bound { + Excluded(from_val) => Excluded(transform(from_val)), + Included(from_val) => Included(transform(from_val)), + Unbounded => Unbounded + } +} + + + +/// `RangeQuery` match all documents that have at least one term within a defined range. +/// +/// Matched document will all get a constant `Score` of one. +/// +/// # Implementation +/// +/// The current implement will iterate over the terms within the range +/// and append all of the document cross into a `BitSet`. +/// +/// # Example +/// +/// ```rust +/// +/// # #[macro_use] +/// # extern crate tantivy; +/// # use tantivy::Index; +/// # use tantivy::schema::{SchemaBuilder, INT_INDEXED}; +/// # use tantivy::collector::CountCollector; +/// # use tantivy::query::Query; +/// # use tantivy::Result; +/// # use tantivy::query::RangeQuery; +/// # +/// # fn run() -> Result<()> { +/// # let mut schema_builder = SchemaBuilder::new(); +/// # let year_field = schema_builder.add_u64_field("year", INT_INDEXED); +/// # let schema = schema_builder.build(); +/// # +/// # let index = Index::create_in_ram(schema); +/// # { +/// # let mut index_writer = index.writer_with_num_threads(1, 6_000_000).unwrap(); +/// # for year in 1950u64..2017u64 { +/// # let num_docs_within_year = 10 + (year - 1950) * (year - 1950); +/// # for _ in 0..num_docs_within_year { +/// # index_writer.add_document(doc!(year_field => year)); +/// # } +/// # } +/// # index_writer.commit().unwrap(); +/// # } +/// # index.load_searchers()?; +/// let searcher = index.searcher(); +/// +/// let docs_in_the_sixties = RangeQuery::new_u64(year_field, 1960..1970); +/// +/// // ... or `1960..=1969` if inclusive range is enabled. +/// let mut count_collector = CountCollector::default(); +/// docs_in_the_sixties.search(&*searcher, &mut count_collector)?; +/// +/// let num_60s_books = count_collector.count(); +/// +/// # assert_eq!(num_60s_books, 2285); +/// # Ok(()) +/// # } +/// # +/// # fn main() { +/// # run().unwrap() +/// # } +/// ``` +#[derive(Debug)] +pub struct RangeQuery { + field: Field, + left_bound: Bound>, + right_bound: Bound>, +} + +impl RangeQuery { + + /// Create a new `RangeQuery` over a `i64` field. + pub fn new_i64>(field: Field, range: TRangeArgument) -> RangeQuery { + let make_term_val = |val: &i64| { + Term::from_field_i64(field, *val).value_bytes().to_owned() + }; + RangeQuery { + field, + left_bound: map_bound(range.start(), &make_term_val), + right_bound: map_bound(range.end(), &make_term_val) + } + } + + /// Create a new `RangeQuery` over a `u64` field. + pub fn new_u64>(field: Field, range: TRangeArgument) -> RangeQuery { + let make_term_val = |val: &u64| { + Term::from_field_u64(field, *val).value_bytes().to_owned() + }; + RangeQuery { + field, + left_bound: map_bound(range.start(), &make_term_val), + right_bound: map_bound(range.end(), &make_term_val) + } + } + + /// Create a new `RangeQuery` over a `Str` field. + pub fn new_str<'b, TRangeArgument: RangeArgument<&'b str>>(field: Field, range: TRangeArgument) -> RangeQuery { + let make_term_val = |val: &&str| { + val.as_bytes().to_vec() + }; + RangeQuery { + field, + left_bound: map_bound(range.start(), &make_term_val), + right_bound: map_bound(range.end(), &make_term_val) + } + } +} + +impl Query for RangeQuery { + fn as_any(&self) -> &Any { + self + } + + fn weight(&self, _searcher: &Searcher) -> Result> { + Ok(box RangeWeight { + field: self.field, + left_bound: self.left_bound.clone(), + right_bound: self.right_bound.clone() + }) + } +} + +pub struct RangeWeight { + field: Field, + left_bound: Bound>, + right_bound: Bound>, +} + +impl RangeWeight { + fn term_range<'a, T>(&self, term_dict: &'a T) -> T::Streamer + where + T: TermDictionary<'a> + 'a, + { + use std::collections::Bound::*; + let mut term_stream_builder = term_dict.range(); + term_stream_builder = match &self.left_bound { + &Included(ref term_val) => term_stream_builder.ge(term_val), + &Excluded(ref term_val) => term_stream_builder.gt(term_val), + &Unbounded => term_stream_builder, + }; + term_stream_builder = match &self.right_bound { + &Included(ref term_val) => term_stream_builder.le(term_val), + &Excluded(ref term_val) => term_stream_builder.lt(term_val), + &Unbounded => term_stream_builder, + }; + term_stream_builder.into_stream() + } +} + +impl Weight for RangeWeight { + fn scorer<'a>(&'a self, reader: &'a SegmentReader) -> Result> { + let max_doc = reader.max_doc(); + let mut doc_bitset = BitSet::with_max_value(max_doc); + + let inverted_index = reader.inverted_index(self.field); + let term_dict = inverted_index.terms(); + let mut term_range = self.term_range(term_dict); + while term_range.advance() { + let term_info = term_range.value(); + let mut block_segment_postings = inverted_index + .read_block_postings_from_terminfo(term_info, IndexRecordOption::Basic); + while block_segment_postings.advance() { + for &doc in block_segment_postings.docs() { + doc_bitset.insert(doc); + } + } + } + let doc_bitset = BitSetDocSet::from(doc_bitset); + Ok(box ConstScorer::new(doc_bitset)) + } +} + +#[cfg(test)] +mod tests { + + use Index; + use schema::{Document, Field, SchemaBuilder, INT_INDEXED}; + use collector::CountCollector; + use std::collections::Bound; + use query::Query; + use Result; + use super::RangeQuery; + + #[test] + fn test_range_query_simple() { + + fn run() -> Result<()> { + let mut schema_builder = SchemaBuilder::new(); + let year_field= schema_builder.add_u64_field("year", INT_INDEXED); + let schema = schema_builder.build(); + + let index = Index::create_in_ram(schema); + { + let mut index_writer = index.writer_with_num_threads(1, 6_000_000).unwrap(); + for year in 1950u64..2017u64 { + let num_docs_within_year = 10 + (year - 1950) * (year - 1950); + for _ in 0..num_docs_within_year { + index_writer.add_document(doc!(year_field => year)); + } + } + index_writer.commit().unwrap(); + } + index.load_searchers().unwrap(); + let searcher = index.searcher(); + + let docs_in_the_sixties = RangeQuery::new_u64(year_field, 1960u64..1970u64); + + // ... or `1960..=1969` if inclusive range is enabled. + let mut count_collector = CountCollector::default(); + docs_in_the_sixties.search(&*searcher, &mut count_collector)?; + assert_eq!(count_collector.count(), 2285); + Ok(()) + } + + run().unwrap(); + + } + + #[test] + fn test_range_query() { + let int_field: Field; + let schema = { + let mut schema_builder = SchemaBuilder::new(); + int_field = schema_builder.add_i64_field("intfield", INT_INDEXED); + schema_builder.build() + }; + + let index = Index::create_in_ram(schema); + { + let mut index_writer = index.writer_with_num_threads(2, 6_000_000).unwrap(); + + for i in 1..100 { + let mut doc = Document::new(); + for j in 1..100 { + if i % j == 0 { + doc.add_i64(int_field, j as i64); + } + } + index_writer.add_document(doc); + } + + index_writer.commit().unwrap(); + } + index.load_searchers().unwrap(); + let searcher = index.searcher(); + let count_multiples = |range_query: RangeQuery| { + let mut count_collector = CountCollector::default(); + range_query + .search(&*searcher, &mut count_collector) + .unwrap(); + count_collector.count() + }; + + assert_eq!( + count_multiples(RangeQuery::new_i64(int_field, 10..11)), + 9 + ); + assert_eq!( + count_multiples(RangeQuery::new_i64(int_field, (Bound::Included(10), Bound::Included(11)) )), + 18 + ); + assert_eq!( + count_multiples(RangeQuery::new_i64(int_field, (Bound::Excluded(9), Bound::Included(10)))), + 9 + ); + assert_eq!( + count_multiples(RangeQuery::new_i64(int_field, 9..)), + 91 + ); + } + +} diff --git a/src/query/scorer.rs b/src/query/scorer.rs index 170e6aa56..2cbeb001d 100644 --- a/src/query/scorer.rs +++ b/src/query/scorer.rs @@ -2,6 +2,8 @@ use DocSet; use DocId; use Score; use collector::Collector; +use postings::SkipResult; +use common::BitSet; use std::ops::{Deref, DerefMut}; /// Scored set of documents matching a query within a specific segment. @@ -49,7 +51,7 @@ impl DocSet for EmptyScorer { DocId::max_value() } - fn size_hint(&self) -> usize { + fn size_hint(&self) -> u32 { 0 } } @@ -59,3 +61,63 @@ impl Scorer for EmptyScorer { 0f32 } } + + +/// Wraps a `DocSet` and simply returns a constant `Scorer`. +/// The `ConstScorer` is useful if you have a `DocSet` where +/// you needed a scorer. +/// +/// The `ConstScorer`'s constant score can be set +/// by calling `.set_score(...)`. +pub struct ConstScorer { + docset: TDocSet, + score: Score, +} + +impl ConstScorer { + + /// Creates a new `ConstScorer`. + pub fn new(docset: TDocSet) -> ConstScorer { + ConstScorer { + docset, + score: 1f32, + } + } + + /// Sets the constant score to a different value. + pub fn set_score(&mut self, score: Score) { + self.score = score; + } +} + +impl DocSet for ConstScorer { + fn advance(&mut self) -> bool { + self.docset.advance() + } + + fn skip_next(&mut self, target: DocId) -> SkipResult { + self.docset.skip_next(target) + } + + fn fill_buffer(&mut self, buffer: &mut [DocId]) -> usize { + self.docset.fill_buffer(buffer) + } + + fn doc(&self) -> DocId { + self.docset.doc() + } + + fn size_hint(&self) -> u32 { + self.docset.size_hint() + } + + fn append_to_bitset(&mut self, bitset: &mut BitSet) { + self.docset.append_to_bitset(bitset); + } +} + +impl Scorer for ConstScorer { + fn score(&self) -> Score { + 1f32 + } +} diff --git a/src/query/term_query/term_scorer.rs b/src/query/term_query/term_scorer.rs index 43b969140..626cbe029 100644 --- a/src/query/term_query/term_scorer.rs +++ b/src/query/term_query/term_scorer.rs @@ -37,7 +37,7 @@ where self.postings.doc() } - fn size_hint(&self) -> usize { + fn size_hint(&self) -> u32 { self.postings.size_hint() } diff --git a/src/termdict/fstdict/termdict.rs b/src/termdict/fstdict/termdict.rs index 5b56e64df..b36be64ae 100644 --- a/src/termdict/fstdict/termdict.rs +++ b/src/termdict/fstdict/termdict.rs @@ -54,7 +54,7 @@ where fn new(w: W, _field_type: FieldType) -> io::Result { let fst_builder = fst::MapBuilder::new(w).map_err(convert_fst_error)?; Ok(TermDictionaryBuilderImpl { - fst_builder: fst_builder, + fst_builder, data: Vec::new(), term_ord: 0, }) @@ -111,7 +111,7 @@ impl<'a> TermDictionary<'a> for TermDictionaryImpl { let values_source = source.slice(split_len, length_offset); let fst_index = open_fst_index(fst_source); TermDictionaryImpl { - fst_index: fst_index, + fst_index, values_mmap: values_source, } } @@ -120,6 +120,10 @@ impl<'a> TermDictionary<'a> for TermDictionaryImpl { self.values_mmap.len() / TermInfo::SIZE_IN_BYTES } + fn term_ord>(&self, key: K) -> Option { + self.fst_index.get(key) + } + fn ord_to_term(&self, mut ord: TermOrdinal, bytes: &mut Vec) -> bool { bytes.clear(); let fst = self.fst_index.as_fst(); @@ -140,10 +144,6 @@ impl<'a> TermDictionary<'a> for TermDictionaryImpl { true } - fn term_ord>(&self, key: K) -> Option { - self.fst_index.get(key) - } - fn term_info_from_ord(&self, term_ord: TermOrdinal) -> TermInfo { let buffer = self.values_mmap.as_slice(); let offset = term_ord as usize * TermInfo::SIZE_IN_BYTES; diff --git a/src/termdict/mod.rs b/src/termdict/mod.rs index 5f84d933a..0dbc6667d 100644 --- a/src/termdict/mod.rs +++ b/src/termdict/mod.rs @@ -86,6 +86,10 @@ where /// Opens a `TermDictionary` given a data source. fn from_source(source: ReadOnlySource) -> Self; + /// Returns the number of terms in the dictionary. + /// Term ordinals range from 0 to `num_terms() - 1`. + fn num_terms(&self) -> usize; + /// Returns the ordinal associated to a given term. fn term_ord>(&self, term: K) -> Option; @@ -103,10 +107,6 @@ where /// Returns the number of terms in the dictionary. fn term_info_from_ord(&self, term_ord: TermOrdinal) -> TermInfo; - /// Returns the number of terms in the dictionary. - /// Term ordinals range from 0 to `num_terms() - 1`. - fn num_terms(&self) -> usize; - /// Lookups the value corresponding to the key. fn get>(&self, target_key: K) -> Option;