summaryrefslogtreecommitdiffstats
path: root/src
diff options
context:
space:
mode:
authorPaul Masurel <paul.masurel@gmail.com>2018-02-05 09:33:25 +0900
committerGitHub <noreply@github.com>2018-02-05 09:33:25 +0900
commit1fc7afa90a69d47d836ea22bae2da785fb75dad7 (patch)
tree0d2a874107cab7684a71038a9d90810096c28dde /src
parent6a104e4f696d47a9aac7639fad5f2654970dd0e9 (diff)
Issue/range query (#242)
BitSet and RangeQuery
Diffstat (limited to 'src')
-rw-r--r--src/collector/count_collector.rs22
-rw-r--r--src/common/bitset.rs396
-rw-r--r--src/common/mod.rs3
-rw-r--r--src/lib.rs107
-rw-r--r--src/postings/docset.rs35
-rw-r--r--src/postings/intersection.rs3
-rw-r--r--src/postings/postings.rs24
-rw-r--r--src/postings/segment_postings.rs20
-rw-r--r--src/postings/vec_postings.rs4
-rw-r--r--src/query/all_query.rs4
-rw-r--r--src/query/bitset/mod.rs268
-rw-r--r--src/query/boolean_query/boolean_query.rs10
-rw-r--r--src/query/boolean_query/boolean_scorer.rs2
-rw-r--r--src/query/boolean_query/boolean_weight.rs38
-rw-r--r--src/query/mod.rs6
-rw-r--r--src/query/phrase_query/phrase_scorer.rs4
-rw-r--r--src/query/range_query.rs292
-rw-r--r--src/query/scorer.rs64
-rw-r--r--src/query/term_query/term_scorer.rs2
-rw-r--r--src/termdict/fstdict/termdict.rs12
-rw-r--r--src/termdict/mod.rs8
21 files changed, 1235 insertions, 89 deletions
diff --git a/src/collector/count_collector.rs b/src/collector/count_collector.rs
index 6707e68..15363e3 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 0000000..fb01e96
--- /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::<Vec<u32>>().fmt(f)
+ }
+}
+
+pub struct TinySetIterator(TinySet);
+impl Iterator for TinySetIterator {
+ type Item = u32;
+
+ fn next(&mut self) -> Option<Self::Item> {
+ 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<u32> {
+ 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<u32> {
+ 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<u32> {
+ 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<u32> = 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<u32> = 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::<Vec<u32>>(), [0, 1, 2]);
+ assert!(TinySet::range_lower(0).is_empty());
+ assert_eq!(
+ TinySet::range_lower(63).into_iter().collect::<Vec<u32>>(),
+ (0u32..63u32).collect::<Vec<_>>()
+ );
+ assert_eq!(TinySet::range_lower(1).into_iter().collect::<Vec<u32>>(), [0]);
+ assert_eq!(TinySet::range_lower(2).into_iter().collect::<Vec<u32>>(), [0, 1]);
+ assert_eq!(
+ TinySet::range_greater_or_equal(3).into_iter().collect::<Vec<u32>>(),
+ (3u32..64u32).collect::<Vec<_>>()
+ );
+ }
+
+ #[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::<u32>(),
+ 45u32);
+ });
+ }
+
+ #[bench]
+ fn bench_tinyarr_sum(b: &mut test::Bencher) {
+ let v = [10u32, 14u32, 21u32] ;
+ b.iter(|| {
+ test::black_box(v)
+ .iter()
+ .cloned()
+ .sum::<u32>()
+ });
+ }
+
+ #[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 39c86aa..aceea84 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 ca6c0df..ec38e09 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<u32> {
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<u32> {
+ 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::<Vec<u32>>()
+ }
+
+
pub fn generate_array(n: usize, ratio: f32) -> Vec<u32> {
generate_array_with_seed(n, ratio, 4)
}
diff --git a/src/postings/docset.rs b/src/postings/docset.rs
index 65c41f7..c030b09 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<TDocSet: DocSet + ?Sized> DocSet for Box<TDocSet> {
@@ -111,30 +119,13 @@ impl<TDocSet: DocSet + ?Sized> DocSet for Box<TDocSet> {
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 b105405..5234f51 100644
--- a/src/postings/intersection.rs
+++ b/src/postings/intersection.rs
@@ -31,7 +31,8 @@ impl<TDocSet: DocSet> IntersectionDocSet<TDocSet> {
}
impl<TDocSet: DocSet> DocSet for IntersectionDocSet<TDocSet> {
- 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 52f1619..ac5516e 100644
--- a/src/postings/postings.rs
+++ b/src/postings/postings.rs
@@ -30,15 +30,15 @@ impl<TPostings: Postings> Postings for Box<TPostings> {
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 682fc82..9fbee7e 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 f6c5ae8..51c402c 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 da36bf3..4d2c3ef 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 0000000..cb8ecde
--- /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<BitSet> 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<DocId> {
+ 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);
<