From e14701e9cd3f605f800efd1fccdf9f4558034201 Mon Sep 17 00:00:00 2001 From: Jason Goldberger Date: Wed, 13 Feb 2019 16:56:01 -0700 Subject: [PATCH 01/26] Add grouped operations (#493) * [WIP] added UserOperation enum, added IndexWriter.run, and added MultiStamp * removed MultiStamp in favor of std::ops::Range * changed IndexWriter::run to return u64, Stamper::stamps to return a Range, added tests, and added docs * changed delete_cursor skipping to use first operation's opstamp vice last. change index_writer test to use 1 thread * added test for order batch of operations * added a test comment --- src/indexer/index_writer.rs | 172 +++++++++++++++++++++++++++++++++--- src/indexer/operation.rs | 7 ++ src/indexer/stamper.rs | 13 +++ 3 files changed, 180 insertions(+), 12 deletions(-) diff --git a/src/indexer/index_writer.rs b/src/indexer/index_writer.rs index 2d74b46fe..8a43b0c3a 100644 --- a/src/indexer/index_writer.rs +++ b/src/indexer/index_writer.rs @@ -1,4 +1,4 @@ -use super::operation::AddOperation; +use super::operation::{AddOperation, UserOperation}; use super::segment_updater::SegmentUpdater; use super::PreparedCommit; use bit_set::BitSet; @@ -26,6 +26,7 @@ use schema::Document; use schema::IndexRecordOption; use schema::Term; use std::mem; +use std::ops::Range; use std::sync::Arc; use std::thread; use std::thread::JoinHandle; @@ -43,8 +44,8 @@ pub const HEAP_SIZE_MAX: usize = u32::max_value() as usize - MARGIN_IN_BYTES; // reaches `PIPELINE_MAX_SIZE_IN_DOCS` const PIPELINE_MAX_SIZE_IN_DOCS: usize = 10_000; -type DocumentSender = channel::Sender; -type DocumentReceiver = channel::Receiver; +type DocumentSender = channel::Sender>; +type DocumentReceiver = channel::Receiver>; /// Split the thread memory budget into /// - the heap size @@ -266,7 +267,7 @@ fn index_documents( memory_budget: usize, segment: &Segment, generation: usize, - document_iterator: &mut impl Iterator, + document_iterator: &mut Iterator>, segment_updater: &mut SegmentUpdater, mut delete_cursor: DeleteCursor, ) -> Result { @@ -274,11 +275,11 @@ fn index_documents( let segment_id = segment.id(); let table_size = initial_table_size(memory_budget); let mut segment_writer = SegmentWriter::for_segment(table_size, segment.clone(), &schema)?; - for doc in document_iterator { - segment_writer.add_document(doc, &schema)?; - + for documents in document_iterator { + for doc in documents { + segment_writer.add_document(doc, &schema)?; + } let mem_usage = segment_writer.mem_usage(); - if mem_usage >= memory_budget - MARGIN_IN_BYTES { info!( "Buffer limit reached, flushing segment with maxdoc={}.", @@ -409,8 +410,12 @@ impl IndexWriter { // this is a valid guarantee as the // peeked document now belongs to // our local iterator. - if let Some(operation) = document_iterator.peek() { - delete_cursor.skip_to(operation.opstamp); + if let Some(operations) = document_iterator.peek() { + if let Some(first) = operations.first() { + delete_cursor.skip_to(first.opstamp); + } else { + return Ok(()); + } } else { // No more documents. // Happens when there is a commit, or if the `IndexWriter` @@ -643,25 +648,168 @@ impl IndexWriter { pub fn add_document(&mut self, document: Document) -> u64 { let opstamp = self.stamper.stamp(); let add_operation = AddOperation { opstamp, document }; - let send_result = self.document_sender.send(add_operation); + let send_result = self.document_sender.send(vec![add_operation]); if let Err(e) = send_result { panic!("Failed to index document. Sending to indexing channel failed. This probably means all of the indexing threads have panicked. {:?}", e); } opstamp } + + /// Gets a range of stamps from the stamper and "pops" the last stamp + /// from the range returning a tuple of the last optstamp and the popped + /// range. + /// + /// The total number of stamps generated by this method is `count + 1`; + /// each operation gets a stamp from the `stamps` iterator and `last_opstamp` + /// is for the batch itself. + fn get_batch_opstamps(&mut self, count: u64) -> (u64, Range) { + let Range { start, end } = self.stamper.stamps(count + 1u64); + let last_opstamp = end - 1; + let stamps = Range { + start: start, + end: last_opstamp, + }; + (last_opstamp, stamps) + } + + /// Runs a group of document operations ensuring that the operations are + /// assigned contigous u64 opstamps and that add operations of the same + /// group are flushed into the same segment. + /// + /// If the indexing pipeline is full, this call may block. + /// + /// Each operation of the given `user_operations` will receive an in-order, + /// contiguous u64 opstamp. The entire batch itself is also given an + /// opstamp that is 1 greater than the last given operation. This + /// `batch_opstamp` is the return value of `run`. An empty group of + /// `user_operations`, an empty `Vec`, still receives + /// a valid opstamp even though no changes were _actually_ made to the index. + /// + /// Like adds and deletes (see `IndexWriter.add_document` and + /// `IndexWriter.delete_term`), the changes made by calling `run` will be + /// visible to readers only after calling `commit()`. + pub fn run(&mut self, user_operations: Vec) -> u64 { + let count = user_operations.len() as u64; + if count == 0 { + return self.stamper.stamp(); + } + let (batch_opstamp, stamps) = self.get_batch_opstamps(count); + + let mut adds: Vec = Vec::new(); + + for (user_op, opstamp) in user_operations.into_iter().zip(stamps) { + match user_op { + UserOperation::Delete(term) => { + let delete_operation = DeleteOperation { + opstamp: opstamp, + term: term, + }; + self.delete_queue.push(delete_operation); + } + UserOperation::Add(doc) => { + let add_operation = AddOperation { + opstamp: opstamp, + document: doc, + }; + adds.push(add_operation); + } + } + } + let send_result = self.document_sender.send(adds); + if let Err(e) = send_result { + panic!("Failed to index document. Sending to indexing channel failed. This probably means all of the indexing threads have panicked. {:?}", e); + }; + + batch_opstamp + } } #[cfg(test)] mod tests { + use super::super::operation::UserOperation; use super::initial_table_size; use directory::error::LockError; use error::*; use indexer::NoMergePolicy; - use schema::{self, Document}; + use schema::{self, Document, IndexRecordOption}; + use query::{TermQuery}; + use collector::TopDocs; use Index; use Term; + #[test] + fn test_operations_group() { + // an operations group with 2 items should cause 3 opstamps 0, 1, and 2. + let mut schema_builder = schema::Schema::builder(); + let text_field = schema_builder.add_text_field("text", schema::TEXT); + let index = Index::create_in_ram(schema_builder.build()); + let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); + let operations = vec![ + UserOperation::Add(doc!(text_field=>"a")), + UserOperation::Add(doc!(text_field=>"b")), + ]; + let batch_opstamp1 = index_writer.run(operations); + assert_eq!(batch_opstamp1, 2u64); + } + + #[test] + fn test_ordered_batched_operations() { + // * one delete for `doc!(field=>"a")` + // * one add for `doc!(field=>"a")` + // * one add for `doc!(field=>"b")` + // * one delete for `doc!(field=>"b")` + // after commit there is one doc with "a" and 0 doc with "b" + let mut schema_builder = schema::Schema::builder(); + let text_field = schema_builder.add_text_field("text", schema::TEXT); + let index = Index::create_in_ram(schema_builder.build()); + let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); + let a_term = Term::from_field_text(text_field, "a"); + let b_term = Term::from_field_text(text_field, "b"); + let operations = vec![ + UserOperation::Delete(a_term), + UserOperation::Add(doc!(text_field=>"a")), + UserOperation::Add(doc!(text_field=>"b")), + UserOperation::Delete(b_term), + ]; + + index_writer.run(operations); + index_writer.commit().expect("failed to commit"); + index.load_searchers().expect("failed to load searchers"); + + let a_term = Term::from_field_text(text_field, "a"); + let b_term = Term::from_field_text(text_field, "b"); + + let a_query = TermQuery::new(a_term, IndexRecordOption::Basic); + let b_query = TermQuery::new(b_term, IndexRecordOption::Basic); + + let searcher = index.searcher(); + + let a_docs = searcher + .search(&a_query, &TopDocs::with_limit(1)) + .expect("search for a failed"); + + let b_docs = searcher + .search(&b_query, &TopDocs::with_limit(1)) + .expect("search for b failed"); + + assert_eq!(a_docs.len(), 1); + assert_eq!(b_docs.len(), 0); + } + + #[test] + fn test_empty_operations_group() { + let schema_builder = schema::Schema::builder(); + let index = Index::create_in_ram(schema_builder.build()); + let mut index_writer = index.writer(3_000_000).unwrap(); + let operations1 = vec![]; + let batch_opstamp1 = index_writer.run(operations1); + assert_eq!(batch_opstamp1, 0u64); + let operations2 = vec![]; + let batch_opstamp2 = index_writer.run(operations2); + assert_eq!(batch_opstamp2, 1u64); + } + #[test] fn test_lockfile_stops_duplicates() { let schema_builder = schema::Schema::builder(); diff --git a/src/indexer/operation.rs b/src/indexer/operation.rs index 260d510e6..fe57a4a3a 100644 --- a/src/indexer/operation.rs +++ b/src/indexer/operation.rs @@ -14,3 +14,10 @@ pub struct AddOperation { pub opstamp: u64, pub document: Document, } + +/// UserOperation is an enum type that encapsulates other operation types. +#[derive(Eq, PartialEq, Debug)] +pub enum UserOperation { + Add(Document), + Delete(Term), +} diff --git a/src/indexer/stamper.rs b/src/indexer/stamper.rs index 38933e727..69b4c51ef 100644 --- a/src/indexer/stamper.rs +++ b/src/indexer/stamper.rs @@ -1,3 +1,4 @@ +use std::ops::Range; use std::sync::atomic::Ordering; use std::sync::Arc; @@ -60,6 +61,16 @@ impl Stamper { pub fn stamp(&self) -> u64 { self.0.fetch_add(1u64, Ordering::SeqCst) as u64 } + + /// Given a desired count `n`, `stamps` returns an iterator that + /// will supply `n` number of u64 stamps. + pub fn stamps(&self, n: u64) -> Range { + let start = self.0.fetch_add(n, Ordering::SeqCst); + Range { + start: start, + end: start + n, + } + } } #[cfg(test)] @@ -78,5 +89,7 @@ mod test { assert_eq!(stamper.stamp(), 10u64); assert_eq!(stamper_clone.stamp(), 11u64); + assert_eq!(stamper.stamps(3u64), (12..15)); + assert_eq!(stamper.stamp(), 15u64); } } From e70a45426aed79eb2bf999e4a984195ce0456fa8 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Wed, 23 Jan 2019 13:21:31 +0900 Subject: [PATCH 02/26] 0.8.2 release Backporting a fix for non x86_64 platforms --- CHANGELOG.md | 15 +++++++++++++++ Cargo.toml | 2 +- src/indexer/stamper.rs | 14 +++++++------- 3 files changed, 23 insertions(+), 8 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 488d256a2..6750f5b83 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,18 @@ +Tantivy 0.8.2 +===================== +Fixing build for x86_64 platforms. (#496) +No need to update from 0.8.1 if tantivy +is building on your platform. + + +Tantivy 0.8.1 +===================== +Hotfix of #476. + +Merge was reflecting deletes before commit was passed. +Thanks @barrotsteindev for reporting the bug. + + Tantivy 0.8.0 ===================== *No change in the index format* diff --git a/Cargo.toml b/Cargo.toml index fc7a1ba0f..6cf5e4ba2 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "tantivy" -version = "0.8.0" +version = "0.8.2" authors = ["Paul Masurel "] license = "MIT" categories = ["database-implementations", "data-structures"] diff --git a/src/indexer/stamper.rs b/src/indexer/stamper.rs index f3fd1f943..036ef964f 100644 --- a/src/indexer/stamper.rs +++ b/src/indexer/stamper.rs @@ -27,22 +27,22 @@ mod archicture_impl { #[cfg(not(target_arch = "x86_64"))] mod archicture_impl { - /// Under other architecture, we rely on a mutex. - use std::sync::Mutex; use std::sync::atomic::Ordering; + /// Under other architecture, we rely on a mutex. + use std::sync::RwLock; #[derive(Default)] - pub struct AtomicU64Ersatz(Mutex); + pub struct AtomicU64Ersatz(RwLock); impl AtomicU64Ersatz { pub fn new(first_opstamp: u64) -> AtomicU64Ersatz { - AtomicU64Ersatz(AtomicUsize::new(first_opstamp)) + AtomicU64Ersatz(RwLock::new(first_opstamp)) } - pub fn fetch_add(&self, val: u64, _order: Ordering) -> u64 { - let lock = self.0.lock().unwrap(); + pub fn fetch_add(&self, incr: u64, _order: Ordering) -> u64 { + let mut lock = self.0.write().unwrap(); let previous_val = *lock; - *lock = previous_val + 1; + *lock = previous_val + incr; previous_val } } From 788b3803d98f8595b3167cab14b3fc1c6c4b77ea Mon Sep 17 00:00:00 2001 From: Jason Goldberger Date: Mon, 18 Feb 2019 08:25:18 -0700 Subject: [PATCH 03/26] updated changelog (#501) * updated changelog * Update CHANGELOG.md * Update CHANGELOG.md --- CHANGELOG.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 1ac9a92b2..44032fcb9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,6 +6,9 @@ previous index format.* - Indexer memory footprint improved. (VInt comp, inlining the first block. (@fulmicoton) - Stemming in other language possible (@pentlander) - Segments with no docs are deleted earlier (@barrotsteindev) +- Added grouped add and delete operations. + They are guaranteed to happen together (i.e. they cannot be split by a commit). + In addition, adds are guaranteed to happen on the same segment. (@elbow-jason) Tantivy 0.8.2 ===================== From 9451fd5b09181f010c15c67b94a3b33096062e23 Mon Sep 17 00:00:00 2001 From: petr-tik Date: Sat, 23 Feb 2019 00:06:50 +0000 Subject: [PATCH 04/26] MsQueue to channel (#495) * Format Made the docstring consistent remove empty line * Move matches to dev deps * Replace MsQueue with an unbounded crossbeam-channel Questions: queue.push ignores Result return How to test pop() calls, if they block * Format Made the docstring consistent remove empty line * Unwrap the Result of queue.pop * Addressed Paul's review wrap the Result-returning send call with expect() implemented the test not to fail after popping from empty queue removed references to the Michael-Scott Queue formatted --- Cargo.toml | 2 +- src/core/executor.rs | 19 ++++---- src/core/pool.rs | 86 ++++++++++++++++++++++++++++++++-- src/indexer/segment_updater.rs | 2 +- 4 files changed, 92 insertions(+), 17 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index b7c7468b5..9b03bacde 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -41,7 +41,6 @@ owning_ref = "0.4" stable_deref_trait = "1.0.0" rust-stemmers = "1.1" downcast-rs = { version="1.0" } -matches = "0.1" bitpacking = "0.6" census = "0.2" fnv = "1.0.6" @@ -58,6 +57,7 @@ winapi = "0.2" [dev-dependencies] rand = "0.6" maplit = "1" +matches = "0.1.8" [profile.release] opt-level = 3 diff --git a/src/core/executor.rs b/src/core/executor.rs index ce5fd18b7..281e96b91 100644 --- a/src/core/executor.rs +++ b/src/core/executor.rs @@ -123,15 +123,14 @@ mod tests { } } -} - -#[test] -fn test_map_multithread() { - let result: Vec = Executor::multi_thread(3, "search-test") - .map(|i| Ok(i * 2), 0..10) - .unwrap(); - assert_eq!(result.len(), 10); - for i in 0..10 { - assert_eq!(result[i], i * 2); + #[test] + fn test_map_multithread() { + let result: Vec = Executor::multi_thread(3, "search-test") + .map(|i| Ok(i * 2), 0..10) + .unwrap(); + assert_eq!(result.len(), 10); + for i in 0..10 { + assert_eq!(result[i], i * 2); + } } } diff --git a/src/core/pool.rs b/src/core/pool.rs index ccac81321..b24d0b3ed 100644 --- a/src/core/pool.rs +++ b/src/core/pool.rs @@ -1,4 +1,5 @@ -use crossbeam::queue::MsQueue; +use crossbeam::crossbeam_channel::unbounded; +use crossbeam::{Receiver, RecvError, Sender}; use std::ops::{Deref, DerefMut}; use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering; @@ -9,6 +10,37 @@ pub struct GenerationItem { item: T, } +/// Queue implementation for the Object Pool below +/// Uses the unbounded Linked-List type queue from crossbeam-channel +/// Splits the Queue into sender and receiver +struct Queue { + sender: Sender, + receiver: Receiver, +} + +impl Queue { + fn new() -> Self { + let (s, r) = unbounded(); + Queue { + sender: s, + receiver: r, + } + } + + /// Sender trait returns a Result type, which is ignored. + /// The Result is not handled at the moment + fn push(&self, elem: T) { + self.sender + .send(elem) + .expect("Sending an item to crossbeam-queue shouldn't fail"); + } + + /// Relies on the underlying crossbeam-channel Receiver + /// to block on empty queue + fn pop(&self) -> Result { + self.receiver.recv() + } +} /// An object pool /// @@ -16,14 +48,14 @@ pub struct GenerationItem { /// Object are wrapped in a `LeasedItem` wrapper and are /// released automatically back into the pool on `Drop`. pub struct Pool { - queue: Arc>>, + queue: Arc>>, freshest_generation: AtomicUsize, next_generation: AtomicUsize, } impl Pool { pub fn new() -> Pool { - let queue = Arc::new(MsQueue::new()); + let queue = Arc::new(Queue::new()); Pool { queue, freshest_generation: AtomicUsize::default(), @@ -77,7 +109,7 @@ impl Pool { pub fn acquire(&self) -> LeasedItem { let generation = self.generation(); loop { - let gen_item = self.queue.pop(); + let gen_item = self.queue.pop().unwrap(); if gen_item.generation >= generation { return LeasedItem { gen_item: Some(gen_item), @@ -93,7 +125,7 @@ impl Pool { pub struct LeasedItem { gen_item: Option>, - recycle_queue: Arc>>, + recycle_queue: Arc>>, } impl Deref for LeasedItem { @@ -130,6 +162,7 @@ impl Drop for LeasedItem { mod tests { use super::Pool; + use super::Queue; use std::iter; #[test] @@ -146,4 +179,47 @@ mod tests { assert_eq!(*pool.acquire(), 11); } } + + #[test] + fn test_queue() { + let q = Queue::new(); + let elem = 5; + q.push(elem); + let res = q.pop(); + assert_eq!(res.unwrap(), elem); + } + + #[test] + fn test_pool_dont_panic_on_empty_pop() { + // When the object pool is exhausted, it shouldn't panic on pop() + use std::sync::Arc; + use std::{thread, time}; + + // Wrap the pool in an Arc, same way as its used in `core/index.rs` + let pool = Arc::new(Pool::new()); + // clone pools outside the move scope of each new thread + let pool1 = Arc::clone(&pool); + let pool2 = Arc::clone(&pool); + let elements_for_pool = vec![1, 2]; + pool.publish_new_generation(elements_for_pool); + + let mut threads = vec![]; + let sleep_dur = time::Duration::from_millis(10); + // spawn one more thread than there are elements in the pool + threads.push(thread::spawn(move || { + // leasing to make sure it's not dropped before sleep is called + let _leased_searcher = &pool.acquire(); + thread::sleep(sleep_dur); + })); + threads.push(thread::spawn(move || { + // leasing to make sure it's not dropped before sleep is called + let _leased_searcher = &pool1.acquire(); + thread::sleep(sleep_dur); + })); + threads.push(thread::spawn(move || { + // leasing to make sure it's not dropped before sleep is called + let _leased_searcher = &pool2.acquire(); + thread::sleep(sleep_dur); + })); + } } diff --git a/src/indexer/segment_updater.rs b/src/indexer/segment_updater.rs index 9f8e9577f..2580bedda 100644 --- a/src/indexer/segment_updater.rs +++ b/src/indexer/segment_updater.rs @@ -62,7 +62,7 @@ pub fn save_new_metas(schema: Schema, directory: &mut Directory) -> Result<()> { /// Save the index meta file. /// This operation is atomic: /// Either -// - it fails, in which case an error is returned, +/// - it fails, in which case an error is returned, /// and the `meta.json` remains untouched, /// - it success, and `meta.json` is written /// and flushed. From b422f9c3892c4941734c282c8dad10b6384b7850 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Sat, 23 Feb 2019 10:33:59 +0900 Subject: [PATCH 05/26] Partially addresses #500 (#502) Using `tantivy_fst`. Storing `Weak` in the Mmap cache. --- Cargo.toml | 5 +- src/core/index.rs | 14 +--- src/core/segment_id.rs | 4 +- src/directory/mmap_directory.rs | 126 ++++++++++++++++-------------- src/directory/mod.rs | 1 - src/directory/ram_directory.rs | 8 +- src/directory/read_only_source.rs | 81 ++++++++++--------- src/directory/shared_vec_slice.rs | 41 ---------- src/lib.rs | 5 +- src/postings/recorder.rs | 5 +- src/postings/segment_postings.rs | 10 +-- src/query/automaton_weight.rs | 2 +- src/query/regex_query.rs | 2 +- src/query/vec_docset.rs | 2 - src/termdict/streamer.rs | 8 +- src/termdict/termdict.rs | 28 +++---- 16 files changed, 148 insertions(+), 194 deletions(-) delete mode 100644 src/directory/shared_vec_slice.rs diff --git a/Cargo.toml b/Cargo.toml index 9b03bacde..da43f7ee3 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -16,7 +16,8 @@ base64 = "0.10.0" byteorder = "1.0" lazy_static = "1" regex = "1.0" -fst = {version="0.3", default-features=false} +tantivy-fst = {path="../tantivy-search/fst", version="0.1"} +memmap = "0.7" fst-regex = { version="0.2" } lz4 = {version="1.20", optional=true} snap = {version="0.2"} @@ -71,7 +72,7 @@ overflow-checks = true [features] # by default no-fail is disabled. We manually enable it when running test. default = ["mmap", "no_fail"] -mmap = ["fst/mmap", "atomicwrites", "fs2"] +mmap = ["atomicwrites", "fs2"] lz4-compression = ["lz4"] no_fail = ["fail/no_fail"] unstable = [] # useful for benches. diff --git a/src/core/index.rs b/src/core/index.rs index 338aebbb2..33165fc33 100644 --- a/src/core/index.rs +++ b/src/core/index.rs @@ -49,6 +49,7 @@ fn load_metas(directory: &Directory) -> Result { } /// Search Index +#[derive(Clone)] pub struct Index { directory: ManagedDirectory, schema: Schema, @@ -390,19 +391,6 @@ impl fmt::Debug for Index { } } -impl Clone for Index { - fn clone(&self) -> Index { - Index { - directory: self.directory.clone(), - schema: self.schema.clone(), - num_searchers: Arc::clone(&self.num_searchers), - searcher_pool: Arc::clone(&self.searcher_pool), - tokenizers: self.tokenizers.clone(), - executor: self.executor.clone(), - } - } -} - #[cfg(test)] mod tests { use directory::RAMDirectory; diff --git a/src/core/segment_id.rs b/src/core/segment_id.rs index 64c5fee38..5727dbe10 100644 --- a/src/core/segment_id.rs +++ b/src/core/segment_id.rs @@ -19,7 +19,7 @@ pub struct SegmentId(Uuid); #[cfg(test)] lazy_static! { static ref AUTO_INC_COUNTER: atomic::AtomicUsize = atomic::AtomicUsize::default(); - static ref EMPTY_ARR: [u8; 8] = [0u8; 8]; + static ref ZERO_ARRAY: [u8; 8] = [0u8; 8]; } // During tests, we generate the segment id in a autoincrement manner @@ -30,7 +30,7 @@ lazy_static! { #[cfg(test)] fn create_uuid() -> Uuid { let new_auto_inc_id = (*AUTO_INC_COUNTER).fetch_add(1, atomic::Ordering::SeqCst); - Uuid::from_fields(new_auto_inc_id as u32, 0, 0, &*EMPTY_ARR).unwrap() + Uuid::from_fields(new_auto_inc_id as u32, 0, 0, &*ZERO_ARRAY).unwrap() } #[cfg(not(test))] diff --git a/src/directory/mmap_directory.rs b/src/directory/mmap_directory.rs index ed4a3563a..fb4fa785c 100644 --- a/src/directory/mmap_directory.rs +++ b/src/directory/mmap_directory.rs @@ -5,14 +5,11 @@ use atomicwrites; use common::make_io_err; use directory::error::LockError; use directory::error::{DeleteError, IOError, OpenDirectoryError, OpenReadError, OpenWriteError}; -use directory::shared_vec_slice::SharedVecSlice; use directory::Directory; use directory::DirectoryLock; use directory::Lock; use directory::ReadOnlySource; use directory::WritePtr; -use fst::raw::MmapReadOnly; -use std::collections::hash_map::Entry as HashMapEntry; use std::collections::HashMap; use std::convert::From; use std::fmt; @@ -25,11 +22,14 @@ use std::result; use std::sync::Arc; use std::sync::RwLock; use tempdir::TempDir; +use memmap::Mmap; +use std::sync::Weak; +use directory::read_only_source::BoxedData; /// Returns None iff the file exists, can be read, but is empty (and hence /// cannot be mmapped). /// -fn open_mmap(full_path: &Path) -> result::Result, OpenReadError> { +fn open_mmap(full_path: &Path) -> result::Result, OpenReadError> { let file = File::open(full_path).map_err(|e| { if e.kind() == io::ErrorKind::NotFound { OpenReadError::FileDoesNotExist(full_path.to_owned()) @@ -48,7 +48,7 @@ fn open_mmap(full_path: &Path) -> result::Result, OpenReadE return Ok(None); } unsafe { - MmapReadOnly::open(&file) + memmap::Mmap::map(&file) .map(Some) .map_err(|e| From::from(IOError::with_path(full_path.to_owned(), e))) } @@ -71,7 +71,7 @@ pub struct CacheInfo { struct MmapCache { counters: CacheCounters, - cache: HashMap, + cache: HashMap>, } impl Default for MmapCache { @@ -84,10 +84,6 @@ impl Default for MmapCache { } impl MmapCache { - /// Removes a `MmapReadOnly` entry from the mmap cache. - fn discard_from_cache(&mut self, full_path: &Path) -> bool { - self.cache.remove(full_path).is_some() - } fn get_info(&mut self) -> CacheInfo { let paths: Vec = self.cache.keys().cloned().collect(); @@ -97,23 +93,27 @@ impl MmapCache { } } - fn get_mmap(&mut self, full_path: &Path) -> Result, OpenReadError> { - Ok(match self.cache.entry(full_path.to_owned()) { - HashMapEntry::Occupied(occupied_entry) => { - let mmap = occupied_entry.get(); - self.counters.hit += 1; - Some(mmap.clone()) - } - HashMapEntry::Vacant(vacant_entry) => { - self.counters.miss += 1; - if let Some(mmap) = open_mmap(full_path)? { - vacant_entry.insert(mmap.clone()); - Some(mmap) - } else { - None + // Returns None if the file exists but as a len of 0 (and hence is not mmappable). + fn get_mmap(&mut self, full_path: &Path) -> Result>, OpenReadError> { + let path_in_cache = self.cache.contains_key(full_path); + if path_in_cache { + { + let mmap_weak_opt = self.cache.get(full_path); + if let Some(mmap_arc) = mmap_weak_opt.and_then(|mmap_weak| mmap_weak.upgrade()) { + self.counters.hit += 1; + return Ok(Some(mmap_arc)); } } - }) + self.cache.remove(full_path); + } + self.counters.miss += 1; + if let Some(mmap) = open_mmap(full_path)? { + let mmap_arc: Arc = Arc::new(Box::new(mmap)); + self.cache.insert(full_path.to_owned(), Arc::downgrade(&mmap_arc)); + Ok(Some(mmap_arc)) + } else { + Ok(None) + } } } @@ -282,11 +282,10 @@ impl Directory for MmapDirectory { ); IOError::with_path(path.to_owned(), make_io_err(msg)) })?; - Ok(mmap_cache .get_mmap(&full_path)? - .map(ReadOnlySource::Mmap) - .unwrap_or_else(|| ReadOnlySource::Anonymous(SharedVecSlice::empty()))) + .map(ReadOnlySource::from) + .unwrap_or_else(|| ReadOnlySource::empty())) } fn open_write(&mut self, path: &Path) -> Result { @@ -324,20 +323,6 @@ impl Directory for MmapDirectory { fn delete(&self, path: &Path) -> result::Result<(), DeleteError> { debug!("Deleting file {:?}", path); let full_path = self.resolve_path(path); - let mut mmap_cache = self.mmap_cache.write().map_err(|_| { - let msg = format!( - "Failed to acquired write lock \ - on mmap cache while deleting {:?}", - path - ); - IOError::with_path(path.to_owned(), make_io_err(msg)) - })?; - mmap_cache.discard_from_cache(path); - - // Removing the entry in the MMap cache. - // The munmap will appear on Drop, - // when the last reference is gone. - mmap_cache.cache.remove(&full_path); match fs::remove_file(&full_path) { Ok(_) => self .sync_directory() @@ -452,25 +437,50 @@ mod tests { w.flush().unwrap(); } } - { - for (i, path) in paths.iter().enumerate() { - let _r = mmap_directory.open_read(path).unwrap(); - assert_eq!(mmap_directory.get_cache_info().mmapped.len(), i + 1); - } - for path in paths.iter() { - let _r = mmap_directory.open_read(path).unwrap(); - assert_eq!(mmap_directory.get_cache_info().mmapped.len(), num_paths); - } - for (i, path) in paths.iter().enumerate() { - mmap_directory.delete(path).unwrap(); - assert_eq!( - mmap_directory.get_cache_info().mmapped.len(), - num_paths - i - 1 - ); - } + + let mut keep = vec![]; + for (i, path) in paths.iter().enumerate() { + keep.push(mmap_directory.open_read(path).unwrap()); + assert_eq!(mmap_directory.get_cache_info().mmapped.len(), i + 1); + } + assert_eq!(mmap_directory.get_cache_info().counters.hit, 0); + assert_eq!(mmap_directory.get_cache_info().counters.miss, 10); + assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 10); + for path in paths.iter() { + let _r = mmap_directory.open_read(path).unwrap(); + assert_eq!(mmap_directory.get_cache_info().mmapped.len(), num_paths); } assert_eq!(mmap_directory.get_cache_info().counters.hit, 10); assert_eq!(mmap_directory.get_cache_info().counters.miss, 10); + assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 10); + + for path in paths.iter() { + let _r = mmap_directory.open_read(path).unwrap(); + assert_eq!(mmap_directory.get_cache_info().mmapped.len(), num_paths); + } + assert_eq!(mmap_directory.get_cache_info().counters.hit, 20); + assert_eq!(mmap_directory.get_cache_info().counters.miss, 10); + assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 10); + drop(keep); + for path in paths.iter() { + let _r = mmap_directory.open_read(path).unwrap(); + assert_eq!(mmap_directory.get_cache_info().mmapped.len(), num_paths); + } + assert_eq!(mmap_directory.get_cache_info().counters.hit, 20); + assert_eq!(mmap_directory.get_cache_info().counters.miss, 20); + assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 10); + + for path in &paths { + mmap_directory.delete(path).unwrap(); + } + assert_eq!(mmap_directory.get_cache_info().counters.hit, 20); + assert_eq!(mmap_directory.get_cache_info().counters.miss, 20); + assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 10); + for path in paths.iter() { + assert!(mmap_directory.open_read(path).is_err()); + } + assert_eq!(mmap_directory.get_cache_info().counters.hit, 20); + assert_eq!(mmap_directory.get_cache_info().counters.miss, 30); assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 0); } diff --git a/src/directory/mod.rs b/src/directory/mod.rs index 29fce4fb4..6f7ef88b8 100644 --- a/src/directory/mod.rs +++ b/src/directory/mod.rs @@ -12,7 +12,6 @@ mod directory_lock; mod managed_directory; mod ram_directory; mod read_only_source; -mod shared_vec_slice; /// Errors specific to the directory module. pub mod error; diff --git a/src/directory/ram_directory.rs b/src/directory/ram_directory.rs index 2f1733e0f..9423affff 100644 --- a/src/directory/ram_directory.rs +++ b/src/directory/ram_directory.rs @@ -1,4 +1,3 @@ -use super::shared_vec_slice::SharedVecSlice; use common::make_io_err; use directory::error::{DeleteError, IOError, OpenReadError, OpenWriteError}; use directory::WritePtr; @@ -71,7 +70,7 @@ impl Write for VecWriter { } #[derive(Clone)] -struct InnerDirectory(Arc>>>>); +struct InnerDirectory(Arc>>); impl InnerDirectory { fn new() -> InnerDirectory { @@ -85,7 +84,7 @@ impl InnerDirectory { path )) })?; - let prev_value = map.insert(path, Arc::new(Vec::from(data))); + let prev_value = map.insert(path, ReadOnlySource::new(Vec::from(data))); Ok(prev_value.is_some()) } @@ -105,8 +104,7 @@ impl InnerDirectory { readable_map .get(path) .ok_or_else(|| OpenReadError::FileDoesNotExist(PathBuf::from(path))) - .map(Arc::clone) - .map(|data| ReadOnlySource::Anonymous(SharedVecSlice::new(data))) + .map(|el| el.clone()) }) } diff --git a/src/directory/read_only_source.rs b/src/directory/read_only_source.rs index 6ed2049e5..8a1dc3ab8 100644 --- a/src/directory/read_only_source.rs +++ b/src/directory/read_only_source.rs @@ -1,9 +1,9 @@ -use super::shared_vec_slice::SharedVecSlice; use common::HasLen; -#[cfg(feature = "mmap")] -use fst::raw::MmapReadOnly; use stable_deref_trait::{CloneStableDeref, StableDeref}; use std::ops::Deref; +use std::sync::Arc; + +pub type BoxedData = Box + Send + Sync + 'static>; /// Read object that represents files in tantivy. /// @@ -11,12 +11,10 @@ use std::ops::Deref; /// the data in the form of a constant read-only `&[u8]`. /// Whatever happens to the directory file, the data /// hold by this object should never be altered or destroyed. -pub enum ReadOnlySource { - /// Mmap source of data - #[cfg(feature = "mmap")] - Mmap(MmapReadOnly), - /// Wrapping a `Vec` - Anonymous(SharedVecSlice), +pub struct ReadOnlySource { + data: Arc, + start: usize, + stop: usize } unsafe impl StableDeref for ReadOnlySource {} @@ -30,19 +28,38 @@ impl Deref for ReadOnlySource { } } + +impl From> for ReadOnlySource { + fn from(data: Arc) -> Self { + let len = data.len(); + ReadOnlySource { + data, + start: 0, + stop: len + } + } +} + impl ReadOnlySource { + + pub(crate) fn new(data: D) -> ReadOnlySource + where D: Deref + Send + Sync + 'static { + let len = data.len(); + ReadOnlySource { + data: Arc::new(Box::new(data)), + start: 0, + stop: len + } + } + /// Creates an empty ReadOnlySource pub fn empty() -> ReadOnlySource { - ReadOnlySource::Anonymous(SharedVecSlice::empty()) + ReadOnlySource::new(&[][..]) } /// Returns the data underlying the ReadOnlySource object. pub fn as_slice(&self) -> &[u8] { - match *self { - #[cfg(feature = "mmap")] - ReadOnlySource::Mmap(ref mmap_read_only) => mmap_read_only.as_slice(), - ReadOnlySource::Anonymous(ref shared_vec) => shared_vec.as_slice(), - } + &self.data[self.start..self.stop] } /// Splits into 2 `ReadOnlySource`, at the offset given @@ -63,22 +80,18 @@ impl ReadOnlySource { /// worth of data in anonymous memory, and only a /// 1KB slice is remaining, the whole `500MBs` /// are retained in memory. - pub fn slice(&self, from_offset: usize, to_offset: usize) -> ReadOnlySource { + pub fn slice(&self, start: usize, stop: usize) -> ReadOnlySource { assert!( - from_offset <= to_offset, + start <= stop, "Requested negative slice [{}..{}]", - from_offset, - to_offset + start, + stop ); - match *self { - #[cfg(feature = "mmap")] - ReadOnlySource::Mmap(ref mmap_read_only) => { - let sliced_mmap = mmap_read_only.range(from_offset, to_offset - from_offset); - ReadOnlySource::Mmap(sliced_mmap) - } - ReadOnlySource::Anonymous(ref shared_vec) => { - ReadOnlySource::Anonymous(shared_vec.slice(from_offset, to_offset)) - } + assert!(stop <= self.len()); + ReadOnlySource { + data: self.data.clone(), + start: self.start + start, + stop: self.start + stop } } @@ -87,8 +100,7 @@ impl ReadOnlySource { /// /// Equivalent to `.slice(from_offset, self.len())` pub fn slice_from(&self, from_offset: usize) -> ReadOnlySource { - let len = self.len(); - self.slice(from_offset, len) + self.slice(from_offset, self.len()) } /// Like `.slice(...)` but enforcing only the `to` @@ -102,19 +114,18 @@ impl ReadOnlySource { impl HasLen for ReadOnlySource { fn len(&self) -> usize { - self.as_slice().len() + self.stop - self.start } } impl Clone for ReadOnlySource { fn clone(&self) -> Self { - self.slice(0, self.len()) + self.slice_from(0) } } impl From> for ReadOnlySource { fn from(data: Vec) -> ReadOnlySource { - let shared_data = SharedVecSlice::from(data); - ReadOnlySource::Anonymous(shared_data) + ReadOnlySource::new(data) } -} +} \ No newline at end of file diff --git a/src/directory/shared_vec_slice.rs b/src/directory/shared_vec_slice.rs deleted file mode 100644 index 1a9157e14..000000000 --- a/src/directory/shared_vec_slice.rs +++ /dev/null @@ -1,41 +0,0 @@ -use std::sync::Arc; - -#[derive(Clone)] -pub struct SharedVecSlice { - pub data: Arc>, - pub start: usize, - pub len: usize, -} - -impl SharedVecSlice { - pub fn empty() -> SharedVecSlice { - SharedVecSlice::new(Arc::new(Vec::new())) - } - - pub fn new(data: Arc>) -> SharedVecSlice { - let data_len = data.len(); - SharedVecSlice { - data, - start: 0, - len: data_len, - } - } - - pub fn as_slice(&self) -> &[u8] { - &self.data[self.start..self.start + self.len] - } - - pub fn slice(&self, from_offset: usize, to_offset: usize) -> SharedVecSlice { - SharedVecSlice { - data: Arc::clone(&self.data), - start: self.start + from_offset, - len: to_offset - from_offset, - } - } -} - -impl From> for SharedVecSlice { - fn from(data: Vec) -> SharedVecSlice { - SharedVecSlice::new(Arc::new(data)) - } -} diff --git a/src/lib.rs b/src/lib.rs index 6b86ccd76..f9a0662ea 100755 --- a/src/lib.rs +++ b/src/lib.rs @@ -123,6 +123,8 @@ extern crate log; #[macro_use] extern crate failure; +#[cfg(feature = "mmap")] +extern crate memmap; #[cfg(feature = "mmap")] extern crate atomicwrites; extern crate base64; @@ -132,8 +134,7 @@ extern crate byteorder; extern crate combine; extern crate crossbeam; extern crate fnv; -extern crate fst; -extern crate fst_regex; +extern crate tantivy_fst; extern crate futures; extern crate futures_cpupool; extern crate htmlescape; diff --git a/src/postings/recorder.rs b/src/postings/recorder.rs index e81e593dd..ad27ac0b5 100644 --- a/src/postings/recorder.rs +++ b/src/postings/recorder.rs @@ -4,7 +4,6 @@ use postings::FieldSerializer; use std::io; use DocId; -const EMPTY_ARRAY: [u32; 0] = [0u32; 0]; const POSITION_END: u32 = 0; #[derive(Default)] @@ -115,7 +114,7 @@ impl Recorder for NothingRecorder { let buffer = buffer_lender.lend_u8(); self.stack.read_to_end(heap, buffer); for doc in VInt32Reader::new(&buffer[..]) { - serializer.write_doc(doc as u32, 0u32, &EMPTY_ARRAY)?; + serializer.write_doc(doc as u32, 0u32, &[][..])?; } Ok(()) } @@ -168,7 +167,7 @@ impl Recorder for TermFrequencyRecorder { let mut u32_it = VInt32Reader::new(&buffer[..]); while let Some(doc) = u32_it.next() { let term_freq = u32_it.next().unwrap_or(self.current_tf); - serializer.write_doc(doc as u32, term_freq, &EMPTY_ARRAY)?; + serializer.write_doc(doc as u32, term_freq, &[][..])?; } Ok(()) diff --git a/src/postings/segment_postings.rs b/src/postings/segment_postings.rs index a5e260619..0f444a497 100644 --- a/src/postings/segment_postings.rs +++ b/src/postings/segment_postings.rs @@ -2,7 +2,7 @@ use common::BitSet; use common::HasLen; use common::{BinarySerializable, VInt}; use docset::{DocSet, SkipResult}; -use fst::Streamer; +use tantivy_fst::Streamer; use owned_read::OwnedRead; use positions::PositionReader; use postings::compression::compressed_block_size; @@ -16,8 +16,6 @@ use schema::IndexRecordOption; use std::cmp::Ordering; use DocId; -const EMPTY_ARR: [u8; 0] = []; - struct PositionComputer { // store the amount of position int // before reading positions. @@ -369,7 +367,7 @@ impl BlockSegmentPostings { let (skip_data_opt, postings_data) = split_into_skips_and_postings(doc_freq, data); let skip_reader = match skip_data_opt { Some(skip_data) => SkipReader::new(skip_data, record_option), - None => SkipReader::new(OwnedRead::new(&EMPTY_ARR[..]), record_option), + None => SkipReader::new(OwnedRead::new(&[][..]), record_option), }; let doc_freq = doc_freq as usize; let num_vint_docs = doc_freq % COMPRESSION_BLOCK_SIZE; @@ -403,7 +401,7 @@ impl BlockSegmentPostings { if let Some(skip_data) = skip_data_opt { self.skip_reader.reset(skip_data); } else { - self.skip_reader.reset(OwnedRead::new(&EMPTY_ARR[..])) + self.skip_reader.reset(OwnedRead::new(&[][..])) } self.doc_offset = 0; self.doc_freq = doc_freq as usize; @@ -626,7 +624,7 @@ mod tests { use common::HasLen; use core::Index; use docset::DocSet; - use fst::Streamer; + use tantivy_fst::Streamer; use schema::IndexRecordOption; use schema::Schema; use schema::Term; diff --git a/src/query/automaton_weight.rs b/src/query/automaton_weight.rs index e0963d605..a952ff6e7 100644 --- a/src/query/automaton_weight.rs +++ b/src/query/automaton_weight.rs @@ -1,6 +1,6 @@ use common::BitSet; use core::SegmentReader; -use fst::Automaton; +use tantivy_fst::Automaton; use query::BitSetDocSet; use query::ConstScorer; use query::{Scorer, Weight}; diff --git a/src/query/regex_query.rs b/src/query/regex_query.rs index caa8f080a..4905be987 100644 --- a/src/query/regex_query.rs +++ b/src/query/regex_query.rs @@ -1,5 +1,5 @@ use error::TantivyError; -use fst_regex::Regex; +use tantivy_fst::Regex; use query::{AutomatonWeight, Query, Weight}; use schema::Field; use std::clone::Clone; diff --git a/src/query/vec_docset.rs b/src/query/vec_docset.rs index cc3f991da..906d3021a 100644 --- a/src/query/vec_docset.rs +++ b/src/query/vec_docset.rs @@ -5,8 +5,6 @@ use docset::DocSet; use std::num::Wrapping; use DocId; -const EMPTY_ARRAY: [u32; 0] = []; - /// Simulate a `Postings` objects from a `VecPostings`. /// `VecPostings` only exist for testing purposes. /// diff --git a/src/termdict/streamer.rs b/src/termdict/streamer.rs index f1dc74532..2f6816eed 100644 --- a/src/termdict/streamer.rs +++ b/src/termdict/streamer.rs @@ -1,8 +1,8 @@ use super::TermDictionary; -use fst::automaton::AlwaysMatch; -use fst::map::{Stream, StreamBuilder}; -use fst::Automaton; -use fst::{IntoStreamer, Streamer}; +use tantivy_fst::automaton::AlwaysMatch; +use tantivy_fst::map::{Stream, StreamBuilder}; +use tantivy_fst::Automaton; +use tantivy_fst::{IntoStreamer, Streamer}; use postings::TermInfo; use termdict::TermOrdinal; diff --git a/src/termdict/termdict.rs b/src/termdict/termdict.rs index b63bb54d0..55fc3a7cf 100644 --- a/src/termdict/termdict.rs +++ b/src/termdict/termdict.rs @@ -3,15 +3,15 @@ use super::{TermStreamer, TermStreamerBuilder}; use common::BinarySerializable; use common::CountingWriter; use directory::ReadOnlySource; -use fst; -use fst::raw::Fst; -use fst::Automaton; +use tantivy_fst; +use tantivy_fst::raw::Fst; +use tantivy_fst::Automaton; use postings::TermInfo; use schema::FieldType; use std::io::{self, Write}; use termdict::TermOrdinal; -fn convert_fst_error(e: fst::Error) -> io::Error { +fn convert_fst_error(e: tantivy_fst::Error) -> io::Error { io::Error::new(io::ErrorKind::Other, e) } @@ -19,7 +19,7 @@ fn convert_fst_error(e: fst::Error) -> io::Error { /// /// Inserting must be done in the order of the `keys`. pub struct TermDictionaryBuilder { - fst_builder: fst::MapBuilder, + fst_builder: tantivy_fst::MapBuilder, term_info_store_writer: TermInfoStoreWriter, term_ord: u64, } @@ -30,7 +30,7 @@ where { /// Creates a new `TermDictionaryBuilder` pub fn create(w: W, _field_type: &FieldType) -> io::Result { - let fst_builder = fst::MapBuilder::new(w).map_err(convert_fst_error)?; + let fst_builder = tantivy_fst::MapBuilder::new(w).map_err(convert_fst_error)?; Ok(TermDictionaryBuilder { fst_builder, term_info_store_writer: TermInfoStoreWriter::new(), @@ -87,17 +87,9 @@ where } } -fn open_fst_index(source: ReadOnlySource) -> fst::Map { - let fst = match source { - ReadOnlySource::Anonymous(data) => { - Fst::from_shared_bytes(data.data, data.start, data.len).expect("FST data is corrupted") - } - #[cfg(feature = "mmap")] - ReadOnlySource::Mmap(mmap_readonly) => { - Fst::from_mmap(mmap_readonly).expect("FST data is corrupted") - } - }; - fst::Map::from(fst) +fn open_fst_index(source: ReadOnlySource) -> tantivy_fst::Map { + let fst = Fst::new(source).expect("FST data is corrupted"); + tantivy_fst::Map::from(fst) } /// The term dictionary contains all of the terms in @@ -107,7 +99,7 @@ fn open_fst_index(source: ReadOnlySource) -> fst::Map { /// respective `TermOrdinal`. The `TermInfoStore` then makes it /// possible to fetch the associated `TermInfo`. pub struct TermDictionary { - fst_index: fst::Map, + fst_index: tantivy_fst::Map, term_info_store: TermInfoStore, } From fdefea9e26151b261347a606922573ab3783a658 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Sat, 23 Feb 2019 10:42:44 +0900 Subject: [PATCH 06/26] Removed path reference to tantivy-fst --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index da43f7ee3..15fcdc747 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -16,7 +16,7 @@ base64 = "0.10.0" byteorder = "1.0" lazy_static = "1" regex = "1.0" -tantivy-fst = {path="../tantivy-search/fst", version="0.1"} +tantivy-fst = "0.1" memmap = "0.7" fst-regex = { version="0.2" } lz4 = {version="1.20", optional=true} From 27c9fa6028b255c0b3aaa17c6901141dc19e9ffd Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Mon, 25 Feb 2019 22:33:17 +0900 Subject: [PATCH 07/26] Jannickj prove bug with facets (#508) * prove bug with facets * Closing #505 Introduce a term id in the TermHashMap --- src/fastfield/multivalued/mod.rs | 14 +++++++++ src/fastfield/multivalued/writer.rs | 2 +- src/indexer/index_writer.rs | 6 ++-- src/postings/mod.rs | 1 - src/postings/postings_writer.rs | 46 ++++++++++++++++------------ src/postings/stacker/memory_arena.rs | 1 + src/postings/stacker/mod.rs | 2 +- src/postings/stacker/term_hashmap.rs | 26 ++++++++++------ 8 files changed, 63 insertions(+), 35 deletions(-) diff --git a/src/fastfield/multivalued/mod.rs b/src/fastfield/multivalued/mod.rs index 733f211bb..c1cd2dd7f 100644 --- a/src/fastfield/multivalued/mod.rs +++ b/src/fastfield/multivalued/mod.rs @@ -8,6 +8,7 @@ pub use self::writer::MultiValueIntFastFieldWriter; mod tests { use schema::Cardinality; + use schema::Facet; use schema::IntOptions; use schema::Schema; use Index; @@ -85,4 +86,17 @@ mod tests { assert_eq!(&vals, &[-5i64, -20i64, 1i64]); } } + #[test] + #[ignore] + fn test_many_facets() { + let mut schema_builder = Schema::builder(); + let field = schema_builder.add_facet_field("facetfield"); + let schema = schema_builder.build(); + let index = Index::create_in_ram(schema); + let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); + for i in 0..100_000 { + index_writer.add_document(doc!(field=> Facet::from(format!("/lang/{}", i).as_str()))); + } + assert!(index_writer.commit().is_ok()); + } } diff --git a/src/fastfield/multivalued/writer.rs b/src/fastfield/multivalued/writer.rs index e5fd45203..a4186ffd7 100644 --- a/src/fastfield/multivalued/writer.rs +++ b/src/fastfield/multivalued/writer.rs @@ -32,7 +32,7 @@ use DocId; /// term ids when the segment is getting serialized. pub struct MultiValueIntFastFieldWriter { field: Field, - vals: Vec, + vals: Vec, doc_index: Vec, is_facet: bool, } diff --git a/src/indexer/index_writer.rs b/src/indexer/index_writer.rs index 8a43b0c3a..2272fb394 100644 --- a/src/indexer/index_writer.rs +++ b/src/indexer/index_writer.rs @@ -1008,9 +1008,9 @@ mod tests { #[test] fn test_hashmap_size() { - assert_eq!(initial_table_size(100_000), 12); - assert_eq!(initial_table_size(1_000_000), 15); - assert_eq!(initial_table_size(10_000_000), 18); + assert_eq!(initial_table_size(100_000), 11); + assert_eq!(initial_table_size(1_000_000), 14); + assert_eq!(initial_table_size(10_000_000), 17); assert_eq!(initial_table_size(1_000_000_000), 19); } diff --git a/src/postings/mod.rs b/src/postings/mod.rs index c94e887a9..eddb10799 100644 --- a/src/postings/mod.rs +++ b/src/postings/mod.rs @@ -31,7 +31,6 @@ pub(crate) use self::stacker::compute_table_size; pub use common::HasLen; pub(crate) const USE_SKIP_INFO_LIMIT: u32 = COMPRESSION_BLOCK_SIZE as u32; - pub(crate) type UnorderedTermId = u64; #[cfg_attr(feature = "cargo-clippy", allow(clippy::enum_variant_names))] diff --git a/src/postings/postings_writer.rs b/src/postings/postings_writer.rs index c408c15df..d70d148a5 100644 --- a/src/postings/postings_writer.rs +++ b/src/postings/postings_writer.rs @@ -51,6 +51,30 @@ pub struct MultiFieldPostingsWriter { per_field_postings_writers: Vec>, } + +fn make_field_partition(term_offsets: & Vec<(&[u8], Addr, UnorderedTermId)>) -> Vec<(Field, usize, usize)> { + let term_offsets_it = term_offsets + .iter() + .map(|(key, _, _)| Term::wrap(key).field()) + .enumerate(); + let mut prev_field = Field(u32::max_value()); + let mut fields = vec![]; + let mut offsets = vec![]; + for (offset, field) in term_offsets_it { + if field != prev_field { + prev_field = field; + fields.push(field); + offsets.push(offset); + } + } + offsets .push(term_offsets.len()); + let mut field_offsets = vec![]; + for i in 0..fields.len() { + field_offsets.push((fields[i], offsets[i] , offsets[i+1])); + } + field_offsets +} + impl MultiFieldPostingsWriter { /// Create a new `MultiFieldPostingsWriter` given /// a schema and a heap. @@ -99,33 +123,15 @@ impl MultiFieldPostingsWriter { let mut term_offsets: Vec<(&[u8], Addr, UnorderedTermId)> = self .term_index .iter() - .map(|(term_bytes, addr, bucket_id)| (term_bytes, addr, bucket_id as UnorderedTermId)) .collect(); term_offsets.sort_unstable_by_key(|&(k, _, _)| k); - let mut offsets: Vec<(Field, usize)> = vec![]; - let term_offsets_it = term_offsets - .iter() - .cloned() - .map(|(key, _, _)| Term::wrap(key).field()) - .enumerate(); - let mut unordered_term_mappings: HashMap> = HashMap::new(); - let mut prev_field = Field(u32::max_value()); - for (offset, field) in term_offsets_it { - if field != prev_field { - offsets.push((field, offset)); - prev_field = field; - } - } - offsets.push((Field(0), term_offsets.len())); - - for i in 0..(offsets.len() - 1) { - let (field, start) = offsets[i]; - let (_, stop) = offsets[i + 1]; + let field_offsets = make_field_partition(&term_offsets); + for (field, start, stop) in field_offsets { let field_entry = self.schema.get_field_entry(field); match *field_entry.field_type() { diff --git a/src/postings/stacker/memory_arena.rs b/src/postings/stacker/memory_arena.rs index 816492d28..973c1dbd0 100644 --- a/src/postings/stacker/memory_arena.rs +++ b/src/postings/stacker/memory_arena.rs @@ -40,6 +40,7 @@ const PAGE_SIZE: usize = 1 << NUM_BITS_PAGE_ADDR; // pages are 1 MB large #[derive(Copy, Clone, Debug)] pub struct Addr(u32); + impl Addr { /// Creates a null pointer. pub fn null_pointer() -> Addr { diff --git a/src/postings/stacker/mod.rs b/src/postings/stacker/mod.rs index be80510d7..0955f4e32 100644 --- a/src/postings/stacker/mod.rs +++ b/src/postings/stacker/mod.rs @@ -4,4 +4,4 @@ mod term_hashmap; pub use self::expull::ExpUnrolledLinkedList; pub use self::memory_arena::{Addr, MemoryArena}; -pub use self::term_hashmap::{compute_table_size, TermHashMap}; +pub use self::term_hashmap::{compute_table_size, TermHashMap}; \ No newline at end of file diff --git a/src/postings/stacker/term_hashmap.rs b/src/postings/stacker/term_hashmap.rs index 072f8d77c..b2c7a26c9 100644 --- a/src/postings/stacker/term_hashmap.rs +++ b/src/postings/stacker/term_hashmap.rs @@ -8,8 +8,7 @@ use postings::stacker::memory_arena::store; use std::iter; use std::mem; use std::slice; - -pub type BucketId = usize; +use postings::UnorderedTermId; /// Returns the actual memory size in bytes /// required to create a table of size $2^num_bits$. @@ -28,6 +27,7 @@ pub fn compute_table_size(num_bits: usize) -> usize { struct KeyValue { key_value_addr: Addr, hash: u32, + unordered_term_id: UnorderedTermId, } impl Default for KeyValue { @@ -35,6 +35,7 @@ impl Default for KeyValue { KeyValue { key_value_addr: Addr::null_pointer(), hash: 0u32, + unordered_term_id: UnorderedTermId::default(), } } } @@ -59,6 +60,7 @@ pub struct TermHashMap { pub heap: MemoryArena, mask: usize, occupied: Vec, + len: usize } struct QuadraticProbing { @@ -85,13 +87,13 @@ pub struct Iter<'a> { } impl<'a> Iterator for Iter<'a> { - type Item = (&'a [u8], Addr, BucketId); + type Item = (&'a [u8], Addr, UnorderedTermId); fn next(&mut self) -> Option { self.inner.next().cloned().map(move |bucket: usize| { let kv = self.hashmap.table[bucket]; let (key, offset): (&'a [u8], Addr) = self.hashmap.get_key_value(kv.key_value_addr); - (key, offset, bucket as BucketId) + (key, offset, kv.unordered_term_id) }) } } @@ -106,6 +108,7 @@ impl TermHashMap { heap, mask: table_size - 1, occupied: Vec::with_capacity(table_size / 2), + len: 0 } } @@ -139,12 +142,16 @@ impl TermHashMap { } } - pub fn set_bucket(&mut self, hash: u32, key_value_addr: Addr, bucket: usize) { + fn set_bucket(&mut self, hash: u32, key_value_addr: Addr, bucket: usize) -> UnorderedTermId { self.occupied.push(bucket); + let unordered_term_id = self.len as UnorderedTermId; + self.len += 1; self.table[bucket] = KeyValue { key_value_addr, hash, + unordered_term_id }; + unordered_term_id } pub fn iter(&self) -> Iter { @@ -184,7 +191,7 @@ impl TermHashMap { /// will be in charge of returning a default value. /// If the key already as an associated value, then it will be passed /// `Some(previous_value)`. - pub fn mutate_or_create(&mut self, key: S, mut updater: TMutator) -> BucketId + pub fn mutate_or_create(&mut self, key: S, mut updater: TMutator) -> UnorderedTermId where S: AsRef<[u8]>, V: Copy + 'static, @@ -200,6 +207,7 @@ impl TermHashMap { let bucket = probe.next_probe(); let kv: KeyValue = self.table[bucket]; if kv.is_empty() { + // The key does not exists yet. let val = updater(None); let num_bytes = std::mem::size_of::() + key_bytes.len() + std::mem::size_of::(); @@ -211,16 +219,16 @@ impl TermHashMap { data[2..stop].copy_from_slice(key_bytes); store(&mut data[stop..], val); } - self.set_bucket(hash, key_addr, bucket); - return bucket as BucketId; + return self.set_bucket(hash, key_addr, bucket); } else if kv.hash == hash { if let Some(val_addr) = self.get_value_addr_if_key_match(key_bytes, kv.key_value_addr) { + let v = self.heap.read(val_addr); let new_v = updater(Some(v)); self.heap.write_at(val_addr, new_v); - return bucket as BucketId; + return kv.unordered_term_id; } } } From 774fcecf2388bccb8e82d943ed18b2ee0d6130d1 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Tue, 26 Feb 2019 10:44:59 +0900 Subject: [PATCH 08/26] cargo fmt --- src/directory/mmap_directory.rs | 10 +++++----- src/directory/read_only_source.rs | 20 ++++++++++---------- src/indexer/index_writer.rs | 16 ++++++++-------- src/lib.rs | 6 +++--- src/positions/reader.rs | 3 ++- src/positions/serializer.rs | 6 ++++-- src/postings/postings_writer.rs | 15 +++++++-------- src/postings/segment_postings.rs | 4 ++-- src/postings/stacker/memory_arena.rs | 1 - src/postings/stacker/mod.rs | 2 +- src/postings/stacker/term_hashmap.rs | 17 ++++++++++------- src/query/automaton_weight.rs | 2 +- src/query/regex_query.rs | 2 +- src/termdict/streamer.rs | 2 +- src/termdict/termdict.rs | 6 +++--- 15 files changed, 58 insertions(+), 54 deletions(-) diff --git a/src/directory/mmap_directory.rs b/src/directory/mmap_directory.rs index fb4fa785c..a843de914 100644 --- a/src/directory/mmap_directory.rs +++ b/src/directory/mmap_directory.rs @@ -5,11 +5,13 @@ use atomicwrites; use common::make_io_err; use directory::error::LockError; use directory::error::{DeleteError, IOError, OpenDirectoryError, OpenReadError, OpenWriteError}; +use directory::read_only_source::BoxedData; use directory::Directory; use directory::DirectoryLock; use directory::Lock; use directory::ReadOnlySource; use directory::WritePtr; +use memmap::Mmap; use std::collections::HashMap; use std::convert::From; use std::fmt; @@ -21,10 +23,8 @@ use std::path::{Path, PathBuf}; use std::result; use std::sync::Arc; use std::sync::RwLock; -use tempdir::TempDir; -use memmap::Mmap; use std::sync::Weak; -use directory::read_only_source::BoxedData; +use tempdir::TempDir; /// Returns None iff the file exists, can be read, but is empty (and hence /// cannot be mmapped). @@ -84,7 +84,6 @@ impl Default for MmapCache { } impl MmapCache { - fn get_info(&mut self) -> CacheInfo { let paths: Vec = self.cache.keys().cloned().collect(); CacheInfo { @@ -109,7 +108,8 @@ impl MmapCache { self.counters.miss += 1; if let Some(mmap) = open_mmap(full_path)? { let mmap_arc: Arc = Arc::new(Box::new(mmap)); - self.cache.insert(full_path.to_owned(), Arc::downgrade(&mmap_arc)); + self.cache + .insert(full_path.to_owned(), Arc::downgrade(&mmap_arc)); Ok(Some(mmap_arc)) } else { Ok(None) diff --git a/src/directory/read_only_source.rs b/src/directory/read_only_source.rs index 8a1dc3ab8..a09e32593 100644 --- a/src/directory/read_only_source.rs +++ b/src/directory/read_only_source.rs @@ -3,7 +3,7 @@ use stable_deref_trait::{CloneStableDeref, StableDeref}; use std::ops::Deref; use std::sync::Arc; -pub type BoxedData = Box + Send + Sync + 'static>; +pub type BoxedData = Box + Send + Sync + 'static>; /// Read object that represents files in tantivy. /// @@ -14,7 +14,7 @@ pub type BoxedData = Box + Send + Sync + 'static>; pub struct ReadOnlySource { data: Arc, start: usize, - stop: usize + stop: usize, } unsafe impl StableDeref for ReadOnlySource {} @@ -28,27 +28,27 @@ impl Deref for ReadOnlySource { } } - impl From> for ReadOnlySource { fn from(data: Arc) -> Self { let len = data.len(); ReadOnlySource { data, start: 0, - stop: len + stop: len, } } } impl ReadOnlySource { - pub(crate) fn new(data: D) -> ReadOnlySource - where D: Deref + Send + Sync + 'static { + where + D: Deref + Send + Sync + 'static, + { let len = data.len(); ReadOnlySource { data: Arc::new(Box::new(data)), start: 0, - stop: len + stop: len, } } @@ -90,8 +90,8 @@ impl ReadOnlySource { assert!(stop <= self.len()); ReadOnlySource { data: self.data.clone(), - start: self.start + start, - stop: self.start + stop + start: self.start + start, + stop: self.start + stop, } } @@ -128,4 +128,4 @@ impl From> for ReadOnlySource { fn from(data: Vec) -> ReadOnlySource { ReadOnlySource::new(data) } -} \ No newline at end of file +} diff --git a/src/indexer/index_writer.rs b/src/indexer/index_writer.rs index 2272fb394..ca14246bf 100644 --- a/src/indexer/index_writer.rs +++ b/src/indexer/index_writer.rs @@ -675,16 +675,16 @@ impl IndexWriter { /// Runs a group of document operations ensuring that the operations are /// assigned contigous u64 opstamps and that add operations of the same /// group are flushed into the same segment. - /// + /// /// If the indexing pipeline is full, this call may block. - /// + /// /// Each operation of the given `user_operations` will receive an in-order, /// contiguous u64 opstamp. The entire batch itself is also given an - /// opstamp that is 1 greater than the last given operation. This + /// opstamp that is 1 greater than the last given operation. This /// `batch_opstamp` is the return value of `run`. An empty group of /// `user_operations`, an empty `Vec`, still receives /// a valid opstamp even though no changes were _actually_ made to the index. - /// + /// /// Like adds and deletes (see `IndexWriter.add_document` and /// `IndexWriter.delete_term`), the changes made by calling `run` will be /// visible to readers only after calling `commit()`. @@ -729,12 +729,12 @@ mod tests { use super::super::operation::UserOperation; use super::initial_table_size; + use collector::TopDocs; use directory::error::LockError; use error::*; use indexer::NoMergePolicy; + use query::TermQuery; use schema::{self, Document, IndexRecordOption}; - use query::{TermQuery}; - use collector::TopDocs; use Index; use Term; @@ -776,10 +776,10 @@ mod tests { index_writer.run(operations); index_writer.commit().expect("failed to commit"); index.load_searchers().expect("failed to load searchers"); - + let a_term = Term::from_field_text(text_field, "a"); let b_term = Term::from_field_text(text_field, "b"); - + let a_query = TermQuery::new(a_term, IndexRecordOption::Basic); let b_query = TermQuery::new(b_term, IndexRecordOption::Basic); diff --git a/src/lib.rs b/src/lib.rs index f9a0662ea..ae1f45d3d 100755 --- a/src/lib.rs +++ b/src/lib.rs @@ -123,8 +123,6 @@ extern crate log; #[macro_use] extern crate failure; -#[cfg(feature = "mmap")] -extern crate memmap; #[cfg(feature = "mmap")] extern crate atomicwrites; extern crate base64; @@ -134,12 +132,13 @@ extern crate byteorder; extern crate combine; extern crate crossbeam; extern crate fnv; -extern crate tantivy_fst; extern crate futures; extern crate futures_cpupool; extern crate htmlescape; extern crate itertools; extern crate levenshtein_automata; +#[cfg(feature = "mmap")] +extern crate memmap; extern crate num_cpus; extern crate owning_ref; extern crate regex; @@ -147,6 +146,7 @@ extern crate rust_stemmers; extern crate scoped_pool; extern crate serde; extern crate stable_deref_trait; +extern crate tantivy_fst; extern crate tempdir; extern crate tempfile; extern crate uuid; diff --git a/src/positions/reader.rs b/src/positions/reader.rs index 750d44cb1..cd8b5f950 100644 --- a/src/positions/reader.rs +++ b/src/positions/reader.rs @@ -151,7 +151,8 @@ impl PositionReader { if self.ahead != Some(0) { // the block currently available is not the block // for the current position - self.bit_packer.decompress(position_data, self.buffer.as_mut(), num_bits); + self.bit_packer + .decompress(position_data, self.buffer.as_mut(), num_bits); self.ahead = Some(0); } let block_len = compressed_block_size(num_bits); diff --git a/src/positions/serializer.rs b/src/positions/serializer.rs index 79660ff05..773be5e14 100644 --- a/src/positions/serializer.rs +++ b/src/positions/serializer.rs @@ -1,9 +1,9 @@ use bitpacking::BitPacker; +use bitpacking::BitPacker4x; use common::BinarySerializable; use common::CountingWriter; use positions::{COMPRESSION_BLOCK_SIZE, LONG_SKIP_INTERVAL}; use std::io::{self, Write}; -use bitpacking::BitPacker4x; pub struct PositionSerializer { bit_packer: BitPacker4x, @@ -53,7 +53,9 @@ impl PositionSerializer { fn flush_block(&mut self) -> io::Result<()> { let num_bits = self.bit_packer.num_bits(&self.block[..]); self.write_skiplist.write_all(&[num_bits])?; - let written_len = self.bit_packer.compress(&self.block[..], &mut self.buffer, num_bits); + let written_len = self + .bit_packer + .compress(&self.block[..], &mut self.buffer, num_bits); self.write_stream.write_all(&self.buffer[..written_len])?; self.block.clear(); if (self.num_ints % LONG_SKIP_INTERVAL) == 0u64 { diff --git a/src/postings/postings_writer.rs b/src/postings/postings_writer.rs index d70d148a5..4f041d336 100644 --- a/src/postings/postings_writer.rs +++ b/src/postings/postings_writer.rs @@ -51,8 +51,9 @@ pub struct MultiFieldPostingsWriter { per_field_postings_writers: Vec>, } - -fn make_field_partition(term_offsets: & Vec<(&[u8], Addr, UnorderedTermId)>) -> Vec<(Field, usize, usize)> { +fn make_field_partition( + term_offsets: &Vec<(&[u8], Addr, UnorderedTermId)>, +) -> Vec<(Field, usize, usize)> { let term_offsets_it = term_offsets .iter() .map(|(key, _, _)| Term::wrap(key).field()) @@ -67,10 +68,10 @@ fn make_field_partition(term_offsets: & Vec<(&[u8], Addr, UnorderedTermId)>) -> offsets.push(offset); } } - offsets .push(term_offsets.len()); + offsets.push(term_offsets.len()); let mut field_offsets = vec![]; for i in 0..fields.len() { - field_offsets.push((fields[i], offsets[i] , offsets[i+1])); + field_offsets.push((fields[i], offsets[i], offsets[i + 1])); } field_offsets } @@ -120,10 +121,8 @@ impl MultiFieldPostingsWriter { &self, serializer: &mut InvertedIndexSerializer, ) -> Result>> { - let mut term_offsets: Vec<(&[u8], Addr, UnorderedTermId)> = self - .term_index - .iter() - .collect(); + let mut term_offsets: Vec<(&[u8], Addr, UnorderedTermId)> = + self.term_index.iter().collect(); term_offsets.sort_unstable_by_key(|&(k, _, _)| k); let mut unordered_term_mappings: HashMap> = diff --git a/src/postings/segment_postings.rs b/src/postings/segment_postings.rs index 0f444a497..90c72f5f9 100644 --- a/src/postings/segment_postings.rs +++ b/src/postings/segment_postings.rs @@ -2,7 +2,6 @@ use common::BitSet; use common::HasLen; use common::{BinarySerializable, VInt}; use docset::{DocSet, SkipResult}; -use tantivy_fst::Streamer; use owned_read::OwnedRead; use positions::PositionReader; use postings::compression::compressed_block_size; @@ -14,6 +13,7 @@ use postings::SkipReader; use postings::USE_SKIP_INFO_LIMIT; use schema::IndexRecordOption; use std::cmp::Ordering; +use tantivy_fst::Streamer; use DocId; struct PositionComputer { @@ -624,11 +624,11 @@ mod tests { use common::HasLen; use core::Index; use docset::DocSet; - use tantivy_fst::Streamer; use schema::IndexRecordOption; use schema::Schema; use schema::Term; use schema::INT_INDEXED; + use tantivy_fst::Streamer; use DocId; use SkipResult; diff --git a/src/postings/stacker/memory_arena.rs b/src/postings/stacker/memory_arena.rs index 973c1dbd0..816492d28 100644 --- a/src/postings/stacker/memory_arena.rs +++ b/src/postings/stacker/memory_arena.rs @@ -40,7 +40,6 @@ const PAGE_SIZE: usize = 1 << NUM_BITS_PAGE_ADDR; // pages are 1 MB large #[derive(Copy, Clone, Debug)] pub struct Addr(u32); - impl Addr { /// Creates a null pointer. pub fn null_pointer() -> Addr { diff --git a/src/postings/stacker/mod.rs b/src/postings/stacker/mod.rs index 0955f4e32..be80510d7 100644 --- a/src/postings/stacker/mod.rs +++ b/src/postings/stacker/mod.rs @@ -4,4 +4,4 @@ mod term_hashmap; pub use self::expull::ExpUnrolledLinkedList; pub use self::memory_arena::{Addr, MemoryArena}; -pub use self::term_hashmap::{compute_table_size, TermHashMap}; \ No newline at end of file +pub use self::term_hashmap::{compute_table_size, TermHashMap}; diff --git a/src/postings/stacker/term_hashmap.rs b/src/postings/stacker/term_hashmap.rs index b2c7a26c9..50e30e559 100644 --- a/src/postings/stacker/term_hashmap.rs +++ b/src/postings/stacker/term_hashmap.rs @@ -5,10 +5,10 @@ use self::murmurhash32::murmurhash2; use super::{Addr, MemoryArena}; use byteorder::{ByteOrder, NativeEndian}; use postings::stacker::memory_arena::store; +use postings::UnorderedTermId; use std::iter; use std::mem; use std::slice; -use postings::UnorderedTermId; /// Returns the actual memory size in bytes /// required to create a table of size $2^num_bits$. @@ -60,7 +60,7 @@ pub struct TermHashMap { pub heap: MemoryArena, mask: usize, occupied: Vec, - len: usize + len: usize, } struct QuadraticProbing { @@ -108,7 +108,7 @@ impl TermHashMap { heap, mask: table_size - 1, occupied: Vec::with_capacity(table_size / 2), - len: 0 + len: 0, } } @@ -144,12 +144,12 @@ impl TermHashMap { fn set_bucket(&mut self, hash: u32, key_value_addr: Addr, bucket: usize) -> UnorderedTermId { self.occupied.push(bucket); - let unordered_term_id = self.len as UnorderedTermId; + let unordered_term_id = self.len as UnorderedTermId; self.len += 1; self.table[bucket] = KeyValue { key_value_addr, hash, - unordered_term_id + unordered_term_id, }; unordered_term_id } @@ -191,7 +191,11 @@ impl TermHashMap { /// will be in charge of returning a default value. /// If the key already as an associated value, then it will be passed /// `Some(previous_value)`. - pub fn mutate_or_create(&mut self, key: S, mut updater: TMutator) -> UnorderedTermId + pub fn mutate_or_create( + &mut self, + key: S, + mut updater: TMutator, + ) -> UnorderedTermId where S: AsRef<[u8]>, V: Copy + 'static, @@ -224,7 +228,6 @@ impl TermHashMap { if let Some(val_addr) = self.get_value_addr_if_key_match(key_bytes, kv.key_value_addr) { - let v = self.heap.read(val_addr); let new_v = updater(Some(v)); self.heap.write_at(val_addr, new_v); diff --git a/src/query/automaton_weight.rs b/src/query/automaton_weight.rs index a952ff6e7..868f38bc1 100644 --- a/src/query/automaton_weight.rs +++ b/src/query/automaton_weight.rs @@ -1,10 +1,10 @@ use common::BitSet; use core::SegmentReader; -use tantivy_fst::Automaton; use query::BitSetDocSet; use query::ConstScorer; use query::{Scorer, Weight}; use schema::{Field, IndexRecordOption}; +use tantivy_fst::Automaton; use termdict::{TermDictionary, TermStreamer}; use Result; diff --git a/src/query/regex_query.rs b/src/query/regex_query.rs index 4905be987..3d3254b2f 100644 --- a/src/query/regex_query.rs +++ b/src/query/regex_query.rs @@ -1,8 +1,8 @@ use error::TantivyError; -use tantivy_fst::Regex; use query::{AutomatonWeight, Query, Weight}; use schema::Field; use std::clone::Clone; +use tantivy_fst::Regex; use Result; use Searcher; diff --git a/src/termdict/streamer.rs b/src/termdict/streamer.rs index 2f6816eed..6dce8b78d 100644 --- a/src/termdict/streamer.rs +++ b/src/termdict/streamer.rs @@ -1,9 +1,9 @@ use super::TermDictionary; +use postings::TermInfo; use tantivy_fst::automaton::AlwaysMatch; use tantivy_fst::map::{Stream, StreamBuilder}; use tantivy_fst::Automaton; use tantivy_fst::{IntoStreamer, Streamer}; -use postings::TermInfo; use termdict::TermOrdinal; /// `TermStreamerBuilder` is a helper object used to define diff --git a/src/termdict/termdict.rs b/src/termdict/termdict.rs index 55fc3a7cf..43ab2eac5 100644 --- a/src/termdict/termdict.rs +++ b/src/termdict/termdict.rs @@ -3,12 +3,12 @@ use super::{TermStreamer, TermStreamerBuilder}; use common::BinarySerializable; use common::CountingWriter; use directory::ReadOnlySource; -use tantivy_fst; -use tantivy_fst::raw::Fst; -use tantivy_fst::Automaton; use postings::TermInfo; use schema::FieldType; use std::io::{self, Write}; +use tantivy_fst; +use tantivy_fst::raw::Fst; +use tantivy_fst::Automaton; use termdict::TermOrdinal; fn convert_fst_error(e: tantivy_fst::Error) -> io::Error { From 9acadb37563913ccdb134cc1585b5fdcf74b5b41 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Tue, 26 Feb 2019 10:50:36 +0900 Subject: [PATCH 09/26] Code cleaning --- src/common/bitpacker.rs | 2 +- src/directory/mmap_directory.rs | 2 +- src/fastfield/reader.rs | 4 ++-- src/indexer/index_writer.rs | 12 ++++++------ src/indexer/stamper.rs | 2 +- src/postings/postings_writer.rs | 2 +- 6 files changed, 12 insertions(+), 12 deletions(-) diff --git a/src/common/bitpacker.rs b/src/common/bitpacker.rs index ca23ca817..9ec7241e4 100644 --- a/src/common/bitpacker.rs +++ b/src/common/bitpacker.rs @@ -80,7 +80,7 @@ where (1u64 << num_bits) - 1u64 }; BitUnpacker { - num_bits: num_bits as u64, + num_bits: u64::from(num_bits) , mask, data, } diff --git a/src/directory/mmap_directory.rs b/src/directory/mmap_directory.rs index a843de914..8489f806f 100644 --- a/src/directory/mmap_directory.rs +++ b/src/directory/mmap_directory.rs @@ -285,7 +285,7 @@ impl Directory for MmapDirectory { Ok(mmap_cache .get_mmap(&full_path)? .map(ReadOnlySource::from) - .unwrap_or_else(|| ReadOnlySource::empty())) + .unwrap_or_else(ReadOnlySource::empty)) } fn open_write(&mut self, path: &Path) -> Result { diff --git a/src/fastfield/reader.rs b/src/fastfield/reader.rs index 5c14b3330..f0af44dd0 100644 --- a/src/fastfield/reader.rs +++ b/src/fastfield/reader.rs @@ -59,7 +59,7 @@ impl FastFieldReader { /// May panic if `doc` is greater than the segment // `maxdoc`. pub fn get(&self, doc: DocId) -> Item { - self.get_u64(doc as u64) + self.get_u64(u64::from(doc )) } pub(crate) fn get_u64(&self, doc: u64) -> Item { @@ -98,7 +98,7 @@ impl FastFieldReader { /// May panic if `start + output.len()` is greater than /// the segment's `maxdoc`. pub fn get_range(&self, start: DocId, output: &mut [Item]) { - self.get_range_u64(start as u64, output); + self.get_range_u64(u64::from(start), output); } /// Returns the minimum value for this fast field. diff --git a/src/indexer/index_writer.rs b/src/indexer/index_writer.rs index ca14246bf..854317e38 100644 --- a/src/indexer/index_writer.rs +++ b/src/indexer/index_writer.rs @@ -666,7 +666,7 @@ impl IndexWriter { let Range { start, end } = self.stamper.stamps(count + 1u64); let last_opstamp = end - 1; let stamps = Range { - start: start, + start, end: last_opstamp, }; (last_opstamp, stamps) @@ -701,15 +701,15 @@ impl IndexWriter { match user_op { UserOperation::Delete(term) => { let delete_operation = DeleteOperation { - opstamp: opstamp, - term: term, + opstamp, + term, }; self.delete_queue.push(delete_operation); } - UserOperation::Add(doc) => { + UserOperation::Add(document) => { let add_operation = AddOperation { - opstamp: opstamp, - document: doc, + opstamp, + document }; adds.push(add_operation); } diff --git a/src/indexer/stamper.rs b/src/indexer/stamper.rs index 69b4c51ef..6cd1c494a 100644 --- a/src/indexer/stamper.rs +++ b/src/indexer/stamper.rs @@ -67,7 +67,7 @@ impl Stamper { pub fn stamps(&self, n: u64) -> Range { let start = self.0.fetch_add(n, Ordering::SeqCst); Range { - start: start, + start, end: start + n, } } diff --git a/src/postings/postings_writer.rs b/src/postings/postings_writer.rs index 4f041d336..def366812 100644 --- a/src/postings/postings_writer.rs +++ b/src/postings/postings_writer.rs @@ -52,7 +52,7 @@ pub struct MultiFieldPostingsWriter { } fn make_field_partition( - term_offsets: &Vec<(&[u8], Addr, UnorderedTermId)>, + term_offsets: &[(&[u8], Addr, UnorderedTermId)] ) -> Vec<(Field, usize, usize)> { let term_offsets_it = term_offsets .iter() From a814a31f1e20b0714a95144e234c66e95a17d10d Mon Sep 17 00:00:00 2001 From: Thomas Schaller Date: Thu, 28 Feb 2019 01:20:43 +0100 Subject: [PATCH 10/26] Remove semicolon from doc! expansion (#509) --- src/macros.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/macros.rs b/src/macros.rs index c86d293a8..7752093ed 100644 --- a/src/macros.rs +++ b/src/macros.rs @@ -61,7 +61,7 @@ macro_rules! doc( }; // if there is a trailing comma retry with the trailing comma stripped. ($($field:expr => $value:expr),+ ,) => { - doc!( $( $field => $value ), *); + doc!( $( $field => $value ), *) }; ); From ec73a9a284cc1938de3b747ec2e10a1e1eac34ce Mon Sep 17 00:00:00 2001 From: Mauri de Souza Nunes Date: Wed, 27 Feb 2019 21:23:00 -0300 Subject: [PATCH 11/26] Remove note about panicking in get_field docs (#503) Since get_field rely on calling get on the underlying InnerSchema HashMap it shouldn't fail if the field was not found, it simply returns None. --- src/schema/schema.rs | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/src/schema/schema.rs b/src/schema/schema.rs index 3255c8839..cf443ad82 100644 --- a/src/schema/schema.rs +++ b/src/schema/schema.rs @@ -178,15 +178,7 @@ impl Schema { SchemaBuilder::default() } - /// Returns the field options associated with a given name. - /// - /// # Panics - /// Panics if the field name does not exist. - /// It is meant as an helper for user who created - /// and control the content of their schema. - /// - /// If panicking is not an option for you, - /// you may use `get(&self, field_name: &str)`. + /// Returns the field option associated with a given name. pub fn get_field(&self, field_name: &str) -> Option { self.0.fields_map.get(field_name).cloned() } From f5c65f1f60457af1d7720dd90bad90aa4e9a31a0 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Wed, 6 Mar 2019 10:30:37 +0900 Subject: [PATCH 12/26] Added comment on the constructor fo TopDocSByField --- src/collector/top_field_collector.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/collector/top_field_collector.rs b/src/collector/top_field_collector.rs index c6c5c29a7..eb909ec30 100644 --- a/src/collector/top_field_collector.rs +++ b/src/collector/top_field_collector.rs @@ -76,6 +76,11 @@ impl TopDocsByField { /// The given field name must be a fast field, otherwise the collector have an error while /// collecting results. /// + /// This constructor is crate-private. Client are supposed to call + /// build `TopDocsByField` object using the `TopDocs` API. + /// + /// e.g.: `TopDocs::with_limit(2).order_by_field(sort_by_field)` + /// /// # Panics /// The method panics if limit is 0 pub(crate) fn new(field: Field, limit: usize) -> TopDocsByField { From e67883138d1e88ec4c67eab16e810bccfa62589f Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Wed, 6 Mar 2019 10:31:00 +0900 Subject: [PATCH 13/26] Cargo fmt --- src/collector/top_field_collector.rs | 3 ++- src/common/bitpacker.rs | 2 +- src/fastfield/reader.rs | 2 +- src/indexer/index_writer.rs | 10 ++-------- src/postings/postings_writer.rs | 2 +- 5 files changed, 7 insertions(+), 12 deletions(-) diff --git a/src/collector/top_field_collector.rs b/src/collector/top_field_collector.rs index eb909ec30..02551310c 100644 --- a/src/collector/top_field_collector.rs +++ b/src/collector/top_field_collector.rs @@ -79,7 +79,8 @@ impl TopDocsByField { /// This constructor is crate-private. Client are supposed to call /// build `TopDocsByField` object using the `TopDocs` API. /// - /// e.g.: `TopDocs::with_limit(2).order_by_field(sort_by_field)` + /// e.g.: + /// `TopDocs::with_limit(2).order_by_field(sort_by_field)` /// /// # Panics /// The method panics if limit is 0 diff --git a/src/common/bitpacker.rs b/src/common/bitpacker.rs index 9ec7241e4..22a21bcd5 100644 --- a/src/common/bitpacker.rs +++ b/src/common/bitpacker.rs @@ -80,7 +80,7 @@ where (1u64 << num_bits) - 1u64 }; BitUnpacker { - num_bits: u64::from(num_bits) , + num_bits: u64::from(num_bits), mask, data, } diff --git a/src/fastfield/reader.rs b/src/fastfield/reader.rs index f0af44dd0..c279427a9 100644 --- a/src/fastfield/reader.rs +++ b/src/fastfield/reader.rs @@ -59,7 +59,7 @@ impl FastFieldReader { /// May panic if `doc` is greater than the segment // `maxdoc`. pub fn get(&self, doc: DocId) -> Item { - self.get_u64(u64::from(doc )) + self.get_u64(u64::from(doc)) } pub(crate) fn get_u64(&self, doc: u64) -> Item { diff --git a/src/indexer/index_writer.rs b/src/indexer/index_writer.rs index 854317e38..44c9010b6 100644 --- a/src/indexer/index_writer.rs +++ b/src/indexer/index_writer.rs @@ -700,17 +700,11 @@ impl IndexWriter { for (user_op, opstamp) in user_operations.into_iter().zip(stamps) { match user_op { UserOperation::Delete(term) => { - let delete_operation = DeleteOperation { - opstamp, - term, - }; + let delete_operation = DeleteOperation { opstamp, term }; self.delete_queue.push(delete_operation); } UserOperation::Add(document) => { - let add_operation = AddOperation { - opstamp, - document - }; + let add_operation = AddOperation { opstamp, document }; adds.push(add_operation); } } diff --git a/src/postings/postings_writer.rs b/src/postings/postings_writer.rs index def366812..9701aa4de 100644 --- a/src/postings/postings_writer.rs +++ b/src/postings/postings_writer.rs @@ -52,7 +52,7 @@ pub struct MultiFieldPostingsWriter { } fn make_field_partition( - term_offsets: &[(&[u8], Addr, UnorderedTermId)] + term_offsets: &[(&[u8], Addr, UnorderedTermId)], ) -> Vec<(Field, usize, usize)> { let term_offsets_it = term_offsets .iter() From 2ccfdb97b5683be2b6912751677fa1a571dd2043 Mon Sep 17 00:00:00 2001 From: Jonathan Fok kan Date: Tue, 5 Mar 2019 20:40:54 -0500 Subject: [PATCH 14/26] WIP: compiling to wasm (#512) * First work to enable compile to wasm * Added back fst-regex/mmap to mmap feature * Removed fst-regex. Forced uuid version 0.7.2. --- Cargo.toml | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 15fcdc747..dba8651a7 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -17,8 +17,7 @@ byteorder = "1.0" lazy_static = "1" regex = "1.0" tantivy-fst = "0.1" -memmap = "0.7" -fst-regex = { version="0.2" } +memmap = {version = "0.7", optional=true} lz4 = {version="1.20", optional=true} snap = {version="0.2"} atomicwrites = {version="0.2.2", optional=true} @@ -34,7 +33,7 @@ fs2={version="0.4", optional=true} itertools = "0.8" levenshtein_automata = {version="0.1", features=["fst_automaton"]} bit-set = "0.5" -uuid = { version = "0.7", features = ["v4", "serde"] } +uuid = { version = "0.7.2", features = ["v4", "serde"] } crossbeam = "0.5" futures = "0.1" futures-cpupool = "0.1" @@ -72,12 +71,11 @@ overflow-checks = true [features] # by default no-fail is disabled. We manually enable it when running test. default = ["mmap", "no_fail"] -mmap = ["atomicwrites", "fs2"] +mmap = ["atomicwrites", "fs2", "memmap"] lz4-compression = ["lz4"] no_fail = ["fail/no_fail"] unstable = [] # useful for benches. +wasm-bindgen = ["uuid/wasm-bindgen"] [badges] travis-ci = { repository = "tantivy-search/tantivy" } - - From 94f1885334cf688cac0c106a7bfc163d1a4b26f7 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Thu, 7 Mar 2019 09:39:30 +0900 Subject: [PATCH 15/26] Issue/513 (#514) * Closes #513 * Clean up and doc * Updated changelog --- CHANGELOG.md | 3 + examples/custom_collector.rs | 4 +- examples/integer_range_search.rs | 7 ++- examples/working_with_json.rs | 2 +- src/core/index.rs | 6 +- src/functional_test.rs | 4 +- src/indexer/merger.rs | 4 +- src/lib.rs | 6 +- src/postings/mod.rs | 4 +- src/postings/segment_postings.rs | 6 +- src/query/query_parser/query_parser.rs | 6 +- src/query/range_query.rs | 10 ++-- src/schema/flags.rs | 77 ++++++++++++++++++++++++++ src/schema/int_options.rs | 74 ++++++++++++++++--------- src/schema/mod.rs | 45 +++++++-------- src/schema/schema.rs | 24 ++++++-- src/schema/text_options.rs | 42 ++++++++++---- src/space_usage/mod.rs | 7 +-- 18 files changed, 233 insertions(+), 98 deletions(-) create mode 100644 src/schema/flags.rs diff --git a/CHANGELOG.md b/CHANGELOG.md index 44032fcb9..cdfc5d2d3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,6 +9,9 @@ previous index format.* - Added grouped add and delete operations. They are guaranteed to happen together (i.e. they cannot be split by a commit). In addition, adds are guaranteed to happen on the same segment. (@elbow-jason) +- Removed `INT_STORED` and `INT_INDEXED`. It is now possible to use `STORED` and `INDEXED` + for int fields. (@fulmicoton) + Tantivy 0.8.2 ===================== diff --git a/examples/custom_collector.rs b/examples/custom_collector.rs index 0d19a83e5..b508d6e7c 100644 --- a/examples/custom_collector.rs +++ b/examples/custom_collector.rs @@ -17,7 +17,7 @@ use tantivy::collector::{Collector, SegmentCollector}; use tantivy::fastfield::FastFieldReader; use tantivy::query::QueryParser; use tantivy::schema::Field; -use tantivy::schema::{Schema, FAST, INT_INDEXED, TEXT}; +use tantivy::schema::{Schema, FAST, INDEXED, TEXT}; use tantivy::Index; use tantivy::SegmentReader; @@ -137,7 +137,7 @@ fn main() -> tantivy::Result<()> { // products, and with a name, a description, and a price. let product_name = schema_builder.add_text_field("name", TEXT); let product_description = schema_builder.add_text_field("description", TEXT); - let price = schema_builder.add_u64_field("price", INT_INDEXED | FAST); + let price = schema_builder.add_u64_field("price", INDEXED | FAST); let schema = schema_builder.build(); // # Indexing documents diff --git a/examples/integer_range_search.rs b/examples/integer_range_search.rs index 4a6d17f74..2aac51873 100644 --- a/examples/integer_range_search.rs +++ b/examples/integer_range_search.rs @@ -7,15 +7,16 @@ extern crate tantivy; use tantivy::collector::Count; use tantivy::query::RangeQuery; -use tantivy::schema::{Schema, INT_INDEXED}; +use tantivy::schema::{Schema, INDEXED}; use tantivy::Index; use tantivy::Result; fn run() -> Result<()> { // For the sake of simplicity, this schema will only have 1 field let mut schema_builder = Schema::builder(); - // INT_INDEXED is shorthand for such fields - let year_field = schema_builder.add_u64_field("year", INT_INDEXED); + + // `INDEXED` is a short-hand to indicate that our field should be "searchable". + let year_field = schema_builder.add_u64_field("year", INDEXED); let schema = schema_builder.build(); let index = Index::create_in_ram(schema); { diff --git a/examples/working_with_json.rs b/examples/working_with_json.rs index 10c89f709..75710cf2b 100644 --- a/examples/working_with_json.rs +++ b/examples/working_with_json.rs @@ -12,7 +12,7 @@ fn main() -> tantivy::Result<()> { let mut schema_builder = Schema::builder(); schema_builder.add_text_field("title", TEXT | STORED); schema_builder.add_text_field("body", TEXT); - schema_builder.add_u64_field("year", INT_INDEXED); + schema_builder.add_u64_field("year", INDEXED); let schema = schema_builder.build(); // Let's assume we have a json-serialized document. diff --git a/src/core/index.rs b/src/core/index.rs index 33165fc33..8638d7fa2 100644 --- a/src/core/index.rs +++ b/src/core/index.rs @@ -394,13 +394,13 @@ impl fmt::Debug for Index { #[cfg(test)] mod tests { use directory::RAMDirectory; - use schema::{Schema, INT_INDEXED, TEXT}; + use schema::{Schema, INDEXED, TEXT}; use Index; #[test] fn test_indexer_for_field() { let mut schema_builder = Schema::builder(); - let num_likes_field = schema_builder.add_u64_field("num_likes", INT_INDEXED); + let num_likes_field = schema_builder.add_u64_field("num_likes", INDEXED); let body_field = schema_builder.add_text_field("body", TEXT); let schema = schema_builder.build(); let index = Index::create_in_ram(schema); @@ -458,7 +458,7 @@ mod tests { fn throw_away_schema() -> Schema { let mut schema_builder = Schema::builder(); - let _ = schema_builder.add_u64_field("num_likes", INT_INDEXED); + let _ = schema_builder.add_u64_field("num_likes", INDEXED); schema_builder.build() } } diff --git a/src/functional_test.rs b/src/functional_test.rs index 7e023c29e..493f73c31 100644 --- a/src/functional_test.rs +++ b/src/functional_test.rs @@ -17,8 +17,8 @@ fn check_index_content(searcher: &Searcher, vals: &HashSet) { fn test_indexing() { let mut schema_builder = Schema::builder(); - let id_field = schema_builder.add_u64_field("id", INT_INDEXED); - let multiples_field = schema_builder.add_u64_field("multiples", INT_INDEXED); + let id_field = schema_builder.add_u64_field("id", INDEXED); + let multiples_field = schema_builder.add_u64_field("multiples", INDEXED); let schema = schema_builder.build(); let index = Index::create_from_tempdir(schema).unwrap(); diff --git a/src/indexer/merger.rs b/src/indexer/merger.rs index 311240102..01a484e17 100644 --- a/src/indexer/merger.rs +++ b/src/indexer/merger.rs @@ -654,7 +654,7 @@ mod tests { use schema::IntOptions; use schema::Term; use schema::TextFieldIndexing; - use schema::INT_INDEXED; + use schema::INDEXED; use std::io::Cursor; use DocAddress; use IndexWriter; @@ -1256,7 +1256,7 @@ mod tests { #[test] fn test_bug_merge() { let mut schema_builder = schema::Schema::builder(); - let int_field = schema_builder.add_u64_field("intvals", INT_INDEXED); + let int_field = schema_builder.add_u64_field("intvals", INDEXED); let index = Index::create_in_ram(schema_builder.build()); let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); index_writer.add_document(doc!(int_field => 1u64)); diff --git a/src/lib.rs b/src/lib.rs index ae1f45d3d..7672d15dd 100755 --- a/src/lib.rs +++ b/src/lib.rs @@ -611,7 +611,7 @@ mod tests { #[test] fn test_indexed_u64() { let mut schema_builder = Schema::builder(); - let field = schema_builder.add_u64_field("value", INT_INDEXED); + let field = schema_builder.add_u64_field("value", INDEXED); let schema = schema_builder.build(); let index = Index::create_in_ram(schema); @@ -634,7 +634,7 @@ mod tests { #[test] fn test_indexed_i64() { let mut schema_builder = Schema::builder(); - let value_field = schema_builder.add_i64_field("value", INT_INDEXED); + let value_field = schema_builder.add_i64_field("value", INDEXED); let schema = schema_builder.build(); let index = Index::create_in_ram(schema); @@ -846,7 +846,7 @@ mod tests { let fast_field_unsigned = schema_builder.add_u64_field("unsigned", FAST); let fast_field_signed = schema_builder.add_i64_field("signed", FAST); let text_field = schema_builder.add_text_field("text", TEXT); - let stored_int_field = schema_builder.add_u64_field("text", INT_STORED); + let stored_int_field = schema_builder.add_u64_field("text", STORED); let schema = schema_builder.build(); let index = Index::create_in_ram(schema); diff --git a/src/postings/mod.rs b/src/postings/mod.rs index eddb10799..e134e7a9e 100644 --- a/src/postings/mod.rs +++ b/src/postings/mod.rs @@ -57,7 +57,7 @@ pub mod tests { use rand::{Rng, SeedableRng}; use schema::Field; use schema::IndexRecordOption; - use schema::{Document, Schema, Term, INT_INDEXED, STRING, TEXT}; + use schema::{Document, Schema, Term, INDEXED, STRING, TEXT}; use std::iter; use DocId; use Score; @@ -316,7 +316,7 @@ pub mod tests { let index = { let mut schema_builder = Schema::builder(); - let value_field = schema_builder.add_u64_field("value", INT_INDEXED); + let value_field = schema_builder.add_u64_field("value", INDEXED); let schema = schema_builder.build(); let index = Index::create_in_ram(schema); diff --git a/src/postings/segment_postings.rs b/src/postings/segment_postings.rs index 90c72f5f9..af26258f4 100644 --- a/src/postings/segment_postings.rs +++ b/src/postings/segment_postings.rs @@ -627,7 +627,7 @@ mod tests { use schema::IndexRecordOption; use schema::Schema; use schema::Term; - use schema::INT_INDEXED; + use schema::INDEXED; use tantivy_fst::Streamer; use DocId; use SkipResult; @@ -760,7 +760,7 @@ mod tests { fn build_block_postings(docs: &[DocId]) -> BlockSegmentPostings { let mut schema_builder = Schema::builder(); - let int_field = schema_builder.add_u64_field("id", INT_INDEXED); + 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_with_num_threads(1, 3_000_000).unwrap(); @@ -831,7 +831,7 @@ mod tests { #[test] fn test_reset_block_segment_postings() { let mut schema_builder = Schema::builder(); - let int_field = schema_builder.add_u64_field("id", INT_INDEXED); + 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_with_num_threads(1, 3_000_000).unwrap(); diff --git a/src/query/query_parser/query_parser.rs b/src/query/query_parser/query_parser.rs index 2dcdacfda..6a359b41d 100644 --- a/src/query/query_parser/query_parser.rs +++ b/src/query/query_parser/query_parser.rs @@ -487,7 +487,7 @@ mod test { use query::Query; use schema::Field; use schema::{IndexRecordOption, TextFieldIndexing, TextOptions}; - use schema::{Schema, Term, INT_INDEXED, STORED, STRING, TEXT}; + use schema::{Schema, Term, INDEXED, STORED, STRING, TEXT}; use tokenizer::{LowerCaser, SimpleTokenizer, StopWordFilter, Tokenizer, TokenizerManager}; use Index; @@ -501,8 +501,8 @@ mod test { .set_stored(); let title = schema_builder.add_text_field("title", TEXT); let text = schema_builder.add_text_field("text", TEXT); - schema_builder.add_i64_field("signed", INT_INDEXED); - schema_builder.add_u64_field("unsigned", INT_INDEXED); + schema_builder.add_i64_field("signed", INDEXED); + schema_builder.add_u64_field("unsigned", INDEXED); schema_builder.add_text_field("notindexed_text", STORED); schema_builder.add_text_field("notindexed_u64", STORED); schema_builder.add_text_field("notindexed_i64", STORED); diff --git a/src/query/range_query.rs b/src/query/range_query.rs index bf5b37967..f111e90e1 100644 --- a/src/query/range_query.rs +++ b/src/query/range_query.rs @@ -40,14 +40,14 @@ fn map_bound TTo>( /// # #[macro_use] /// # extern crate tantivy; /// # use tantivy::Index; -/// # use tantivy::schema::{Schema, INT_INDEXED}; +/// # use tantivy::schema::{Schema, INDEXED}; /// # use tantivy::collector::Count; /// # use tantivy::Result; /// # use tantivy::query::RangeQuery; /// # /// # fn run() -> Result<()> { /// # let mut schema_builder = Schema::builder(); -/// # let year_field = schema_builder.add_u64_field("year", INT_INDEXED); +/// # let year_field = schema_builder.add_u64_field("year", INDEXED); /// # let schema = schema_builder.build(); /// # /// # let index = Index::create_in_ram(schema); @@ -293,7 +293,7 @@ mod tests { use super::RangeQuery; use collector::Count; - use schema::{Document, Field, Schema, INT_INDEXED}; + use schema::{Document, Field, Schema, INDEXED}; use std::collections::Bound; use Index; use Result; @@ -302,7 +302,7 @@ mod tests { fn test_range_query_simple() { fn run() -> Result<()> { let mut schema_builder = Schema::builder(); - let year_field = schema_builder.add_u64_field("year", INT_INDEXED); + let year_field = schema_builder.add_u64_field("year", INDEXED); let schema = schema_builder.build(); let index = Index::create_in_ram(schema); @@ -335,7 +335,7 @@ mod tests { let int_field: Field; let schema = { let mut schema_builder = Schema::builder(); - int_field = schema_builder.add_i64_field("intfield", INT_INDEXED); + int_field = schema_builder.add_i64_field("intfield", INDEXED); schema_builder.build() }; diff --git a/src/schema/flags.rs b/src/schema/flags.rs new file mode 100644 index 000000000..e766b4c27 --- /dev/null +++ b/src/schema/flags.rs @@ -0,0 +1,77 @@ +use schema::IntOptions; +use schema::TextOptions; +use std::ops::BitOr; + +#[derive(Clone)] +pub struct StoredFlag; +/// Flag to mark the field as stored. +/// This flag can apply to any kind of field. +/// +/// A stored fields of a document can be retrieved given its `DocId`. +/// Stored field are stored together and LZ4 compressed. +/// Reading the stored fields of a document is relatively slow. +/// (~ 100 microsecs) +/// +/// It should not be used during scoring or collection. +pub const STORED: SchemaFlagList = SchemaFlagList { + head: StoredFlag, + tail: (), +}; + +#[derive(Clone)] +pub struct IndexedFlag; +/// Flag to mark the field as indexed. +/// +/// The `INDEXED` flag can only be used when building `IntOptions` (`u64` and `i64` fields) +/// Of course, text fields can also be indexed... But this is expressed by using either the +/// `STRING` (untokenized) or `TEXT` (tokenized with the english tokenizer) flags. +pub const INDEXED: SchemaFlagList = SchemaFlagList { + head: IndexedFlag, + tail: (), +}; + +#[derive(Clone)] +pub struct FastFlag; +/// Flag to mark the field as a fast field (similar to Lucene's DocValues) +/// +/// Fast fields can be random-accessed rapidly. Fields useful for scoring, filtering +/// or collection should be mark as fast fields. +/// The `FAST` flag can only be used when building `IntOptions` (`u64` and `i64` fields) +pub const FAST: SchemaFlagList = SchemaFlagList { + head: FastFlag, + tail: (), +}; + +impl BitOr> for SchemaFlagList + where Head: Clone, OldHead: Clone, OldTail: Clone { + type Output = SchemaFlagList>; + + fn bitor(self, head: SchemaFlagList) -> Self::Output { + SchemaFlagList { + head: head.head, + tail: self.clone(), + } + } +} + +impl> BitOr for SchemaFlagList { + type Output = IntOptions; + + fn bitor(self, rhs: IntOptions) -> Self::Output { + self.head.into() | rhs + } +} + +impl> BitOr for SchemaFlagList { + type Output = TextOptions; + + fn bitor(self, rhs: TextOptions) -> Self::Output { + self.head.into() | rhs + } +} + +#[derive(Clone)] +pub struct SchemaFlagList { + pub head: Head, + pub tail: Tail, +} diff --git a/src/schema/int_options.rs b/src/schema/int_options.rs index b4a69cf67..b1599bd75 100644 --- a/src/schema/int_options.rs +++ b/src/schema/int_options.rs @@ -1,3 +1,4 @@ +use schema::flags::{SchemaFlagList, FastFlag, IndexedFlag, StoredFlag}; use std::ops::BitOr; /// Express whether a field is single-value or multi-valued. @@ -85,41 +86,62 @@ impl Default for IntOptions { } } -/// Shortcut for a u64 fast field. -/// -/// Such a shortcut can be composed as follows `STORED | FAST | INT_INDEXED` -pub const FAST: IntOptions = IntOptions { - indexed: false, - stored: false, - fast: Some(Cardinality::SingleValue), -}; +impl From<()> for IntOptions { + fn from(_: ()) -> IntOptions { + IntOptions::default() + } +} -/// Shortcut for a u64 indexed field. -/// -/// Such a shortcut can be composed as follows `STORED | FAST | INT_INDEXED` -pub const INT_INDEXED: IntOptions = IntOptions { - indexed: true, - stored: false, - fast: None, -}; +impl From for IntOptions { + fn from(_: FastFlag) -> Self { + IntOptions { + indexed: false, + stored: false, + fast: Some(Cardinality::SingleValue), + } + } +} -/// Shortcut for a u64 stored field. -/// -/// Such a shortcut can be composed as follows `STORED | FAST | INT_INDEXED` -pub const INT_STORED: IntOptions = IntOptions { - indexed: false, - stored: true, - fast: None, -}; +impl From for IntOptions { + fn from(_: StoredFlag) -> Self { + IntOptions { + indexed: false, + stored: true, + fast: None, + } + } +} -impl BitOr for IntOptions { +impl From for IntOptions { + fn from(_: IndexedFlag) -> Self { + IntOptions { + indexed: true, + stored: false, + fast: None, + } + } +} + +impl> BitOr for IntOptions { type Output = IntOptions; - fn bitor(self, other: IntOptions) -> IntOptions { + fn bitor(self, other: T) -> IntOptions { let mut res = IntOptions::default(); + let other = other.into(); res.indexed = self.indexed | other.indexed; res.stored = self.stored | other.stored; res.fast = self.fast.or(other.fast); res } } + +impl From> for IntOptions +where + Head: Clone, + Tail: Clone, + Self: BitOr + From + From, +{ + fn from(head_tail: SchemaFlagList) -> Self { + Self::from(head_tail.head) | Self::from(head_tail.tail) + } +} diff --git a/src/schema/mod.rs b/src/schema/mod.rs index d3ab23158..a1d90bef9 100644 --- a/src/schema/mod.rs +++ b/src/schema/mod.rs @@ -33,7 +33,7 @@ let title_options = TextOptions::default() .set_indexing_options(TextFieldIndexing::default() .set_tokenizer("default") .set_index_option(IndexRecordOption::WithFreqsAndPositions)); -schema_builder.add_text_field("title_options", title_options); +schema_builder.add_text_field("title", title_options); let schema = schema_builder.build(); ``` @@ -53,23 +53,8 @@ The effect of each possible setting is described more in detail On the other hand setting the field as stored or not determines whether the field should be returned when [`searcher.doc(doc_address)`](../struct.Searcher.html#method.doc) is called. -### Shortcuts -For convenience, a few special values of `TextOptions`. -They can be composed using the `|` operator. -The example can be rewritten : - - -``` -use tantivy::schema::*; -let mut schema_builder = Schema::builder(); -schema_builder.add_text_field("title_options", TEXT | STORED); -let schema = schema_builder.build(); -``` - - - -## Setting a u64 field +## Setting a u64 or a i64 field ### Example @@ -98,6 +83,23 @@ u64 that are indexed as fast will be stored in a special data structure that wil make it possible to access the u64 value given the doc id rapidly. This is useful if the value of the field is required during scoring or collection for instance. + +### Shortcuts + + +For convenience, it is possible to define your field indexing options by combining different flags +using the `|` operator. + +For instance, a schema containing the two fields defined in the example above could be rewritten : + +``` +use tantivy::schema::*; +let mut schema_builder = Schema::builder(); +schema_builder.add_u64_field("num_stars", INDEXED | STORED); +schema_builder.add_text_field("title", TEXT | STORED); +let schema = schema_builder.build(); +``` + */ mod document; @@ -116,13 +118,15 @@ mod named_field_document; mod text_options; mod value; +mod flags; + pub use self::named_field_document::NamedFieldDocument; pub use self::schema::DocParsingError; pub use self::schema::{Schema, SchemaBuilder}; pub use self::value::Value; pub use self::facet::Facet; -pub use self::facet::FACET_SEP_BYTE; +pub(crate) use self::facet::FACET_SEP_BYTE; pub use self::document::Document; pub use self::field::Field; @@ -135,15 +139,12 @@ pub use self::field_value::FieldValue; pub use self::index_record_option::IndexRecordOption; pub use self::text_options::TextFieldIndexing; pub use self::text_options::TextOptions; -pub use self::text_options::STORED; pub use self::text_options::STRING; pub use self::text_options::TEXT; +pub use self::flags::{FAST, INDEXED, STORED}; pub use self::int_options::Cardinality; pub use self::int_options::IntOptions; -pub use self::int_options::FAST; -pub use self::int_options::INT_INDEXED; -pub use self::int_options::INT_STORED; use regex::Regex; diff --git a/src/schema/schema.rs b/src/schema/schema.rs index cf443ad82..7c7c103e5 100644 --- a/src/schema/schema.rs +++ b/src/schema/schema.rs @@ -52,9 +52,13 @@ impl SchemaBuilder { /// by the second one. /// The first field will get a field id /// but only the second one will be indexed - pub fn add_u64_field(&mut self, field_name_str: &str, field_options: IntOptions) -> Field { + pub fn add_u64_field>( + &mut self, + field_name_str: &str, + field_options: T, + ) -> Field { let field_name = String::from(field_name_str); - let field_entry = FieldEntry::new_u64(field_name, field_options); + let field_entry = FieldEntry::new_u64(field_name, field_options.into()); self.add_field(field_entry) } @@ -68,9 +72,13 @@ impl SchemaBuilder { /// by the second one. /// The first field will get a field id /// but only the second one will be indexed - pub fn add_i64_field(&mut self, field_name_str: &str, field_options: IntOptions) -> Field { + pub fn add_i64_field>( + &mut self, + field_name_str: &str, + field_options: T, + ) -> Field { let field_name = String::from(field_name_str); - let field_entry = FieldEntry::new_i64(field_name, field_options); + let field_entry = FieldEntry::new_i64(field_name, field_options.into()); self.add_field(field_entry) } @@ -84,9 +92,13 @@ impl SchemaBuilder { /// by the second one. /// The first field will get a field id /// but only the second one will be indexed - pub fn add_text_field(&mut self, field_name_str: &str, field_options: TextOptions) -> Field { + pub fn add_text_field>( + &mut self, + field_name_str: &str, + field_options: T, + ) -> Field { let field_name = String::from(field_name_str); - let field_entry = FieldEntry::new_text(field_name, field_options); + let field_entry = FieldEntry::new_text(field_name, field_options.into()); self.add_field(field_entry) } diff --git a/src/schema/text_options.rs b/src/schema/text_options.rs index 2ae3fbb4f..78f58819a 100644 --- a/src/schema/text_options.rs +++ b/src/schema/text_options.rs @@ -1,3 +1,5 @@ +use schema::flags::SchemaFlagList; +use schema::flags::StoredFlag; use schema::IndexRecordOption; use std::borrow::Cow; use std::ops::BitOr; @@ -109,19 +111,11 @@ pub const TEXT: TextOptions = TextOptions { stored: false, }; -/// A stored fields of a document can be retrieved given its `DocId`. -/// Stored field are stored together and LZ4 compressed. -/// Reading the stored fields of a document is relatively slow. -/// (100 microsecs) -pub const STORED: TextOptions = TextOptions { - indexing: None, - stored: true, -}; - -impl BitOr for TextOptions { +impl> BitOr for TextOptions { type Output = TextOptions; - fn bitor(self, other: TextOptions) -> TextOptions { + fn bitor(self, other: T) -> TextOptions { + let other = other.into(); let mut res = TextOptions::default(); res.indexing = self.indexing.or(other.indexing); res.stored = self.stored | other.stored; @@ -129,6 +123,32 @@ impl BitOr for TextOptions { } } +impl From<()> for TextOptions { + fn from(_: ()) -> TextOptions { + TextOptions::default() + } +} + +impl From for TextOptions { + fn from(_: StoredFlag) -> TextOptions { + TextOptions { + indexing: None, + stored: true, + } + } +} + +impl From> for TextOptions +where + Head: Clone, + Tail: Clone, + Self: BitOr + From + From, +{ + fn from(head_tail: SchemaFlagList) -> Self { + Self::from(head_tail.head) | Self::from(head_tail.tail) + } +} + #[cfg(test)] mod tests { use schema::*; diff --git a/src/space_usage/mod.rs b/src/space_usage/mod.rs index 9ca77637f..f361d6860 100644 --- a/src/space_usage/mod.rs +++ b/src/space_usage/mod.rs @@ -295,8 +295,7 @@ mod test { use core::Index; use schema::Field; use schema::Schema; - use schema::STORED; - use schema::{FAST, INT_INDEXED, TEXT}; + use schema::{FAST, INDEXED, STORED, TEXT}; use space_usage::ByteCount; use space_usage::PerFieldSpaceUsage; use Term; @@ -332,7 +331,7 @@ mod test { #[test] fn test_fast_indexed() { let mut schema_builder = Schema::builder(); - let name = schema_builder.add_u64_field("name", FAST | INT_INDEXED); + let name = schema_builder.add_u64_field("name", FAST | INDEXED); let schema = schema_builder.build(); let index = Index::create_in_ram(schema.clone()); @@ -450,7 +449,7 @@ mod test { #[test] fn test_deletes() { let mut schema_builder = Schema::builder(); - let name = schema_builder.add_u64_field("name", INT_INDEXED); + let name = schema_builder.add_u64_field("name", INDEXED); let schema = schema_builder.build(); let index = Index::create_in_ram(schema.clone()); From a934577168b1aabd06e2fe5b8b66d115c051c8d7 Mon Sep 17 00:00:00 2001 From: barrotsteindev Date: Fri, 15 Mar 2019 15:10:37 +0200 Subject: [PATCH 16/26] WIP: date field (#487) * initial version, still a work in progress * remove redudant or * add chrono::DateTime and index i64 * add more tests * fix tests * pass DateTime by ptr * remove println! * document query_parser rfc 3339 date support * added some more docs about implementation to schema.rs * enforce DateTime is UTC, and re-export chrono * added DateField to changelog * fixed conflict * use INDEXED instead of INT_INDEXED for date fields --- CHANGELOG.md | 1 + Cargo.toml | 2 + src/fastfield/multivalued/mod.rs | 103 +++++++++++++++++++++++++ src/indexer/merger.rs | 13 +++- src/indexer/segment_writer.rs | 11 +++ src/lib.rs | 4 + src/postings/postings_writer.rs | 4 +- src/query/query_parser/query_parser.rs | 27 +++++++ src/schema/document.rs | 8 +- src/schema/field_entry.rs | 20 ++++- src/schema/field_type.rs | 12 ++- src/schema/schema.rs | 22 ++++++ src/schema/term.rs | 13 ++++ src/schema/value.rs | 29 +++++++ 14 files changed, 259 insertions(+), 10 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index cdfc5d2d3..3bec8fbff 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -11,6 +11,7 @@ previous index format.* In addition, adds are guaranteed to happen on the same segment. (@elbow-jason) - Removed `INT_STORED` and `INT_INDEXED`. It is now possible to use `STORED` and `INDEXED` for int fields. (@fulmicoton) +- Added DateTime field (@barrotsteindev) Tantivy 0.8.2 diff --git a/Cargo.toml b/Cargo.toml index dba8651a7..450f03e35 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -50,6 +50,7 @@ htmlescape = "0.3.1" fail = "0.2" scoped-pool = "1.0" murmurhash32 = "0.2" +chrono = "0.4" [target.'cfg(windows)'.dependencies] winapi = "0.2" @@ -58,6 +59,7 @@ winapi = "0.2" rand = "0.6" maplit = "1" matches = "0.1.8" +time = "0.1.42" [profile.release] opt-level = 3 diff --git a/src/fastfield/multivalued/mod.rs b/src/fastfield/multivalued/mod.rs index c1cd2dd7f..b8e288f52 100644 --- a/src/fastfield/multivalued/mod.rs +++ b/src/fastfield/multivalued/mod.rs @@ -7,11 +7,16 @@ pub use self::writer::MultiValueIntFastFieldWriter; #[cfg(test)] mod tests { + extern crate time; + + use query::QueryParser; + use collector::TopDocs; use schema::Cardinality; use schema::Facet; use schema::IntOptions; use schema::Schema; use Index; + use self::time::Duration; #[test] fn test_multivalued_u64() { @@ -48,6 +53,104 @@ mod tests { } } + #[test] + fn test_multivalued_date() { + let mut schema_builder = Schema::builder(); + let date_field = schema_builder.add_date_field( + "multi_date_field", + IntOptions::default() + .set_fast(Cardinality::MultiValues) + .set_indexed() + .set_stored(), + ); + let time_i = schema_builder.add_i64_field( + "time_stamp_i", + IntOptions::default() + .set_stored(), + ); + let schema = schema_builder.build(); + let index = Index::create_in_ram(schema); + let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); + let first_time_stamp = chrono::Utc::now(); + index_writer.add_document(doc!(date_field=>first_time_stamp, date_field=>first_time_stamp, time_i=>1i64)); + index_writer.add_document(doc!(time_i=>0i64)); + // add one second + index_writer.add_document(doc!(date_field=>first_time_stamp + Duration::seconds(1), time_i=>2i64)); + // add another second + let two_secs_ahead = first_time_stamp + Duration::seconds(2); + index_writer.add_document(doc!(date_field=>two_secs_ahead, date_field=>two_secs_ahead,date_field=>two_secs_ahead, time_i=>3i64)); + assert!(index_writer.commit().is_ok()); + + index.load_searchers().unwrap(); + let searcher = index.searcher(); + let reader = searcher.segment_reader(0); + assert_eq!(reader.num_docs(), 4); + + { + let parser = QueryParser::for_index(&index, vec![date_field]); + let query = parser.parse_query(&format!("\"{}\"", first_time_stamp.to_rfc3339()).to_string()) + .expect("could not parse query"); + let results = searcher.search(&query, &TopDocs::with_limit(5)) + .expect("could not query index"); + + assert_eq!(results.len(), 1); + for (_score, doc_address) in results { + let retrieved_doc = searcher.doc(doc_address).expect("cannot fetch doc"); + assert_eq!(retrieved_doc.get_first(date_field).expect("cannot find value").date_value().timestamp(), first_time_stamp.timestamp()); + assert_eq!(retrieved_doc.get_first(time_i).expect("cannot find value").i64_value(), 1i64); + } + } + + { + let parser = QueryParser::for_index(&index, vec![date_field]); + let query = parser.parse_query(&format!("\"{}\"", two_secs_ahead.to_rfc3339()).to_string()) + .expect("could not parse query"); + let results = searcher.search(&query, &TopDocs::with_limit(5)) + .expect("could not query index"); + + assert_eq!(results.len(), 1); + + for (_score, doc_address) in results { + let retrieved_doc = searcher.doc(doc_address).expect("cannot fetch doc"); + assert_eq!(retrieved_doc.get_first(date_field).expect("cannot find value").date_value().timestamp(), two_secs_ahead.timestamp()); + assert_eq!(retrieved_doc.get_first(time_i).expect("cannot find value").i64_value(), 3i64); + } + } + + + // TODO: support Date range queries +// { +// let parser = QueryParser::for_index(&index, vec![date_field]); +// let range_q = format!("\"{}\"..\"{}\"", +// (first_time_stamp + Duration::seconds(1)).to_rfc3339(), +// (first_time_stamp + Duration::seconds(3)).to_rfc3339() +// ); +// let query = parser.parse_query(&range_q) +// .expect("could not parse query"); +// let results = searcher.search(&query, &TopDocs::with_limit(5)) +// .expect("could not query index"); +// +// +// assert_eq!(results.len(), 2); +// for (i, doc_pair) in results.iter().enumerate() { +// let retrieved_doc = searcher.doc(doc_pair.1).expect("cannot fetch doc"); +// let offset_sec = match i { +// 0 => 1, +// 1 => 3, +// _ => panic!("should not have more than 2 docs") +// }; +// let time_i_val = match i { +// 0 => 2, +// 1 => 3, +// _ => panic!("should not have more than 2 docs") +// }; +// assert_eq!(retrieved_doc.get_first(date_field).expect("cannot find value").date_value().timestamp(), +// (first_time_stamp + Duration::seconds(offset_sec)).timestamp()); +// assert_eq!(retrieved_doc.get_first(time_i).expect("cannot find value").i64_value(), time_i_val); +// } +// } + } + #[test] fn test_multivalued_i64() { let mut schema_builder = Schema::builder(); diff --git a/src/indexer/merger.rs b/src/indexer/merger.rs index 01a484e17..37c07b7a6 100644 --- a/src/indexer/merger.rs +++ b/src/indexer/merger.rs @@ -194,7 +194,7 @@ impl IndexMerger { fast_field_serializer, )?; } - FieldType::U64(ref options) | FieldType::I64(ref options) => { + FieldType::U64(ref options) | FieldType::I64(ref options) | FieldType::Date(ref options) => { match options.get_fastfield_cardinality() { Some(Cardinality::SingleValue) => { self.write_single_fast_field(field, fast_field_serializer)?; @@ -671,10 +671,12 @@ mod tests { ) .set_stored(); let text_field = schema_builder.add_text_field("text", text_fieldtype); + let date_field = schema_builder.add_date_field("date", INDEXED); let score_fieldtype = schema::IntOptions::default().set_fast(Cardinality::SingleValue); let score_field = schema_builder.add_u64_field("score", score_fieldtype); let bytes_score_field = schema_builder.add_bytes_field("score_bytes"); let index = Index::create_in_ram(schema_builder.build()); + let curr_time = chrono::Utc::now(); let add_score_bytes = |doc: &mut Document, score: u32| { let mut bytes = Vec::new(); @@ -692,6 +694,7 @@ mod tests { let mut doc = Document::default(); doc.add_text(text_field, "af b"); doc.add_u64(score_field, 3); + doc.add_date(date_field, &curr_time); add_score_bytes(&mut doc, 3); index_writer.add_document(doc); } @@ -717,6 +720,7 @@ mod tests { { let mut doc = Document::default(); doc.add_text(text_field, "af b"); + doc.add_date(date_field, &curr_time); doc.add_u64(score_field, 11); add_score_bytes(&mut doc, 11); index_writer.add_document(doc); @@ -774,6 +778,13 @@ mod tests { DocAddress(0, 4) ] ); + assert_eq!( + get_doc_ids(vec![Term::from_field_date(date_field, &curr_time)]), + vec![ + DocAddress(0, 0), + DocAddress(0, 3) + ] + ); } { let doc = searcher.doc(DocAddress(0, 0)).unwrap(); diff --git a/src/indexer/segment_writer.rs b/src/indexer/segment_writer.rs index 9d76db41a..4eb42c843 100644 --- a/src/indexer/segment_writer.rs +++ b/src/indexer/segment_writer.rs @@ -171,6 +171,17 @@ impl SegmentWriter { } } } + FieldType::Date(ref int_option) => { + if int_option.is_indexed() { + for field_value in field_values { + let term = Term::from_field_i64( + field_value.field(), + field_value.value().date_value().timestamp(), + ); + self.multifield_postings.subscribe(doc_id, &term); + } + } + } FieldType::I64(ref int_option) => { if int_option.is_indexed() { for field_value in field_values { diff --git a/src/lib.rs b/src/lib.rs index 7672d15dd..6beb6f32d 100755 --- a/src/lib.rs +++ b/src/lib.rs @@ -187,10 +187,14 @@ pub use error::TantivyError as Error; extern crate census; extern crate owned_read; +pub extern crate chrono; /// Tantivy result. pub type Result = std::result::Result; +/// Tantivy DateTime +pub type DateTime = chrono::DateTime; + mod common; mod core; mod indexer; diff --git a/src/postings/postings_writer.rs b/src/postings/postings_writer.rs index 9701aa4de..97de4cd10 100644 --- a/src/postings/postings_writer.rs +++ b/src/postings/postings_writer.rs @@ -33,7 +33,7 @@ fn posting_from_field_entry(field_entry: &FieldEntry) -> Box { } }) .unwrap_or_else(|| SpecializedPostingsWriter::::new_boxed()), - FieldType::U64(_) | FieldType::I64(_) | FieldType::HierarchicalFacet => { + FieldType::U64(_) | FieldType::I64(_) | FieldType::Date(_) | FieldType::HierarchicalFacet => { SpecializedPostingsWriter::::new_boxed() } FieldType::Bytes => { @@ -148,7 +148,7 @@ impl MultiFieldPostingsWriter { .collect(); unordered_term_mappings.insert(field, mapping); } - FieldType::U64(_) | FieldType::I64(_) => {} + FieldType::U64(_) | FieldType::I64(_) | FieldType::Date(_) => {} FieldType::Bytes => {} } diff --git a/src/query/query_parser/query_parser.rs b/src/query/query_parser/query_parser.rs index 6a359b41d..25b08ae1f 100644 --- a/src/query/query_parser/query_parser.rs +++ b/src/query/query_parser/query_parser.rs @@ -50,6 +50,8 @@ pub enum QueryParserError { /// The query contains a range query with a phrase as one of the bounds. /// Only terms can be used as bounds. RangeMustNotHavePhrase, + /// The format for the date field is not RFC 3339 compliant. + DateFormatError(chrono::ParseError), } impl From for QueryParserError { @@ -58,6 +60,12 @@ impl From for QueryParserError { } } +impl From for QueryParserError { + fn from(err: chrono::ParseError) -> QueryParserError { + QueryParserError::DateFormatError(err) + } +} + /// Recursively remove empty clause from the AST /// /// Returns `None` iff the `logical_ast` ended up being empty. @@ -127,6 +135,8 @@ fn trim_ast(logical_ast: LogicalAST) -> Option { /// a word lexicographically between `a` and `c` (inclusive lower bound, exclusive upper bound). /// Inclusive bounds are `[]`, exclusive are `{}`. /// +/// * date values: The query parser supports rfc3339 formatted dates. For example "2002-10-02T15:00:00.05Z" +/// /// * all docs query: A plain `*` will match all documents in the index. /// #[derive(Clone)] @@ -229,6 +239,12 @@ impl QueryParser { let term = Term::from_field_i64(field, val); Ok(vec![(0, term)]) } + FieldType::Date(_) => { + match chrono::DateTime::parse_from_rfc3339(phrase) { + Ok(x) => Ok(vec![(0, Term::from_field_date(field, &x.with_timezone(&chrono::Utc)))]), + Err(e) => Err(QueryParserError::DateFormatError(e)) + } + } FieldType::U64(_) => { let val: u64 = u64::from_str(phrase)?; let term = Term::from_field_u64(field, val); @@ -508,6 +524,7 @@ mod test { schema_builder.add_text_field("notindexed_i64", STORED); schema_builder.add_text_field("nottokenized", STRING); schema_builder.add_text_field("with_stop_words", text_options); + schema_builder.add_date_field("date", INDEXED); let schema = schema_builder.build(); let default_fields = vec![title, text]; let tokenizer_manager = TokenizerManager::default(); @@ -767,6 +784,16 @@ mod test { ); } + #[test] + pub fn test_query_parser_expected_date() { + let query_parser = make_query_parser(); + assert_matches!( + query_parser.parse_query("date:18a"), + Err(QueryParserError::DateFormatError(_)) + ); + assert!(query_parser.parse_query("date:\"1985-04-12T23:20:50.52Z\"").is_ok()); + } + #[test] pub fn test_query_parser_not_empty_but_no_tokens() { let query_parser = make_query_parser(); diff --git a/src/schema/document.rs b/src/schema/document.rs index 9e04bb107..2c535446a 100644 --- a/src/schema/document.rs +++ b/src/schema/document.rs @@ -3,6 +3,7 @@ use common::BinarySerializable; use common::VInt; use itertools::Itertools; use std::io::{self, Read, Write}; +use DateTime; /// Tantivy's Document is the object that can /// be indexed and then searched for. @@ -82,11 +83,16 @@ impl Document { self.add(FieldValue::new(field, Value::U64(value))); } - /// Add a u64 field + /// Add a i64 field pub fn add_i64(&mut self, field: Field, value: i64) { self.add(FieldValue::new(field, Value::I64(value))); } + /// Add a date field + pub fn add_date(&mut self, field: Field, value: &DateTime) { + self.add(FieldValue::new(field, Value::Date(DateTime::from(*value)))); + } + /// Add a bytes field pub fn add_bytes(&mut self, field: Field, value: Vec) { self.add(FieldValue::new(field, Value::Bytes(value))) diff --git a/src/schema/field_entry.rs b/src/schema/field_entry.rs index e63fa99ab..fc5972b94 100644 --- a/src/schema/field_entry.rs +++ b/src/schema/field_entry.rs @@ -48,6 +48,15 @@ impl FieldEntry { } } + /// Creates a new date field entry in the schema, given + /// a name, and some options. + pub fn new_date(field_name: String, field_type: IntOptions) -> FieldEntry { + FieldEntry { + name: field_name, + field_type: FieldType::Date(field_type), + } + } + /// Creates a field entry for a facet. pub fn new_facet(field_name: String) -> FieldEntry { FieldEntry { @@ -78,7 +87,7 @@ impl FieldEntry { pub fn is_indexed(&self) -> bool { match self.field_type { FieldType::Str(ref options) => options.get_indexing_options().is_some(), - FieldType::U64(ref options) | FieldType::I64(ref options) => options.is_indexed(), + FieldType::U64(ref options) | FieldType::I64(ref options) | FieldType::Date(ref options) => options.is_indexed(), FieldType::HierarchicalFacet => true, FieldType::Bytes => false, } @@ -95,7 +104,7 @@ impl FieldEntry { /// Returns true iff the field is stored pub fn is_stored(&self) -> bool { match self.field_type { - FieldType::U64(ref options) | FieldType::I64(ref options) => options.is_stored(), + FieldType::U64(ref options) | FieldType::I64(ref options) | FieldType::Date(ref options) => options.is_stored(), FieldType::Str(ref options) => options.is_stored(), // TODO make stored hierarchical facet optional FieldType::HierarchicalFacet => true, @@ -125,6 +134,10 @@ impl Serialize for FieldEntry { s.serialize_field("type", "i64")?; s.serialize_field("options", options)?; } + FieldType::Date(ref options) => { + s.serialize_field("type", "date")?; + s.serialize_field("options", options)?; + } FieldType::HierarchicalFacet => { s.serialize_field("type", "hierarchical_facet")?; } @@ -188,7 +201,7 @@ impl<'de> Deserialize<'de> for FieldEntry { "bytes" => { field_type = Some(FieldType::Bytes); } - "text" | "u64" | "i64" => { + "text" | "u64" | "i64" | "date" => { // These types require additional options to create a field_type } _ => panic!("unhandled type"), @@ -205,6 +218,7 @@ impl<'de> Deserialize<'de> for FieldEntry { "text" => field_type = Some(FieldType::Str(map.next_value()?)), "u64" => field_type = Some(FieldType::U64(map.next_value()?)), "i64" => field_type = Some(FieldType::I64(map.next_value()?)), + "date" => field_type = Some(FieldType::Date(map.next_value()?)), _ => { let msg = format!("Unrecognised type {}", ty); return Err(de::Error::custom(msg)); diff --git a/src/schema/field_type.rs b/src/schema/field_type.rs index 092f7bea0..79c2656a6 100644 --- a/src/schema/field_type.rs +++ b/src/schema/field_type.rs @@ -34,6 +34,8 @@ pub enum Type { U64, /// `i64` I64, + /// `date(i64) timestamp` + Date, /// `tantivy::schema::Facet`. Passed as a string in JSON. HierarchicalFacet, /// `Vec` @@ -50,6 +52,8 @@ pub enum FieldType { U64(IntOptions), /// Signed 64-bits integers 64 field type configuration I64(IntOptions), + /// Signed 64-bits Date 64 field type configuration, + Date(IntOptions), /// Hierachical Facet HierarchicalFacet, /// Bytes (one per document) @@ -63,6 +67,7 @@ impl FieldType { FieldType::Str(_) => Type::Str, FieldType::U64(_) => Type::U64, FieldType::I64(_) => Type::I64, + FieldType::Date(_) => Type::Date, FieldType::HierarchicalFacet => Type::HierarchicalFacet, FieldType::Bytes => Type::Bytes, } @@ -75,6 +80,7 @@ impl FieldType { FieldType::U64(ref int_options) | FieldType::I64(ref int_options) => { int_options.is_indexed() } + FieldType::Date(ref date_options) => date_options.is_indexed(), FieldType::HierarchicalFacet => true, FieldType::Bytes => false, } @@ -89,7 +95,7 @@ impl FieldType { FieldType::Str(ref text_options) => text_options .get_indexing_options() .map(|indexing_options| indexing_options.index_option()), - FieldType::U64(ref int_options) | FieldType::I64(ref int_options) => { + FieldType::U64(ref int_options) | FieldType::I64(ref int_options) | FieldType::Date(ref int_options) => { if int_options.is_indexed() { Some(IndexRecordOption::Basic) } else { @@ -110,7 +116,7 @@ impl FieldType { match *json { JsonValue::String(ref field_text) => match *self { FieldType::Str(_) => Ok(Value::Str(field_text.clone())), - FieldType::U64(_) | FieldType::I64(_) => Err(ValueParsingError::TypeError( + FieldType::U64(_) | FieldType::I64(_) | FieldType::Date(_) => Err(ValueParsingError::TypeError( format!("Expected an integer, got {:?}", json), )), FieldType::HierarchicalFacet => Ok(Value::Facet(Facet::from(field_text))), @@ -122,7 +128,7 @@ impl FieldType { }), }, JsonValue::Number(ref field_val_num) => match *self { - FieldType::I64(_) => { + FieldType::I64(_) | FieldType::Date(_) => { if let Some(field_val_i64) = field_val_num.as_i64() { Ok(Value::I64(field_val_i64)) } else { diff --git a/src/schema/schema.rs b/src/schema/schema.rs index 7c7c103e5..994b71c9a 100644 --- a/src/schema/schema.rs +++ b/src/schema/schema.rs @@ -82,6 +82,28 @@ impl SchemaBuilder { self.add_field(field_entry) } + /// Adds a new date field. + /// Returns the associated field handle + /// Internally, Tantivy simply stores dates as i64 UTC timestamps, + /// while the user supplies DateTime values for convenience. + /// + /// # Caution + /// + /// Appending two fields with the same name + /// will result in the shadowing of the first + /// by the second one. + /// The first field will get a field id + /// but only the second one will be indexed + pub fn add_date_field>( + &mut self, + field_name_str: &str, + field_options: T + ) -> Field { + let field_name = String::from(field_name_str); + let field_entry = FieldEntry::new_date(field_name, field_options.into()); + self.add_field(field_entry) + } + /// Adds a new text field. /// Returns the associated field handle /// diff --git a/src/schema/term.rs b/src/schema/term.rs index 393e2fac4..a7b450fa6 100644 --- a/src/schema/term.rs +++ b/src/schema/term.rs @@ -5,6 +5,7 @@ use byteorder::{BigEndian, ByteOrder}; use common; use schema::Facet; use std::str; +use DateTime; /// Size (in bytes) of the buffer of a int field. const INT_TERM_LEN: usize = 4 + 8; @@ -30,6 +31,18 @@ impl Term { Term::from_field_u64(field, val_u64) } + /// Builds a term given a field, and a DateTime value + /// + /// Assuming the term has a field id of 1, and a timestamp i64 value of 3234, + /// the Term will have 8 bytes. + /// + /// The first four byte are dedicated to storing the field id as a u64. + /// The 4 following bytes are encoding the DateTime as i64 timestamp value. + pub fn from_field_date(field: Field, val: &DateTime) -> Term { + let val_timestamp = val.timestamp(); + Term::from_field_i64(field, val_timestamp) + } + /// Creates a `Term` given a facet. pub fn from_facet(field: Field, facet: &Facet) -> Term { let bytes = facet.encoded_str().as_bytes(); diff --git a/src/schema/value.rs b/src/schema/value.rs index 64b0dc795..0090b4435 100644 --- a/src/schema/value.rs +++ b/src/schema/value.rs @@ -2,6 +2,7 @@ use schema::Facet; use serde::de::Visitor; use serde::{Deserialize, Deserializer, Serialize, Serializer}; use std::fmt; +use DateTime; /// Value represents the value of a any field. /// It is an enum over all over all of the possible field type. @@ -13,6 +14,8 @@ pub enum Value { U64(u64), /// Signed 64-bits Integer `i64` I64(i64), + /// Signed 64-bits Date time stamp `date` + Date(DateTime), /// Hierarchical Facet Facet(Facet), /// Arbitrarily sized byte array @@ -28,6 +31,7 @@ impl Serialize for Value { Value::Str(ref v) => serializer.serialize_str(v), Value::U64(u) => serializer.serialize_u64(u), Value::I64(u) => serializer.serialize_i64(u), + Value::Date(ref date) => serializer.serialize_i64(date.timestamp()), Value::Facet(ref facet) => facet.serialize(serializer), Value::Bytes(ref bytes) => serializer.serialize_bytes(bytes), } @@ -102,6 +106,17 @@ impl Value { _ => panic!("This is not a text field."), } } + + /// Returns the Date-value, provided the value is of the `Date` type. + /// + /// # Panics + /// If the value is not of type `Date` + pub fn date_value(&self) -> &DateTime { + match *self { + Value::Date(ref value) => value, + _ => panic!("This is not a date field."), + } + } } impl From for Value { @@ -122,6 +137,10 @@ impl From for Value { } } +impl From for Value { + fn from(date_time: DateTime) -> Value { Value::Date(date_time) } +} + impl<'a> From<&'a str> for Value { fn from(s: &'a str) -> Value { Value::Str(s.to_string()) @@ -145,12 +164,14 @@ mod binary_serialize { use common::BinarySerializable; use schema::Facet; use std::io::{self, Read, Write}; + use chrono::{Utc, TimeZone}; const TEXT_CODE: u8 = 0; const U64_CODE: u8 = 1; const I64_CODE: u8 = 2; const HIERARCHICAL_FACET_CODE: u8 = 3; const BYTES_CODE: u8 = 4; + const DATE_CODE: u8 = 5; impl BinarySerializable for Value { fn serialize(&self, writer: &mut W) -> io::Result<()> { @@ -167,6 +188,10 @@ mod binary_serialize { I64_CODE.serialize(writer)?; val.serialize(writer) } + Value::Date(ref val) => { + DATE_CODE.serialize(writer)?; + val.timestamp().serialize(writer) + } Value::Facet(ref facet) => { HIERARCHICAL_FACET_CODE.serialize(writer)?; facet.serialize(writer) @@ -192,6 +217,10 @@ mod binary_serialize { let value = i64::deserialize(reader)?; Ok(Value::I64(value)) } + DATE_CODE=> { + let timestamp = i64::deserialize(reader)?; + Ok(Value::Date(Utc.timestamp(timestamp, 0))) + } HIERARCHICAL_FACET_CODE => Ok(Value::Facet(Facet::deserialize(reader)?)), BYTES_CODE => Ok(Value::Bytes(Vec::::deserialize(reader)?)), _ => Err(io::Error::new( From 663dd89c051c30546903fcc5e7e6d47b2e91c17a Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Wed, 20 Mar 2019 08:39:22 +0900 Subject: [PATCH 17/26] Feature/reader (#517) Adding IndexReader to the API. Making it possible to watch for changes. * Closes #500 --- CHANGELOG.md | 3 + Cargo.toml | 3 +- examples/basic_search.rs | 33 ++- examples/custom_collector.rs | 4 +- examples/custom_tokenizer.rs | 4 +- examples/deleting_updating_documents.rs | 16 +- examples/faceted_search.rs | 4 +- examples/integer_range_search.rs | 5 +- examples/iterating_docs_and_positions.rs | 4 +- examples/snippet.rs | 5 +- examples/stop_words.rs | 4 +- src/collector/count_collector.rs | 4 +- src/collector/facet_collector.rs | 28 +- src/collector/int_facet_collector.rs | 3 +- src/collector/mod.rs | 6 +- src/collector/multi_collector.rs | 7 +- src/collector/top_field_collector.rs | 29 +- src/collector/top_score_collector.rs | 9 +- src/core/index.rs | 187 ++++++++----- src/core/mod.rs | 2 +- src/core/segment_reader.rs | 4 +- src/directory/directory.rs | 18 ++ src/directory/directory_lock.rs | 2 +- src/directory/error.rs | 13 + src/directory/managed_directory.rs | 5 + src/directory/mmap_directory.rs | 320 ++++++++++++++++++----- src/directory/mod.rs | 3 + src/directory/ram_directory.rs | 141 ++++------ src/directory/tests.rs | 58 +++- src/directory/watch_event_router.rs | 156 +++++++++++ src/error.rs | 1 + src/fastfield/bytes/mod.rs | 4 +- src/fastfield/multivalued/mod.rs | 141 ++++++---- src/fastfield/multivalued/reader.rs | 3 +- src/functional_test.rs | 5 +- src/indexer/index_writer.rs | 94 ++++--- src/indexer/merger.rs | 105 ++++---- src/indexer/segment_updater.rs | 21 +- src/lib.rs | 144 +++++----- src/postings/mod.rs | 18 +- src/postings/postings_writer.rs | 7 +- src/postings/segment_postings.rs | 6 +- src/query/all_query.rs | 5 +- src/query/boolean_query/mod.rs | 27 +- src/query/fuzzy_query.rs | 9 +- src/query/phrase_query/mod.rs | 16 +- src/query/query_parser/query_parser.rs | 17 +- src/query/range_query.rs | 12 +- src/query/regex_query.rs | 8 +- src/query/term_query/mod.rs | 7 +- src/query/term_query/term_query.rs | 5 +- src/reader/mod.rs | 187 +++++++++++++ src/{core => reader}/pool.rs | 0 src/schema/document.rs | 2 +- src/schema/field_entry.rs | 8 +- src/schema/field_type.rs | 10 +- src/schema/flags.rs | 10 +- src/schema/int_options.rs | 2 +- src/schema/schema.rs | 2 +- src/schema/value.rs | 16 +- src/snippet/mod.rs | 10 +- src/space_usage/mod.rs | 23 +- src/termdict/mod.rs | 3 +- 63 files changed, 1354 insertions(+), 654 deletions(-) create mode 100644 src/directory/watch_event_router.rs create mode 100644 src/reader/mod.rs rename src/{core => reader}/pool.rs (100%) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3bec8fbff..d692de690 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,8 @@ Tantivy 0.9.0 ===================== *0.9.0 index format is not compatible with the previous index format.* +- MAJOR BUGFIX : + Some `Mmap` objects were being leaked, and would never get released. (@fulmicoton) - Removed most unsafe (@fulmicoton) - Indexer memory footprint improved. (VInt comp, inlining the first block. (@fulmicoton) - Stemming in other language possible (@pentlander) @@ -12,6 +14,7 @@ previous index format.* - Removed `INT_STORED` and `INT_INDEXED`. It is now possible to use `STORED` and `INDEXED` for int fields. (@fulmicoton) - Added DateTime field (@barrotsteindev) +- Added IndexReader. By default, index is reloaded automatically upon new commits (@fulmicoton) Tantivy 0.8.2 diff --git a/Cargo.toml b/Cargo.toml index 450f03e35..1e04cf4fa 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -32,6 +32,7 @@ num_cpus = "1.2" fs2={version="0.4", optional=true} itertools = "0.8" levenshtein_automata = {version="0.1", features=["fst_automaton"]} +notify = {version="4", optional=true} bit-set = "0.5" uuid = { version = "0.7.2", features = ["v4", "serde"] } crossbeam = "0.5" @@ -73,7 +74,7 @@ overflow-checks = true [features] # by default no-fail is disabled. We manually enable it when running test. default = ["mmap", "no_fail"] -mmap = ["atomicwrites", "fs2", "memmap"] +mmap = ["atomicwrites", "fs2", "memmap", "notify"] lz4-compression = ["lz4"] no_fail = ["fail/no_fail"] unstable = [] # useful for benches. diff --git a/examples/basic_search.rs b/examples/basic_search.rs index 3a0a71e2f..78c2c2d3b 100644 --- a/examples/basic_search.rs +++ b/examples/basic_search.rs @@ -20,6 +20,7 @@ use tantivy::collector::TopDocs; use tantivy::query::QueryParser; use tantivy::schema::*; use tantivy::Index; +use tantivy::ReloadPolicy; use tempdir::TempDir; fn main() -> tantivy::Result<()> { @@ -170,24 +171,33 @@ fn main() -> tantivy::Result<()> { // // ### Searcher // - // Let's search our index. Start by reloading - // searchers in the index. This should be done - // after every `commit()`. - index.load_searchers()?; + // A reader is required to get search the index. + // It acts as a `Searcher` pool that reloads itself, + // depending on a `ReloadPolicy`. + // + // For a search server you will typically create one reader for the entire lifetime of your + // program, and acquire a new searcher for every single request. + // + // In the code below, we rely on the 'ON_COMMIT' policy: the reader + // will reload the index automatically after each commit. + let reader = index + .reader_builder() + .reload_policy(ReloadPolicy::OnCommit) + .try_into()?; // We now need to acquire a searcher. - // Some search experience might require more than - // one query. // - // The searcher ensure that we get to work - // with a consistent version of the index. + // A searcher points to snapshotted, immutable version of the index. + // + // Some search experience might require more than + // one query. Using the same searcher ensures that all of these queries will run on the + // same version of the index. // // Acquiring a `searcher` is very cheap. // - // You should acquire a searcher every time you - // start processing a request and + // You should acquire a searcher every time you start processing a request and // and release it right after your query is finished. - let searcher = index.searcher(); + let searcher = reader.searcher(); // ### Query @@ -224,7 +234,6 @@ fn main() -> tantivy::Result<()> { // Since the body field was not configured as stored, // the document returned will only contain // a title. - for (_score, doc_address) in top_docs { let retrieved_doc = searcher.doc(doc_address)?; println!("{}", schema.to_json(&retrieved_doc)); diff --git a/examples/custom_collector.rs b/examples/custom_collector.rs index b508d6e7c..40a1d1a0d 100644 --- a/examples/custom_collector.rs +++ b/examples/custom_collector.rs @@ -170,9 +170,9 @@ fn main() -> tantivy::Result<()> { price => 5_200u64 )); index_writer.commit()?; - index.load_searchers()?; - let searcher = index.searcher(); + let reader = index.reader()?; + let searcher = reader.searcher(); let query_parser = QueryParser::for_index(&index, vec![product_name, product_description]); // here we want to get a hit on the 'ken' in Frankenstein diff --git a/examples/custom_tokenizer.rs b/examples/custom_tokenizer.rs index 72b69184d..5730adb10 100644 --- a/examples/custom_tokenizer.rs +++ b/examples/custom_tokenizer.rs @@ -91,9 +91,9 @@ fn main() -> tantivy::Result<()> { increasing confidence in the success of my undertaking."# )); index_writer.commit()?; - index.load_searchers()?; - let searcher = index.searcher(); + let reader = index.reader()?; + let searcher = reader.searcher(); // The query parser can interpret human queries. // Here, if the user does not specify which diff --git a/examples/deleting_updating_documents.rs b/examples/deleting_updating_documents.rs index ed59fa8c8..82fdd90b2 100644 --- a/examples/deleting_updating_documents.rs +++ b/examples/deleting_updating_documents.rs @@ -14,12 +14,16 @@ use tantivy::collector::TopDocs; use tantivy::query::TermQuery; use tantivy::schema::*; use tantivy::Index; +use tantivy::IndexReader; // A simple helper function to fetch a single document // given its id from our index. // It will be helpful to check our work. -fn extract_doc_given_isbn(index: &Index, isbn_term: &Term) -> tantivy::Result> { - let searcher = index.searcher(); +fn extract_doc_given_isbn( + reader: &IndexReader, + isbn_term: &Term, +) -> tantivy::Result> { + let searcher = reader.searcher(); // This is the simplest query you can think of. // It matches all of the documents containing a specific term. @@ -85,12 +89,12 @@ fn main() -> tantivy::Result<()> { isbn => "978-9176370711", )); index_writer.commit()?; - index.load_searchers()?; + let reader = index.reader()?; let frankenstein_isbn = Term::from_field_text(isbn, "978-9176370711"); // Oops our frankenstein doc seems mispelled - let frankenstein_doc_misspelled = extract_doc_given_isbn(&index, &frankenstein_isbn)?.unwrap(); + let frankenstein_doc_misspelled = extract_doc_given_isbn(&reader, &frankenstein_isbn)?.unwrap(); assert_eq!( schema.to_json(&frankenstein_doc_misspelled), r#"{"isbn":["978-9176370711"],"title":["Frankentein"]}"#, @@ -129,10 +133,10 @@ fn main() -> tantivy::Result<()> { // Everything happened as if the document was updated. index_writer.commit()?; // We reload our searcher to make our change available to clients. - index.load_searchers()?; + reader.reload()?; // No more typo! - let frankenstein_new_doc = extract_doc_given_isbn(&index, &frankenstein_isbn)?.unwrap(); + let frankenstein_new_doc = extract_doc_given_isbn(&reader, &frankenstein_isbn)?.unwrap(); assert_eq!( schema.to_json(&frankenstein_new_doc), r#"{"isbn":["978-9176370711"],"title":["Frankenstein"]}"#, diff --git a/examples/faceted_search.rs b/examples/faceted_search.rs index 9d68f2a4e..0a99c7131 100644 --- a/examples/faceted_search.rs +++ b/examples/faceted_search.rs @@ -55,9 +55,9 @@ fn main() -> tantivy::Result<()> { index_writer.commit()?; - index.load_searchers()?; + let reader = index.reader()?; - let searcher = index.searcher(); + let searcher = reader.searcher(); let mut facet_collector = FacetCollector::for_field(tags); facet_collector.add_facet("/pools"); diff --git a/examples/integer_range_search.rs b/examples/integer_range_search.rs index 2aac51873..dea3145b6 100644 --- a/examples/integer_range_search.rs +++ b/examples/integer_range_search.rs @@ -19,6 +19,7 @@ fn run() -> Result<()> { let year_field = schema_builder.add_u64_field("year", INDEXED); let schema = schema_builder.build(); let index = Index::create_in_ram(schema); + let reader = index.reader()?; { let mut index_writer = index.writer_with_num_threads(1, 6_000_000)?; for year in 1950u64..2019u64 { @@ -27,8 +28,8 @@ fn run() -> Result<()> { index_writer.commit()?; // The index will be a range of years } - index.load_searchers()?; - let searcher = index.searcher(); + reader.reload()?; + let searcher = reader.searcher(); // The end is excluded i.e. here we are searching up to 1969 let docs_in_the_sixties = RangeQuery::new_u64(year_field, 1960..1970); // Uses a Count collector to sum the total number of docs in the range diff --git a/examples/iterating_docs_and_positions.rs b/examples/iterating_docs_and_positions.rs index 62513ea7a..4668de3c0 100644 --- a/examples/iterating_docs_and_positions.rs +++ b/examples/iterating_docs_and_positions.rs @@ -33,9 +33,9 @@ fn main() -> tantivy::Result<()> { index_writer.add_document(doc!(title => "The modern Promotheus")); index_writer.commit()?; - index.load_searchers()?; + let reader = index.reader()?; - let searcher = index.searcher(); + let searcher = reader.searcher(); // A tantivy index is actually a collection of segments. // Similarly, a searcher just wraps a list `segment_reader`. diff --git a/examples/snippet.rs b/examples/snippet.rs index 35ba07557..b79ede83b 100644 --- a/examples/snippet.rs +++ b/examples/snippet.rs @@ -48,9 +48,8 @@ fn main() -> tantivy::Result<()> { // ... index_writer.commit()?; - index.load_searchers()?; - - let searcher = index.searcher(); + let reader = index.reader()?; + let searcher = reader.searcher(); let query_parser = QueryParser::for_index(&index, vec![title, body]); let query = query_parser.parse_query("sycamore spring")?; diff --git a/examples/stop_words.rs b/examples/stop_words.rs index cdfe054e8..a6b338060 100644 --- a/examples/stop_words.rs +++ b/examples/stop_words.rs @@ -96,9 +96,9 @@ fn main() -> tantivy::Result<()> { index_writer.commit()?; - index.load_searchers()?; + let reader = index.reader()?; - let searcher = index.searcher(); + let searcher = reader.searcher(); let query_parser = QueryParser::for_index(&index, vec![title, body]); diff --git a/src/collector/count_collector.rs b/src/collector/count_collector.rs index ea2a1d9cd..85ceaa3ab 100644 --- a/src/collector/count_collector.rs +++ b/src/collector/count_collector.rs @@ -40,8 +40,8 @@ use SegmentReader; /// index_writer.commit().unwrap(); /// } /// -/// index.load_searchers()?; -/// let searcher = index.searcher(); +/// let reader = index.reader()?; +/// let searcher = reader.searcher(); /// /// { /// let query_parser = QueryParser::for_index(&index, vec![title]); diff --git a/src/collector/facet_collector.rs b/src/collector/facet_collector.rs index 083bd65ae..16ce9428b 100644 --- a/src/collector/facet_collector.rs +++ b/src/collector/facet_collector.rs @@ -122,17 +122,16 @@ fn facet_depth(facet_bytes: &[u8]) -> usize { /// facet => Facet::from("/lang/en"), /// facet => Facet::from("/category/biography") /// )); -/// index_writer.commit().unwrap(); +/// index_writer.commit()?; /// } -/// -/// index.load_searchers()?; -/// let searcher = index.searcher(); +/// let reader = index.reader()?; +/// let searcher = reader.searcher(); /// /// { /// let mut facet_collector = FacetCollector::for_field(facet); /// facet_collector.add_facet("/lang"); /// facet_collector.add_facet("/category"); -/// let facet_counts = searcher.search(&AllQuery, &facet_collector).unwrap(); +/// let facet_counts = searcher.search(&AllQuery, &facet_collector)?; /// /// // This lists all of the facet counts /// let facets: Vec<(&Facet, u64)> = facet_counts @@ -147,7 +146,7 @@ fn facet_depth(facet_bytes: &[u8]) -> usize { /// { /// let mut facet_collector = FacetCollector::for_field(facet); /// facet_collector.add_facet("/category/fiction"); -/// let facet_counts = searcher.search(&AllQuery, &facet_collector).unwrap(); +/// let facet_counts = searcher.search(&AllQuery, &facet_collector)?; /// /// // This lists all of the facet counts /// let facets: Vec<(&Facet, u64)> = facet_counts @@ -163,7 +162,7 @@ fn facet_depth(facet_bytes: &[u8]) -> usize { /// { /// let mut facet_collector = FacetCollector::for_field(facet); /// facet_collector.add_facet("/category/fiction"); -/// let facet_counts = searcher.search(&AllQuery, &facet_collector).unwrap(); +/// let facet_counts = searcher.search(&AllQuery, &facet_collector)?; /// /// // This lists all of the facet counts /// let facets: Vec<(&Facet, u64)> = facet_counts.top_k("/category/fiction", 1); @@ -483,8 +482,8 @@ mod tests { index_writer.add_document(doc); } index_writer.commit().unwrap(); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let mut facet_collector = FacetCollector::for_field(facet_field); facet_collector.add_facet(Facet::from("/top1")); let counts = searcher.search(&AllQuery, &facet_collector).unwrap(); @@ -532,8 +531,8 @@ mod tests { facet_field => Facet::from_text(&"/subjects/B/b"), )); index_writer.commit().unwrap(); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); assert_eq!(searcher.num_docs(), 1); let mut facet_collector = FacetCollector::for_field(facet_field); facet_collector.add_facet("/subjects"); @@ -579,9 +578,7 @@ mod tests { index_writer.add_document(doc); } index_writer.commit().unwrap(); - index.load_searchers().unwrap(); - - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let mut facet_collector = FacetCollector::for_field(facet_field); facet_collector.add_facet("/facet"); @@ -635,8 +632,7 @@ mod bench { index_writer.add_document(doc); } index_writer.commit().unwrap(); - index.load_searchers().unwrap(); - + let reader = index.reader().unwrap(); b.iter(|| { let searcher = index.searcher(); let facet_collector = FacetCollector::for_field(facet_field); diff --git a/src/collector/int_facet_collector.rs b/src/collector/int_facet_collector.rs index 01f00cc37..4232343e6 100644 --- a/src/collector/int_facet_collector.rs +++ b/src/collector/int_facet_collector.rs @@ -101,8 +101,7 @@ mod tests { assert_eq!(index_writer.commit().unwrap(), 10u64); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().searcher(); let mut ffvf_i64: IntFacetCollector = IntFacetCollector::new(num_field_i64); let mut ffvf_u64: IntFacetCollector = IntFacetCollector::new(num_field_u64); diff --git a/src/collector/mod.rs b/src/collector/mod.rs index 046b26ed0..515799fdf 100644 --- a/src/collector/mod.rs +++ b/src/collector/mod.rs @@ -53,9 +53,9 @@ use tantivy::collector::{Count, TopDocs}; # index_writer.add_document(doc!( # title => "The Diary of Muadib", # )); -# index_writer.commit().unwrap(); -# index.load_searchers()?; -# let searcher = index.searcher(); +# index_writer.commit()?; +# let reader = index.reader()?; +# let searcher = reader.searcher(); # let query_parser = QueryParser::for_index(&index, vec![title]); # let query = query_parser.parse_query("diary")?; let (doc_count, top_docs): (usize, Vec<(Score, DocAddress)>) = diff --git a/src/collector/multi_collector.rs b/src/collector/multi_collector.rs index f12c648a4..cc3bfc488 100644 --- a/src/collector/multi_collector.rs +++ b/src/collector/multi_collector.rs @@ -134,8 +134,8 @@ impl FruitHandle { /// index_writer.commit().unwrap(); /// } /// -/// index.load_searchers()?; -/// let searcher = index.searcher(); +/// let reader = index.reader()?; +/// let searcher = reader.searcher(); /// /// let mut collectors = MultiCollector::new(); /// let top_docs_handle = collectors.add_collector(TopDocs::with_limit(2)); @@ -278,8 +278,7 @@ mod tests { index_writer.add_document(doc!(text=>"abc")); index_writer.commit().unwrap(); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let term = Term::from_field_text(text, "abc"); let query = TermQuery::new(term, IndexRecordOption::Basic); diff --git a/src/collector/top_field_collector.rs b/src/collector/top_field_collector.rs index 02551310c..b1a2d5ec8 100644 --- a/src/collector/top_field_collector.rs +++ b/src/collector/top_field_collector.rs @@ -23,15 +23,16 @@ use SegmentReader; /// # use tantivy::schema::{Schema, Field, FAST, TEXT}; /// # use tantivy::{Index, Result, DocAddress}; /// # use tantivy::query::{Query, QueryParser}; +/// use tantivy::Searcher; /// use tantivy::collector::TopDocs; /// -/// # fn main() { +/// # fn main() -> tantivy::Result<()> { /// # let mut schema_builder = Schema::builder(); /// # let title = schema_builder.add_text_field("title", TEXT); /// # let rating = schema_builder.add_u64_field("rating", FAST); /// # let schema = schema_builder.build(); /// # let index = Index::create_in_ram(schema); -/// # let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); +/// # let mut index_writer = index.writer_with_num_threads(1, 3_000_000)?; /// # index_writer.add_document(doc!( /// # title => "The Name of the Wind", /// # rating => 92u64, @@ -39,13 +40,14 @@ use SegmentReader; /// # index_writer.add_document(doc!(title => "The Diary of Muadib", rating => 97u64)); /// # index_writer.add_document(doc!(title => "A Dairy Cow", rating => 63u64)); /// # index_writer.add_document(doc!(title => "The Diary of a Young Girl", rating => 80u64)); -/// # index_writer.commit().unwrap(); -/// # index.load_searchers().unwrap(); -/// # let query = QueryParser::for_index(&index, vec![title]).parse_query("diary").unwrap(); -/// # let top_docs = docs_sorted_by_rating(&index, &query, rating).unwrap(); +/// # index_writer.commit()?; +/// # let reader = index.reader()?; +/// # let query = QueryParser::for_index(&index, vec![title]).parse_query("diary")?; +/// # let top_docs = docs_sorted_by_rating(&reader.searcher(), &query, rating)?; /// # assert_eq!(top_docs, /// # vec![(97u64, DocAddress(0u32, 1)), /// # (80u64, DocAddress(0u32, 3))]); +/// # Ok(()) /// # } /// # /// /// Searches the document matching the given query, and @@ -53,7 +55,9 @@ use SegmentReader; /// /// given in argument. /// /// /// /// `field` is required to be a FAST field. -/// fn docs_sorted_by_rating(index: &Index, query: &Query, sort_by_field: Field) +/// fn docs_sorted_by_rating(searcher: &Searcher, +/// query: &Query, +/// sort_by_field: Field) /// -> Result> { /// /// // This is where we build our collector! @@ -61,8 +65,7 @@ use SegmentReader; /// /// // ... and here is our documents. Not this is a simple vec. /// // The `u64` in the pair is the value of our fast field for each documents. -/// index.searcher() -/// .search(query, &top_docs_by_rating) +/// searcher.search(query, &top_docs_by_rating) /// } /// ``` pub struct TopDocsByField { @@ -177,7 +180,7 @@ mod tests { size => 16u64, )); }); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let top_collector = TopDocs::with_limit(4).order_by_field(size); let top_docs: Vec<(u64, DocAddress)> = searcher.search(&query, &top_collector).unwrap(); @@ -204,7 +207,7 @@ mod tests { size => 12u64, )); }); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let top_collector: TopDocsByField = TopDocs::with_limit(4).order_by_field(Field(2)); let segment_reader = searcher.segment_reader(0u32); top_collector @@ -224,7 +227,7 @@ mod tests { size => 12u64, )); }); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let segment = searcher.segment_reader(0); let top_collector: TopDocsByField = TopDocs::with_limit(4).order_by_field(size); assert_matches!( @@ -247,8 +250,6 @@ mod tests { let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); doc_adder(&mut index_writer); index_writer.commit().unwrap(); - index.load_searchers().unwrap(); - let query_parser = QueryParser::for_index(&index, vec![query_field]); let query = query_parser.parse_query(query).unwrap(); (index, query) diff --git a/src/collector/top_score_collector.rs b/src/collector/top_score_collector.rs index 869022686..43b7424d6 100644 --- a/src/collector/top_score_collector.rs +++ b/src/collector/top_score_collector.rs @@ -51,8 +51,8 @@ use SegmentReader; /// index_writer.commit().unwrap(); /// } /// -/// index.load_searchers()?; -/// let searcher = index.searcher(); +/// let reader = index.reader()?; +/// let searcher = reader.searcher(); /// /// let query_parser = QueryParser::for_index(&index, vec![title]); /// let query = query_parser.parse_query("diary")?; @@ -148,7 +148,6 @@ mod tests { index_writer.add_document(doc!(text_field=>"I like Droopy")); assert!(index_writer.commit().is_ok()); } - index.load_searchers().unwrap(); index } @@ -159,6 +158,8 @@ mod tests { let query_parser = QueryParser::for_index(&index, vec![field]); let text_query = query_parser.parse_query("droopy tax").unwrap(); let score_docs: Vec<(Score, DocAddress)> = index + .reader() + .unwrap() .searcher() .search(&text_query, &TopDocs::with_limit(4)) .unwrap(); @@ -179,6 +180,8 @@ mod tests { let query_parser = QueryParser::for_index(&index, vec![field]); let text_query = query_parser.parse_query("droopy tax").unwrap(); let score_docs: Vec<(Score, DocAddress)> = index + .reader() + .unwrap() .searcher() .search(&text_query, &TopDocs::with_limit(2)) .unwrap(); diff --git a/src/core/index.rs b/src/core/index.rs index 8638d7fa2..62a31cc95 100644 --- a/src/core/index.rs +++ b/src/core/index.rs @@ -1,19 +1,14 @@ -use super::pool::LeasedItem; -use super::pool::Pool; use super::segment::create_segment; use super::segment::Segment; -use core::searcher::Searcher; use core::Executor; use core::IndexMeta; use core::SegmentId; use core::SegmentMeta; -use core::SegmentReader; use core::META_FILEPATH; use directory::ManagedDirectory; #[cfg(feature = "mmap")] use directory::MmapDirectory; use directory::INDEX_WRITER_LOCK; -use directory::META_LOCK; use directory::{Directory, RAMDirectory}; use error::DataCorruption; use error::TantivyError; @@ -21,6 +16,8 @@ use indexer::index_writer::open_index_writer; use indexer::index_writer::HEAP_SIZE_MIN; use indexer::segment_updater::save_new_metas; use num_cpus; +use reader::IndexReader; +use reader::IndexReaderBuilder; use schema::Field; use schema::FieldType; use schema::Schema; @@ -28,7 +25,6 @@ use serde_json; use std::borrow::BorrowMut; use std::fmt; use std::path::Path; -use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; use tokenizer::BoxedTokenizer; use tokenizer::TokenizerManager; @@ -53,8 +49,6 @@ fn load_metas(directory: &Directory) -> Result { pub struct Index { directory: ManagedDirectory, schema: Schema, - num_searchers: Arc, - searcher_pool: Arc>, executor: Arc, tokenizers: TokenizerManager, } @@ -159,16 +153,12 @@ impl Index { /// Creates a new index given a directory and an `IndexMeta`. fn create_from_metas(directory: ManagedDirectory, metas: &IndexMeta) -> Result { let schema = metas.schema.clone(); - let n_cpus = num_cpus::get(); let index = Index { directory, schema, - num_searchers: Arc::new(AtomicUsize::new(n_cpus)), - searcher_pool: Arc::new(Pool::new()), tokenizers: TokenizerManager::default(), executor: Arc::new(Executor::single_thread()), }; - index.load_searchers()?; Ok(index) } @@ -198,6 +188,22 @@ impl Index { } } + /// Create a default `IndexReader` for the given index. + /// + /// See [`Index.reader_builder()`](#method.reader_builder). + pub fn reader(&self) -> Result { + self.reader_builder().try_into() + } + + /// Create a `IndexReader` for the given index. + /// + /// Most project should create at most one reader for a given index. + /// This method is typically called only once per `Index` instance, + /// over the lifetime of most problem. + pub fn reader_builder(&self) -> IndexReaderBuilder { + IndexReaderBuilder::new(self.clone()) + } + /// Opens a new directory from an index path. #[cfg(feature = "mmap")] pub fn open_in_dir>(directory_path: P) -> Result { @@ -336,53 +342,6 @@ impl Index { .map(|segment_meta| segment_meta.id()) .collect()) } - - /// Sets the number of searchers to use - /// - /// Only works after the next call to `load_searchers` - pub fn set_num_searchers(&mut self, num_searchers: usize) { - self.num_searchers.store(num_searchers, Ordering::Release); - } - - /// Update searchers so that they reflect the state of the last - /// `.commit()`. - /// - /// If indexing happens in the same process as searching, - /// you most likely want to call `.load_searchers()` right after each - /// successful call to `.commit()`. - /// - /// If indexing and searching happen in different processes, the way to - /// get the freshest `index` at all time, is to watch `meta.json` and - /// call `load_searchers` whenever a changes happen. - pub fn load_searchers(&self) -> Result<()> { - let _meta_lock = self.directory().acquire_lock(&META_LOCK)?; - let searchable_segments = self.searchable_segments()?; - let segment_readers: Vec = searchable_segments - .iter() - .map(SegmentReader::open) - .collect::>()?; - let schema = self.schema(); - let num_searchers: usize = self.num_searchers.load(Ordering::Acquire); - let searchers = (0..num_searchers) - .map(|_| Searcher::new(schema.clone(), self.clone(), segment_readers.clone())) - .collect(); - self.searcher_pool.publish_new_generation(searchers); - Ok(()) - } - - /// Returns a searcher - /// - /// This method should be called every single time a search - /// query is performed. - /// The searchers are taken from a pool of `num_searchers` searchers. - /// If no searcher is available - /// this may block. - /// - /// The same searcher must be used for a given query, as it ensures - /// the use of a consistent segment set. - pub fn searcher(&self) -> LeasedItem { - self.searcher_pool.acquire() - } } impl fmt::Debug for Index { @@ -394,8 +353,16 @@ impl fmt::Debug for Index { #[cfg(test)] mod tests { use directory::RAMDirectory; + use schema::Field; use schema::{Schema, INDEXED, TEXT}; + use std::path::PathBuf; + use std::thread; + use std::time::Duration; + use tempdir::TempDir; use Index; + use IndexReader; + use IndexWriter; + use ReloadPolicy; #[test] fn test_indexer_for_field() { @@ -461,4 +428,106 @@ mod tests { let _ = schema_builder.add_u64_field("num_likes", INDEXED); schema_builder.build() } + + #[test] + fn test_index_on_commit_reload_policy() { + let schema = throw_away_schema(); + let field = schema.get_field("num_likes").unwrap(); + let index = Index::create_in_ram(schema); + let reader = index + .reader_builder() + .reload_policy(ReloadPolicy::OnCommit) + .try_into() + .unwrap(); + assert_eq!(reader.searcher().num_docs(), 0); + let mut writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); + test_index_on_commit_reload_policy_aux(field, &mut writer, &reader); + } + + #[test] + fn test_index_on_commit_reload_policy_mmap() { + let schema = throw_away_schema(); + let field = schema.get_field("num_likes").unwrap(); + let tempdir = TempDir::new("index").unwrap(); + let tempdir_path = PathBuf::from(tempdir.path()); + let index = Index::create_in_dir(&tempdir_path, schema).unwrap(); + let mut writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); + writer.commit().unwrap(); + let reader = index + .reader_builder() + .reload_policy(ReloadPolicy::OnCommit) + .try_into() + .unwrap(); + assert_eq!(reader.searcher().num_docs(), 0); + test_index_on_commit_reload_policy_aux(field, &mut writer, &reader); + } + + #[test] + fn test_index_manual_policy_mmap() { + let schema = throw_away_schema(); + let field = schema.get_field("num_likes").unwrap(); + let index = Index::create_from_tempdir(schema).unwrap(); + let mut writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); + writer.commit().unwrap(); + let reader = index + .reader_builder() + .reload_policy(ReloadPolicy::Manual) + .try_into() + .unwrap(); + assert_eq!(reader.searcher().num_docs(), 0); + writer.add_document(doc!(field=>1u64)); + writer.commit().unwrap(); + thread::sleep(Duration::from_millis(500)); + assert_eq!(reader.searcher().num_docs(), 0); + reader.reload().unwrap(); + assert_eq!(reader.searcher().num_docs(), 1); + } + + #[test] + fn test_index_on_commit_reload_policy_different_directories() { + let schema = throw_away_schema(); + let field = schema.get_field("num_likes").unwrap(); + let tempdir = TempDir::new("index").unwrap(); + let tempdir_path = PathBuf::from(tempdir.path()); + let write_index = Index::create_in_dir(&tempdir_path, schema).unwrap(); + let read_index = Index::open_in_dir(&tempdir_path).unwrap(); + let reader = read_index + .reader_builder() + .reload_policy(ReloadPolicy::OnCommit) + .try_into() + .unwrap(); + assert_eq!(reader.searcher().num_docs(), 0); + let mut writer = write_index.writer_with_num_threads(1, 3_000_000).unwrap(); + test_index_on_commit_reload_policy_aux(field, &mut writer, &reader); + } + + fn test_index_on_commit_reload_policy_aux( + field: Field, + writer: &mut IndexWriter, + reader: &IndexReader, + ) { + assert_eq!(reader.searcher().num_docs(), 0); + writer.add_document(doc!(field=>1u64)); + writer.commit().unwrap(); + let mut count = 0; + for _ in 0..100 { + count = reader.searcher().num_docs(); + if count > 0 { + break; + } + thread::sleep(Duration::from_millis(100)); + } + assert_eq!(count, 1); + writer.add_document(doc!(field=>2u64)); + writer.commit().unwrap(); + let mut count = 0; + for _ in 0..10 { + count = reader.searcher().num_docs(); + if count > 1 { + break; + } + thread::sleep(Duration::from_millis(100)); + } + assert_eq!(count, 2); + } } diff --git a/src/core/mod.rs b/src/core/mod.rs index 1aae75f8b..9e5717afa 100644 --- a/src/core/mod.rs +++ b/src/core/mod.rs @@ -2,7 +2,6 @@ mod executor; pub mod index; mod index_meta; mod inverted_index_reader; -mod pool; pub mod searcher; mod segment; mod segment_component; @@ -25,6 +24,7 @@ pub use self::segment_reader::SegmentReader; use std::path::PathBuf; lazy_static! { + /// The meta file contains all the information about the list of segments and the schema /// of the index. pub static ref META_FILEPATH: PathBuf = PathBuf::from("meta.json"); diff --git a/src/core/segment_reader.rs b/src/core/segment_reader.rs index 597ecae39..a8ac2b0fb 100644 --- a/src/core/segment_reader.rs +++ b/src/core/segment_reader.rs @@ -477,9 +477,7 @@ mod test { // ok, now we should have a deleted doc index_writer2.commit().unwrap(); } - - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let docs: Vec = searcher.segment_reader(0).doc_ids_alive().collect(); assert_eq!(vec![0u32, 2u32], docs); } diff --git a/src/directory/directory.rs b/src/directory/directory.rs index c5f975c9d..943998e2d 100644 --- a/src/directory/directory.rs +++ b/src/directory/directory.rs @@ -1,6 +1,8 @@ use directory::directory_lock::Lock; use directory::error::LockError; use directory::error::{DeleteError, OpenReadError, OpenWriteError}; +use directory::WatchCallback; +use directory::WatchHandle; use directory::{ReadOnlySource, WritePtr}; use std::fmt; use std::io; @@ -187,6 +189,22 @@ pub trait Directory: DirectoryClone + fmt::Debug + Send + Sync + 'static { } } } + + /// Registers a callback that will be called whenever a change on the `meta.json` + /// using the `atomic_write` API is detected. + /// + /// The behavior when using `.watch()` on a file using `.open_write(...)` is, on the other + /// hand, undefined. + /// + /// The file will be watched for the lifetime of the returned `WatchHandle`. The caller is + /// required to keep it. + /// It does not override previous callbacks. When the file is modified, all callback that are + /// registered (and whose `WatchHandle` is still alive) are triggered. + /// + /// Internally, tantivy only uses this API to detect new commits to implement the + /// `OnCommit` `ReloadPolicy`. Not implementing watch in a `Directory` only prevents the + /// `OnCommit` `ReloadPolicy` to work properly. + fn watch(&self, watch_callback: WatchCallback) -> WatchHandle; } /// DirectoryClone diff --git a/src/directory/directory_lock.rs b/src/directory/directory_lock.rs index f341ea6f9..67c2585dd 100644 --- a/src/directory/directory_lock.rs +++ b/src/directory/directory_lock.rs @@ -43,7 +43,7 @@ lazy_static! { is_blocking: false }; /// The meta lock file is here to protect the segment files being opened by - /// `.load_searchers()` from being garbage collected. + /// `IndexReader::reload()` from being garbage collected. /// It makes it possible for another process to safely consume /// our index in-writing. Ideally, we may have prefered `RWLock` semantics /// here, but it is difficult to achieve on Windows. diff --git a/src/directory/error.rs b/src/directory/error.rs index 1179ad28d..e56971029 100644 --- a/src/directory/error.rs +++ b/src/directory/error.rs @@ -73,6 +73,14 @@ pub enum OpenDirectoryError { DoesNotExist(PathBuf), /// The path exists but is not a directory. NotADirectory(PathBuf), + /// IoError + IoError(io::Error), +} + +impl From for OpenDirectoryError { + fn from(io_err: io::Error) -> Self { + OpenDirectoryError::IoError(io_err) + } } impl fmt::Display for OpenDirectoryError { @@ -84,6 +92,11 @@ impl fmt::Display for OpenDirectoryError { OpenDirectoryError::NotADirectory(ref path) => { write!(f, "the path '{:?}' exists but is not a directory", path) } + OpenDirectoryError::IoError(ref err) => write!( + f, + "IOError while trying to open/create the directory. {:?}", + err + ), } } } diff --git a/src/directory/managed_directory.rs b/src/directory/managed_directory.rs index 5367494f0..ddd8dfbba 100644 --- a/src/directory/managed_directory.rs +++ b/src/directory/managed_directory.rs @@ -4,6 +4,7 @@ use directory::DirectoryLock; use directory::Lock; use directory::META_LOCK; use directory::{ReadOnlySource, WritePtr}; +use directory::{WatchCallback, WatchHandle}; use error::DataCorruption; use serde_json; use std::collections::HashSet; @@ -241,6 +242,10 @@ impl Directory for ManagedDirectory { fn acquire_lock(&self, lock: &Lock) -> result::Result { self.directory.acquire_lock(lock) } + + fn watch(&self, watch_callback: WatchCallback) -> WatchHandle { + self.directory.watch(watch_callback) + } } impl Clone for ManagedDirectory { diff --git a/src/directory/mmap_directory.rs b/src/directory/mmap_directory.rs index 8489f806f..70c277d56 100644 --- a/src/directory/mmap_directory.rs +++ b/src/directory/mmap_directory.rs @@ -1,8 +1,13 @@ extern crate fs2; +extern crate notify; use self::fs2::FileExt; +use self::notify::RawEvent; +use self::notify::RecursiveMode; +use self::notify::Watcher; use atomicwrites; use common::make_io_err; +use core::META_FILEPATH; use directory::error::LockError; use directory::error::{DeleteError, IOError, OpenDirectoryError, OpenReadError, OpenWriteError}; use directory::read_only_source::BoxedData; @@ -10,6 +15,9 @@ use directory::Directory; use directory::DirectoryLock; use directory::Lock; use directory::ReadOnlySource; +use directory::WatchCallback; +use directory::WatchCallbackList; +use directory::WatchHandle; use directory::WritePtr; use memmap::Mmap; use std::collections::HashMap; @@ -21,14 +29,16 @@ use std::io::{self, Seek, SeekFrom}; use std::io::{BufWriter, Read, Write}; use std::path::{Path, PathBuf}; use std::result; +use std::sync::mpsc::{channel, Receiver, Sender}; use std::sync::Arc; +use std::sync::Mutex; use std::sync::RwLock; use std::sync::Weak; +use std::thread; use tempdir::TempDir; /// Returns None iff the file exists, can be read, but is empty (and hence -/// cannot be mmapped). -/// +/// cannot be mmapped) fn open_mmap(full_path: &Path) -> result::Result, OpenReadError> { let file = File::open(full_path).map_err(|e| { if e.kind() == io::ErrorKind::NotFound { @@ -84,7 +94,7 @@ impl Default for MmapCache { } impl MmapCache { - fn get_info(&mut self) -> CacheInfo { + fn get_info(&self) -> CacheInfo { let paths: Vec = self.cache.keys().cloned().collect(); CacheInfo { counters: self.counters.clone(), @@ -92,28 +102,105 @@ impl MmapCache { } } + fn remove_weak_ref(&mut self) { + let keys_to_remove: Vec = self + .cache + .iter() + .filter(|(_, mmap_weakref)| mmap_weakref.upgrade().is_none()) + .map(|(key, _)| key.clone()) + .collect(); + for key in keys_to_remove { + self.cache.remove(&key); + } + } + // Returns None if the file exists but as a len of 0 (and hence is not mmappable). fn get_mmap(&mut self, full_path: &Path) -> Result>, OpenReadError> { - let path_in_cache = self.cache.contains_key(full_path); - if path_in_cache { - { - let mmap_weak_opt = self.cache.get(full_path); - if let Some(mmap_arc) = mmap_weak_opt.and_then(|mmap_weak| mmap_weak.upgrade()) { - self.counters.hit += 1; - return Ok(Some(mmap_arc)); - } + if let Some(mmap_weak) = self.cache.get(full_path) { + if let Some(mmap_arc) = mmap_weak.upgrade() { + self.counters.hit += 1; + return Ok(Some(mmap_arc)); } - self.cache.remove(full_path); } + self.cache.remove(full_path); self.counters.miss += 1; - if let Some(mmap) = open_mmap(full_path)? { + Ok(if let Some(mmap) = open_mmap(full_path)? { let mmap_arc: Arc = Arc::new(Box::new(mmap)); - self.cache - .insert(full_path.to_owned(), Arc::downgrade(&mmap_arc)); - Ok(Some(mmap_arc)) + let mmap_weak = Arc::downgrade(&mmap_arc); + self.cache.insert(full_path.to_owned(), mmap_weak); + Some(mmap_arc) } else { - Ok(None) - } + None + }) + } +} + +struct InnerWatcherWrapper { + _watcher: Mutex, + watcher_router: WatchCallbackList, +} + +impl InnerWatcherWrapper { + pub fn new(path: &Path) -> Result<(Self, Receiver), notify::Error> { + let (tx, watcher_recv): (Sender, Receiver) = channel(); + // We need to initialize the + let mut watcher = notify::raw_watcher(tx)?; + watcher.watch(path, RecursiveMode::Recursive)?; + let inner = InnerWatcherWrapper { + _watcher: Mutex::new(watcher), + watcher_router: Default::default(), + }; + Ok((inner, watcher_recv)) + } +} + +#[derive(Clone)] +pub(crate) struct WatcherWrapper { + inner: Arc, +} + +impl WatcherWrapper { + pub fn new(path: &Path) -> Result { + let (inner, watcher_recv) = InnerWatcherWrapper::new(path).map_err(|err| match err { + notify::Error::PathNotFound => OpenDirectoryError::DoesNotExist(path.to_owned()), + _ => { + panic!("Unknown error while starting watching directory {:?}", path); + } + })?; + let watcher_wrapper = WatcherWrapper { + inner: Arc::new(inner), + }; + let watcher_wrapper_clone = watcher_wrapper.clone(); + thread::Builder::new() + .name("meta-file-watch-thread".to_string()) + .spawn(move || { + loop { + match watcher_recv.recv().map(|evt| evt.path) { + Ok(Some(changed_path)) => { + // ... Actually subject to false positive. + // We might want to be more accurate than this at one point. + if let Some(filename) = changed_path.file_name() { + if filename == *META_FILEPATH { + watcher_wrapper_clone.inner.watcher_router.broadcast(); + } + } + } + Ok(None) => { + // not an event we are interested in. + } + Err(_e) => { + // the watch send channel was dropped + break; + } + } + } + }) + .expect("Failed to spawn thread to watch meta.json"); + Ok(watcher_wrapper) + } + + pub fn watch(&mut self, watch_callback: WatchCallback) -> WatchHandle { + self.inner.watcher_router.subscribe(watch_callback) } } @@ -131,31 +218,62 @@ impl MmapCache { /// On Windows the semantics are again different. #[derive(Clone)] pub struct MmapDirectory { + inner: Arc, +} + +struct MmapDirectoryInner { root_path: PathBuf, - mmap_cache: Arc>, - _temp_directory: Arc>, + mmap_cache: RwLock, + _temp_directory: Option, + watcher: RwLock, +} + +impl MmapDirectoryInner { + fn new( + root_path: PathBuf, + temp_directory: Option, + ) -> Result { + let watch_wrapper = WatcherWrapper::new(&root_path)?; + let mmap_directory_inner = MmapDirectoryInner { + root_path, + mmap_cache: Default::default(), + _temp_directory: temp_directory, + watcher: RwLock::new(watch_wrapper), + }; + Ok(mmap_directory_inner) + } + + fn watch(&self, watch_callback: WatchCallback) -> WatchHandle { + let mut wlock = self.watcher.write().unwrap(); + wlock.watch(watch_callback) + } } impl fmt::Debug for MmapDirectory { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "MmapDirectory({:?})", self.root_path) + write!(f, "MmapDirectory({:?})", self.inner.root_path) } } impl MmapDirectory { + fn new( + root_path: PathBuf, + temp_directory: Option, + ) -> Result { + let inner = MmapDirectoryInner::new(root_path, temp_directory)?; + Ok(MmapDirectory { + inner: Arc::new(inner), + }) + } + /// Creates a new MmapDirectory in a temporary directory. /// /// This is mostly useful to test the MmapDirectory itself. /// For your unit tests, prefer the RAMDirectory. - pub fn create_from_tempdir() -> io::Result { - let tempdir = TempDir::new("index")?; + pub fn create_from_tempdir() -> Result { + let tempdir = TempDir::new("index").map_err(OpenDirectoryError::IoError)?; let tempdir_path = PathBuf::from(tempdir.path()); - let directory = MmapDirectory { - root_path: tempdir_path, - mmap_cache: Arc::new(RwLock::new(MmapCache::default())), - _temp_directory: Arc::new(Some(tempdir)), - }; - Ok(directory) + MmapDirectory::new(tempdir_path, Some(tempdir)) } /// Opens a MmapDirectory in a directory. @@ -173,18 +291,14 @@ impl MmapDirectory { directory_path, ))) } else { - Ok(MmapDirectory { - root_path: PathBuf::from(directory_path), - mmap_cache: Arc::new(RwLock::new(MmapCache::default())), - _temp_directory: Arc::new(None), - }) + Ok(MmapDirectory::new(PathBuf::from(directory_path), None)?) } } /// Joins a relative_path to the directory `root_path` /// to create a proper complete `filepath`. fn resolve_path(&self, relative_path: &Path) -> PathBuf { - self.root_path.join(relative_path) + self.inner.root_path.join(relative_path) } /// Sync the root directory. @@ -209,7 +323,7 @@ impl MmapDirectory { .custom_flags(winbase::FILE_FLAG_BACKUP_SEMANTICS); } - let fd = open_opts.open(&self.root_path)?; + let fd = open_opts.open(&self.inner.root_path)?; fd.sync_all()?; Ok(()) } @@ -219,9 +333,15 @@ impl MmapDirectory { /// /// The `MmapDirectory` embeds a `MmapDirectory` /// to avoid multiplying the `mmap` system calls. - pub fn get_cache_info(&mut self) -> CacheInfo { - self.mmap_cache + pub fn get_cache_info(&self) -> CacheInfo { + self.inner + .mmap_cache .write() + .expect("mmap cache lock is poisoned") + .remove_weak_ref(); + self.inner + .mmap_cache + .read() .expect("Mmap cache lock is poisoned.") .get_info() } @@ -274,7 +394,7 @@ impl Directory for MmapDirectory { debug!("Open Read {:?}", path); let full_path = self.resolve_path(path); - let mut mmap_cache = self.mmap_cache.write().map_err(|_| { + let mut mmap_cache = self.inner.mmap_cache.write().map_err(|_| { let msg = format!( "Failed to acquired write lock \ on mmap cache while reading {:?}", @@ -288,6 +408,30 @@ impl Directory for MmapDirectory { .unwrap_or_else(ReadOnlySource::empty)) } + /// Any entry associated to the path in the mmap will be + /// removed before the file is deleted. + fn delete(&self, path: &Path) -> result::Result<(), DeleteError> { + debug!("Deleting file {:?}", path); + let full_path = self.resolve_path(path); + match fs::remove_file(&full_path) { + Ok(_) => self + .sync_directory() + .map_err(|e| IOError::with_path(path.to_owned(), e).into()), + Err(e) => { + if e.kind() == io::ErrorKind::NotFound { + Err(DeleteError::FileDoesNotExist(path.to_owned())) + } else { + Err(IOError::with_path(path.to_owned(), e).into()) + } + } + } + } + + fn exists(&self, path: &Path) -> bool { + let full_path = self.resolve_path(path); + full_path.exists() + } + fn open_write(&mut self, path: &Path) -> Result { debug!("Open Write {:?}", path); let full_path = self.resolve_path(path); @@ -318,30 +462,6 @@ impl Directory for MmapDirectory { Ok(BufWriter::new(Box::new(writer))) } - /// Any entry associated to the path in the mmap will be - /// removed before the file is deleted. - fn delete(&self, path: &Path) -> result::Result<(), DeleteError> { - debug!("Deleting file {:?}", path); - let full_path = self.resolve_path(path); - match fs::remove_file(&full_path) { - Ok(_) => self - .sync_directory() - .map_err(|e| IOError::with_path(path.to_owned(), e).into()), - Err(e) => { - if e.kind() == io::ErrorKind::NotFound { - Err(DeleteError::FileDoesNotExist(path.to_owned())) - } else { - Err(IOError::with_path(path.to_owned(), e).into()) - } - } - } - } - - fn exists(&self, path: &Path) -> bool { - let full_path = self.resolve_path(path); - full_path.exists() - } - fn atomic_read(&self, path: &Path) -> Result, OpenReadError> { let full_path = self.resolve_path(path); let mut buffer = Vec::new(); @@ -388,6 +508,10 @@ impl Directory for MmapDirectory { _file: file, }))) } + + fn watch(&self, watch_callback: WatchCallback) -> WatchHandle { + self.inner.watch(watch_callback) + } } #[cfg(test)] @@ -397,6 +521,13 @@ mod tests { // The following tests are specific to the MmapDirectory use super::*; + use schema::{Schema, SchemaBuilder, TEXT}; + use std::fs; + use std::sync::atomic::{AtomicUsize, Ordering}; + use std::thread; + use std::time::Duration; + use Index; + use ReloadPolicy; #[test] fn test_open_non_existant_path() { @@ -421,7 +552,7 @@ mod tests { #[test] fn test_cache() { - let content = "abc".as_bytes(); + let content = b"abc"; // here we test if the cache releases // mmaps correctly. @@ -456,26 +587,27 @@ mod tests { for path in paths.iter() { let _r = mmap_directory.open_read(path).unwrap(); - assert_eq!(mmap_directory.get_cache_info().mmapped.len(), num_paths); + assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 10); } + assert_eq!(mmap_directory.get_cache_info().counters.hit, 20); assert_eq!(mmap_directory.get_cache_info().counters.miss, 10); assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 10); drop(keep); for path in paths.iter() { let _r = mmap_directory.open_read(path).unwrap(); - assert_eq!(mmap_directory.get_cache_info().mmapped.len(), num_paths); + assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 1); } assert_eq!(mmap_directory.get_cache_info().counters.hit, 20); assert_eq!(mmap_directory.get_cache_info().counters.miss, 20); - assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 10); + assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 0); for path in &paths { mmap_directory.delete(path).unwrap(); } assert_eq!(mmap_directory.get_cache_info().counters.hit, 20); assert_eq!(mmap_directory.get_cache_info().counters.miss, 20); - assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 10); + assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 0); for path in paths.iter() { assert!(mmap_directory.open_read(path).is_err()); } @@ -484,4 +616,56 @@ mod tests { assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 0); } + #[test] + fn test_watch_wrapper() { + let counter: Arc = Default::default(); + let counter_clone = counter.clone(); + let tmp_dir: TempDir = tempdir::TempDir::new("test_watch_wrapper").unwrap(); + let tmp_dirpath = tmp_dir.path().to_owned(); + let mut watch_wrapper = WatcherWrapper::new(&tmp_dirpath).unwrap(); + let tmp_file = tmp_dirpath.join("coucou"); + let _handle = watch_wrapper.watch(Box::new(move || { + counter_clone.fetch_add(1, Ordering::SeqCst); + })); + assert_eq!(counter.load(Ordering::SeqCst), 0); + fs::write(&tmp_file, b"whateverwilldo").unwrap(); + thread::sleep(Duration::new(0, 1_000u32)); + } + + #[test] + fn test_mmap_released() { + let mmap_directory = MmapDirectory::create_from_tempdir().unwrap(); + let mut schema_builder: SchemaBuilder = Schema::builder(); + let text_field = schema_builder.add_text_field("text", TEXT); + let schema = schema_builder.build(); + { + let index = Index::create(mmap_directory.clone(), schema).unwrap(); + let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); + for _num_commits in 0..16 { + for _ in 0..10 { + index_writer.add_document(doc!(text_field=>"abc")); + } + index_writer.commit().unwrap(); + } + let reader = index + .reader_builder() + .reload_policy(ReloadPolicy::Manual) + .try_into() + .unwrap(); + for _ in 0..30 { + index_writer.add_document(doc!(text_field=>"abc")); + index_writer.commit().unwrap(); + reader.reload().unwrap(); + } + index_writer.wait_merging_threads().unwrap(); + reader.reload().unwrap(); + let num_segments = reader.searcher().segment_readers().len(); + assert_eq!(num_segments, 4); + assert_eq!( + num_segments * 7, + mmap_directory.get_cache_info().mmapped.len() + ); + } + assert_eq!(mmap_directory.get_cache_info().mmapped.len(), 0); + } } diff --git a/src/directory/mod.rs b/src/directory/mod.rs index 6f7ef88b8..8d880b0f9 100644 --- a/src/directory/mod.rs +++ b/src/directory/mod.rs @@ -12,6 +12,7 @@ mod directory_lock; mod managed_directory; mod ram_directory; mod read_only_source; +mod watch_event_router; /// Errors specific to the directory module. pub mod error; @@ -21,6 +22,8 @@ pub use self::directory::{Directory, DirectoryClone}; pub use self::directory_lock::{Lock, INDEX_WRITER_LOCK, META_LOCK}; pub use self::ram_directory::RAMDirectory; pub use self::read_only_source::ReadOnlySource; +pub(crate) use self::watch_event_router::WatchCallbackList; +pub use self::watch_event_router::{WatchCallback, WatchHandle}; use std::io::{BufWriter, Seek, Write}; #[cfg(feature = "mmap")] diff --git a/src/directory/ram_directory.rs b/src/directory/ram_directory.rs index 9423affff..985117740 100644 --- a/src/directory/ram_directory.rs +++ b/src/directory/ram_directory.rs @@ -1,7 +1,8 @@ -use common::make_io_err; -use directory::error::{DeleteError, IOError, OpenReadError, OpenWriteError}; +use core::META_FILEPATH; +use directory::error::{DeleteError, OpenReadError, OpenWriteError}; +use directory::WatchCallbackList; use directory::WritePtr; -use directory::{Directory, ReadOnlySource}; +use directory::{Directory, ReadOnlySource, WatchCallback, WatchHandle}; use std::collections::HashMap; use std::fmt; use std::io::{self, BufWriter, Cursor, Seek, SeekFrom, Write}; @@ -21,13 +22,13 @@ use std::sync::{Arc, RwLock}; /// struct VecWriter { path: PathBuf, - shared_directory: InnerDirectory, + shared_directory: RAMDirectory, data: Cursor>, is_flushed: bool, } impl VecWriter { - fn new(path_buf: PathBuf, shared_directory: InnerDirectory) -> VecWriter { + fn new(path_buf: PathBuf, shared_directory: RAMDirectory) -> VecWriter { VecWriter { path: path_buf, data: Cursor::new(Vec::new()), @@ -63,74 +64,44 @@ impl Write for VecWriter { fn flush(&mut self) -> io::Result<()> { self.is_flushed = true; - self.shared_directory - .write(self.path.clone(), self.data.get_ref())?; + let mut fs = self.shared_directory.fs.write().unwrap(); + fs.write(self.path.clone(), self.data.get_ref()); Ok(()) } } -#[derive(Clone)] -struct InnerDirectory(Arc>>); +#[derive(Default)] +struct InnerDirectory { + fs: HashMap, + watch_router: WatchCallbackList, +} impl InnerDirectory { - fn new() -> InnerDirectory { - InnerDirectory(Arc::new(RwLock::new(HashMap::new()))) - } - - fn write(&self, path: PathBuf, data: &[u8]) -> io::Result { - let mut map = self.0.write().map_err(|_| { - make_io_err(format!( - "Failed to lock the directory, when trying to write {:?}", - path - )) - })?; - let prev_value = map.insert(path, ReadOnlySource::new(Vec::from(data))); - Ok(prev_value.is_some()) + fn write(&mut self, path: PathBuf, data: &[u8]) -> bool { + let data = ReadOnlySource::new(Vec::from(data)); + self.fs.insert(path, data).is_some() } fn open_read(&self, path: &Path) -> Result { - self.0 - .read() - .map_err(|_| { - let msg = format!( - "Failed to acquire read lock for the \ - directory when trying to read {:?}", - path - ); - let io_err = make_io_err(msg); - OpenReadError::IOError(IOError::with_path(path.to_owned(), io_err)) - }) - .and_then(|readable_map| { - readable_map - .get(path) - .ok_or_else(|| OpenReadError::FileDoesNotExist(PathBuf::from(path))) - .map(|el| el.clone()) - }) + self.fs + .get(path) + .ok_or_else(|| OpenReadError::FileDoesNotExist(PathBuf::from(path))) + .map(|el| el.clone()) } - fn delete(&self, path: &Path) -> result::Result<(), DeleteError> { - self.0 - .write() - .map_err(|_| { - let msg = format!( - "Failed to acquire write lock for the \ - directory when trying to delete {:?}", - path - ); - let io_err = make_io_err(msg); - DeleteError::IOError(IOError::with_path(path.to_owned(), io_err)) - }) - .and_then(|mut writable_map| match writable_map.remove(path) { - Some(_) => Ok(()), - None => Err(DeleteError::FileDoesNotExist(PathBuf::from(path))), - }) + fn delete(&mut self, path: &Path) -> result::Result<(), DeleteError> { + match self.fs.remove(path) { + Some(_) => Ok(()), + None => Err(DeleteError::FileDoesNotExist(PathBuf::from(path))), + } } fn exists(&self, path: &Path) -> bool { - self.0 - .read() - .expect("Failed to get read lock directory.") - .contains_key(path) + self.fs.contains_key(path) + } + + fn watch(&mut self, watch_handle: WatchCallback) -> WatchHandle { + self.watch_router.subscribe(watch_handle) } } @@ -145,33 +116,36 @@ impl fmt::Debug for RAMDirectory { /// It is mainly meant for unit testing. /// Writes are only made visible upon flushing. /// -#[derive(Clone)] +#[derive(Clone, Default)] pub struct RAMDirectory { - fs: InnerDirectory, + fs: Arc>, } impl RAMDirectory { /// Constructor pub fn create() -> RAMDirectory { - RAMDirectory { - fs: InnerDirectory::new(), - } + Self::default() } } impl Directory for RAMDirectory { fn open_read(&self, path: &Path) -> result::Result { - self.fs.open_read(path) + self.fs.read().unwrap().open_read(path) + } + + fn delete(&self, path: &Path) -> result::Result<(), DeleteError> { + self.fs.write().unwrap().delete(path) + } + + fn exists(&self, path: &Path) -> bool { + self.fs.read().unwrap().exists(path) } fn open_write(&mut self, path: &Path) -> Result { + let mut fs = self.fs.write().unwrap(); let path_buf = PathBuf::from(path); - let vec_writer = VecWriter::new(path_buf.clone(), self.fs.clone()); - - let exists = self - .fs - .write(path_buf.clone(), &Vec::new()) - .map_err(|err| IOError::with_path(path.to_owned(), err))?; + let vec_writer = VecWriter::new(path_buf.clone(), self.clone()); + let exists = fs.write(path_buf.clone(), &[]); // force the creation of the file to mimic the MMap directory. if exists { Err(OpenWriteError::FileAlreadyExists(path_buf)) @@ -180,17 +154,8 @@ impl Directory for RAMDirectory { } } - fn delete(&self, path: &Path) -> result::Result<(), DeleteError> { - self.fs.delete(path) - } - - fn exists(&self, path: &Path) -> bool { - self.fs.exists(path) - } - fn atomic_read(&self, path: &Path) -> Result, OpenReadError> { - let read = self.open_read(path)?; - Ok(read.as_slice().to_owned()) + Ok(self.open_read(path)?.as_slice().to_owned()) } fn atomic_write(&mut self, path: &Path, data: &[u8]) -> io::Result<()> { @@ -199,10 +164,20 @@ impl Directory for RAMDirectory { msg.unwrap_or("Undefined".to_string()) ))); let path_buf = PathBuf::from(path); - let mut vec_writer = VecWriter::new(path_buf.clone(), self.fs.clone()); - self.fs.write(path_buf, &Vec::new())?; + + // Reserve the path to prevent calls to .write() to succeed. + self.fs.write().unwrap().write(path_buf.clone(), &[]); + + let mut vec_writer = VecWriter::new(path_buf.clone(), self.clone()); vec_writer.write_all(data)?; vec_writer.flush()?; + if path == Path::new(&*META_FILEPATH) { + self.fs.write().unwrap().watch_router.broadcast(); + } Ok(()) } + + fn watch(&self, watch_callback: WatchCallback) -> WatchHandle { + self.fs.write().unwrap().watch(watch_callback) + } } diff --git a/src/directory/tests.rs b/src/directory/tests.rs index 11bedfeb5..13cb34ede 100644 --- a/src/directory/tests.rs +++ b/src/directory/tests.rs @@ -1,7 +1,13 @@ use super::*; use std::io::{Seek, SeekFrom, Write}; +use std::mem; use std::path::{Path, PathBuf}; +use std::sync::atomic::AtomicUsize; +use std::sync::atomic::Ordering; +use std::sync::Arc; +use std::thread; use std::time; +use std::time::Duration; lazy_static! { static ref TEST_PATH: &'static Path = Path::new("some_path_for_test"); @@ -30,19 +36,18 @@ fn ram_directory_panics_if_flush_forgotten() { fn test_simple(directory: &mut Directory) { { - { - let mut write_file = directory.open_write(*TEST_PATH).unwrap(); - assert!(directory.exists(*TEST_PATH)); - write_file.write_all(&[4]).unwrap(); - write_file.write_all(&[3]).unwrap(); - write_file.write_all(&[7, 3, 5]).unwrap(); - write_file.flush().unwrap(); - } + let mut write_file = directory.open_write(*TEST_PATH).unwrap(); + assert!(directory.exists(*TEST_PATH)); + write_file.write_all(&[4]).unwrap(); + write_file.write_all(&[3]).unwrap(); + write_file.write_all(&[7, 3, 5]).unwrap(); + write_file.flush().unwrap(); + } + { let read_file = directory.open_read(*TEST_PATH).unwrap(); let data: &[u8] = &*read_file; assert_eq!(data, &[4u8, 3u8, 7u8, 3u8, 5u8]); } - assert!(directory.delete(*TEST_PATH).is_ok()); assert!(!directory.exists(*TEST_PATH)); } @@ -121,6 +126,41 @@ fn test_directory(directory: &mut Directory) { test_directory_delete(directory); test_lock_non_blocking(directory); test_lock_blocking(directory); + test_watch(directory); +} + +fn test_watch(directory: &mut Directory) { + let counter: Arc = Default::default(); + let counter_clone = counter.clone(); + let watch_callback = Box::new(move || { + counter_clone.fetch_add(1, Ordering::SeqCst); + }); + assert!(directory + .atomic_write(Path::new("meta.json"), b"random_test_data") + .is_ok()); + thread::sleep(Duration::new(0, 10_000)); + assert_eq!(0, counter.load(Ordering::SeqCst)); + + let watch_handle = directory.watch(watch_callback); + for i in 0..10 { + assert_eq!(i, counter.load(Ordering::SeqCst)); + assert!(directory + .atomic_write(Path::new("meta.json"), b"random_test_data_2") + .is_ok()); + for _ in 0..100 { + if counter.load(Ordering::SeqCst) > i { + break; + } + thread::sleep(Duration::from_millis(10)); + } + assert_eq!(i + 1, counter.load(Ordering::SeqCst)); + } + mem::drop(watch_handle); + assert!(directory + .atomic_write(Path::new("meta.json"), b"random_test_data") + .is_ok()); + thread::sleep(Duration::from_millis(200)); + assert_eq!(10, counter.load(Ordering::SeqCst)); } fn test_lock_non_blocking(directory: &mut Directory) { diff --git a/src/directory/watch_event_router.rs b/src/directory/watch_event_router.rs new file mode 100644 index 000000000..820c73a11 --- /dev/null +++ b/src/directory/watch_event_router.rs @@ -0,0 +1,156 @@ +use std::sync::Arc; +use std::sync::RwLock; +use std::sync::Weak; + +/// Type alias for callbacks registered when watching files of a `Directory`. +pub type WatchCallback = Box () + Sync + Send>; + +/// Helper struct to implement the watch method in `Directory` implementations. +/// +/// It registers callbacks (See `.subscribe(...)`) and +/// calls them upon calls to `.broadcast(...)`. +#[derive(Default)] +pub struct WatchCallbackList { + router: RwLock>>, +} + +/// Controls how long a directory should watch for a file change. +/// +/// After all the clones of `WatchHandle` are dropped, the associated will not be called when a +/// file change is detected. +#[must_use = "This `WatchHandle` controls the lifetime of the watch and should therefore be used."] +#[derive(Clone)] +pub struct WatchHandle(Arc); + +impl WatchCallbackList { + /// Suscribes a new callback and returns a handle that controls the lifetime of the callback. + pub fn subscribe(&self, watch_callback: WatchCallback) -> WatchHandle { + let watch_callback_arc = Arc::new(watch_callback); + let watch_callback_weak = Arc::downgrade(&watch_callback_arc); + self.router.write().unwrap().push(watch_callback_weak); + WatchHandle(watch_callback_arc) + } + + fn list_callback(&self) -> Vec> { + let mut callbacks = vec![]; + let mut router_wlock = self.router.write().unwrap(); + let mut i = 0; + while i < router_wlock.len() { + if let Some(watch) = router_wlock[i].upgrade() { + callbacks.push(watch); + i += 1; + } else { + router_wlock.swap_remove(i); + } + } + callbacks + } + + /// Triggers all callbacks + pub fn broadcast(&self) { + let callbacks = self.list_callback(); + let spawn_res = std::thread::Builder::new() + .name("watch-callbacks".to_string()) + .spawn(move || { + for callback in callbacks { + callback(); + } + }); + if let Err(err) = spawn_res { + error!( + "Failed to spawn thread to call watch callbacks. Cause: {:?}", + err + ); + } + } +} + +#[cfg(test)] +mod tests { + use directory::WatchCallbackList; + use std::mem; + use std::sync::atomic::{AtomicUsize, Ordering}; + use std::sync::Arc; + use std::thread; + use std::time::Duration; + + const WAIT_TIME: u64 = 20; + + #[test] + fn test_watch_event_router_simple() { + let watch_event_router = WatchCallbackList::default(); + let counter: Arc = Default::default(); + let counter_clone = counter.clone(); + let inc_callback = Box::new(move || { + counter_clone.fetch_add(1, Ordering::SeqCst); + }); + watch_event_router.broadcast(); + assert_eq!(0, counter.load(Ordering::SeqCst)); + let handle_a = watch_event_router.subscribe(inc_callback); + thread::sleep(Duration::from_millis(WAIT_TIME)); + assert_eq!(0, counter.load(Ordering::SeqCst)); + watch_event_router.broadcast(); + thread::sleep(Duration::from_millis(WAIT_TIME)); + assert_eq!(1, counter.load(Ordering::SeqCst)); + watch_event_router.broadcast(); + watch_event_router.broadcast(); + watch_event_router.broadcast(); + thread::sleep(Duration::from_millis(WAIT_TIME)); + assert_eq!(4, counter.load(Ordering::SeqCst)); + mem::drop(handle_a); + watch_event_router.broadcast(); + thread::sleep(Duration::from_millis(WAIT_TIME)); + assert_eq!(4, counter.load(Ordering::SeqCst)); + } + + #[test] + fn test_watch_event_router_multiple_callback_same_key() { + let watch_event_router = WatchCallbackList::default(); + let counter: Arc = Default::default(); + let inc_callback = |inc: usize| { + let counter_clone = counter.clone(); + Box::new(move || { + counter_clone.fetch_add(inc, Ordering::SeqCst); + }) + }; + let handle_a = watch_event_router.subscribe(inc_callback(1)); + let handle_a2 = watch_event_router.subscribe(inc_callback(10)); + thread::sleep(Duration::from_millis(WAIT_TIME)); + assert_eq!(0, counter.load(Ordering::SeqCst)); + watch_event_router.broadcast(); + watch_event_router.broadcast(); + thread::sleep(Duration::from_millis(WAIT_TIME)); + assert_eq!(22, counter.load(Ordering::SeqCst)); + mem::drop(handle_a); + watch_event_router.broadcast(); + thread::sleep(Duration::from_millis(WAIT_TIME)); + assert_eq!(32, counter.load(Ordering::SeqCst)); + mem::drop(handle_a2); + watch_event_router.broadcast(); + watch_event_router.broadcast(); + thread::sleep(Duration::from_millis(WAIT_TIME)); + assert_eq!(32, counter.load(Ordering::SeqCst)); + } + + #[test] + fn test_watch_event_router_multiple_callback_different_key() { + let watch_event_router = WatchCallbackList::default(); + let counter: Arc = Default::default(); + let counter_clone = counter.clone(); + let inc_callback = Box::new(move || { + counter_clone.fetch_add(1, Ordering::SeqCst); + }); + let handle_a = watch_event_router.subscribe(inc_callback); + assert_eq!(0, counter.load(Ordering::SeqCst)); + watch_event_router.broadcast(); + watch_event_router.broadcast(); + thread::sleep(Duration::from_millis(WAIT_TIME)); + assert_eq!(2, counter.load(Ordering::SeqCst)); + thread::sleep(Duration::from_millis(WAIT_TIME)); + mem::drop(handle_a); + watch_event_router.broadcast(); + thread::sleep(Duration::from_millis(WAIT_TIME)); + assert_eq!(2, counter.load(Ordering::SeqCst)); + } + +} diff --git a/src/error.rs b/src/error.rs index 7c8bb25e7..5da069105 100644 --- a/src/error.rs +++ b/src/error.rs @@ -162,6 +162,7 @@ impl From for TantivyError { OpenDirectoryError::NotADirectory(directory_path) => { TantivyError::InvalidArgument(format!("{:?} is not a directory", directory_path)) } + OpenDirectoryError::IoError(err) => TantivyError::IOError(IOError::from(err)), } } } diff --git a/src/fastfield/bytes/mod.rs b/src/fastfield/bytes/mod.rs index 1a551ecc0..b3e73a590 100644 --- a/src/fastfield/bytes/mod.rs +++ b/src/fastfield/bytes/mod.rs @@ -22,9 +22,7 @@ mod tests { index_writer.add_document(doc!(field=>vec![1u8, 3, 5, 7, 9])); index_writer.add_document(doc!(field=>vec![0u8; 1000])); assert!(index_writer.commit().is_ok()); - - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let reader = searcher.segment_reader(0); let bytes_reader = reader.bytes_fast_field_reader(field).unwrap(); diff --git a/src/fastfield/multivalued/mod.rs b/src/fastfield/multivalued/mod.rs index b8e288f52..3e2a30eb2 100644 --- a/src/fastfield/multivalued/mod.rs +++ b/src/fastfield/multivalued/mod.rs @@ -9,14 +9,14 @@ mod tests { extern crate time; - use query::QueryParser; + use self::time::Duration; use collector::TopDocs; + use query::QueryParser; use schema::Cardinality; use schema::Facet; use schema::IntOptions; use schema::Schema; use Index; - use self::time::Duration; #[test] fn test_multivalued_u64() { @@ -34,11 +34,12 @@ mod tests { index_writer.add_document(doc!(field=>5u64, field=>20u64,field=>1u64)); assert!(index_writer.commit().is_ok()); - index.load_searchers().unwrap(); - let searcher = index.searcher(); - let reader = searcher.segment_reader(0); + let searcher = index.reader().unwrap().searcher(); + let segment_reader = searcher.segment_reader(0); let mut vals = Vec::new(); - let multi_value_reader = reader.multi_fast_field_reader::(field).unwrap(); + let multi_value_reader = segment_reader + .multi_fast_field_reader::(field) + .unwrap(); { multi_value_reader.get_vals(2, &mut vals); assert_eq!(&vals, &[4u64]); @@ -63,92 +64,121 @@ mod tests { .set_indexed() .set_stored(), ); - let time_i = schema_builder.add_i64_field( - "time_stamp_i", - IntOptions::default() - .set_stored(), - ); + let time_i = + schema_builder.add_i64_field("time_stamp_i", IntOptions::default().set_stored()); let schema = schema_builder.build(); let index = Index::create_in_ram(schema); let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); let first_time_stamp = chrono::Utc::now(); - index_writer.add_document(doc!(date_field=>first_time_stamp, date_field=>first_time_stamp, time_i=>1i64)); + index_writer.add_document( + doc!(date_field=>first_time_stamp, date_field=>first_time_stamp, time_i=>1i64), + ); index_writer.add_document(doc!(time_i=>0i64)); // add one second - index_writer.add_document(doc!(date_field=>first_time_stamp + Duration::seconds(1), time_i=>2i64)); + index_writer + .add_document(doc!(date_field=>first_time_stamp + Duration::seconds(1), time_i=>2i64)); // add another second let two_secs_ahead = first_time_stamp + Duration::seconds(2); index_writer.add_document(doc!(date_field=>two_secs_ahead, date_field=>two_secs_ahead,date_field=>two_secs_ahead, time_i=>3i64)); assert!(index_writer.commit().is_ok()); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let reader = searcher.segment_reader(0); assert_eq!(reader.num_docs(), 4); { let parser = QueryParser::for_index(&index, vec![date_field]); - let query = parser.parse_query(&format!("\"{}\"", first_time_stamp.to_rfc3339()).to_string()) + let query = parser + .parse_query(&format!("\"{}\"", first_time_stamp.to_rfc3339()).to_string()) .expect("could not parse query"); - let results = searcher.search(&query, &TopDocs::with_limit(5)) + let results = searcher + .search(&query, &TopDocs::with_limit(5)) .expect("could not query index"); assert_eq!(results.len(), 1); for (_score, doc_address) in results { let retrieved_doc = searcher.doc(doc_address).expect("cannot fetch doc"); - assert_eq!(retrieved_doc.get_first(date_field).expect("cannot find value").date_value().timestamp(), first_time_stamp.timestamp()); - assert_eq!(retrieved_doc.get_first(time_i).expect("cannot find value").i64_value(), 1i64); + assert_eq!( + retrieved_doc + .get_first(date_field) + .expect("cannot find value") + .date_value() + .timestamp(), + first_time_stamp.timestamp() + ); + assert_eq!( + retrieved_doc + .get_first(time_i) + .expect("cannot find value") + .i64_value(), + 1i64 + ); } } { let parser = QueryParser::for_index(&index, vec![date_field]); - let query = parser.parse_query(&format!("\"{}\"", two_secs_ahead.to_rfc3339()).to_string()) + let query = parser + .parse_query(&format!("\"{}\"", two_secs_ahead.to_rfc3339()).to_string()) .expect("could not parse query"); - let results = searcher.search(&query, &TopDocs::with_limit(5)) + let results = searcher + .search(&query, &TopDocs::with_limit(5)) .expect("could not query index"); assert_eq!(results.len(), 1); for (_score, doc_address) in results { let retrieved_doc = searcher.doc(doc_address).expect("cannot fetch doc"); - assert_eq!(retrieved_doc.get_first(date_field).expect("cannot find value").date_value().timestamp(), two_secs_ahead.timestamp()); - assert_eq!(retrieved_doc.get_first(time_i).expect("cannot find value").i64_value(), 3i64); + assert_eq!( + retrieved_doc + .get_first(date_field) + .expect("cannot find value") + .date_value() + .timestamp(), + two_secs_ahead.timestamp() + ); + assert_eq!( + retrieved_doc + .get_first(time_i) + .expect("cannot find value") + .i64_value(), + 3i64 + ); } } - // TODO: support Date range queries -// { -// let parser = QueryParser::for_index(&index, vec![date_field]); -// let range_q = format!("\"{}\"..\"{}\"", -// (first_time_stamp + Duration::seconds(1)).to_rfc3339(), -// (first_time_stamp + Duration::seconds(3)).to_rfc3339() -// ); -// let query = parser.parse_query(&range_q) -// .expect("could not parse query"); -// let results = searcher.search(&query, &TopDocs::with_limit(5)) -// .expect("could not query index"); -// -// -// assert_eq!(results.len(), 2); -// for (i, doc_pair) in results.iter().enumerate() { -// let retrieved_doc = searcher.doc(doc_pair.1).expect("cannot fetch doc"); -// let offset_sec = match i { -// 0 => 1, -// 1 => 3, -// _ => panic!("should not have more than 2 docs") -// }; -// let time_i_val = match i { -// 0 => 2, -// 1 => 3, -// _ => panic!("should not have more than 2 docs") -// }; -// assert_eq!(retrieved_doc.get_first(date_field).expect("cannot find value").date_value().timestamp(), -// (first_time_stamp + Duration::seconds(offset_sec)).timestamp()); -// assert_eq!(retrieved_doc.get_first(time_i).expect("cannot find value").i64_value(), time_i_val); -// } -// } + // { + // let parser = QueryParser::for_index(&index, vec![date_field]); + // let range_q = format!("\"{}\"..\"{}\"", + // (first_time_stamp + Duration::seconds(1)).to_rfc3339(), + // (first_time_stamp + Duration::seconds(3)).to_rfc3339() + // ); + // let query = parser.parse_query(&range_q) + // .expect("could not parse query"); + // let results = searcher.search(&query, &TopDocs::with_limit(5)) + // .expect("could not query index"); + // + // + // assert_eq!(results.len(), 2); + // for (i, doc_pair) in results.iter().enumerate() { + // let retrieved_doc = searcher.doc(doc_pair.1).expect("cannot fetch doc"); + // let offset_sec = match i { + // 0 => 1, + // 1 => 3, + // _ => panic!("should not have more than 2 docs") + // }; + // let time_i_val = match i { + // 0 => 2, + // 1 => 3, + // _ => panic!("should not have more than 2 docs") + // }; + // assert_eq!(retrieved_doc.get_first(date_field).expect("cannot find value").date_value().timestamp(), + // (first_time_stamp + Duration::seconds(offset_sec)).timestamp()); + // assert_eq!(retrieved_doc.get_first(time_i).expect("cannot find value").i64_value(), time_i_val); + // } + // } } #[test] @@ -167,8 +197,7 @@ mod tests { index_writer.add_document(doc!(field=> -5i64, field => -20i64, field=>1i64)); assert!(index_writer.commit().is_ok()); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let reader = searcher.segment_reader(0); let mut vals = Vec::new(); let multi_value_reader = reader.multi_fast_field_reader::(field).unwrap(); diff --git a/src/fastfield/multivalued/reader.rs b/src/fastfield/multivalued/reader.rs index d7df15a37..3456de525 100644 --- a/src/fastfield/multivalued/reader.rs +++ b/src/fastfield/multivalued/reader.rs @@ -75,8 +75,7 @@ mod tests { index_writer.add_document(doc); } index_writer.commit().expect("Commit failed"); - index.load_searchers().expect("Reloading searchers"); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let segment_reader = searcher.segment_reader(0); let mut facet_reader = segment_reader.facet_reader(facet_field).unwrap(); diff --git a/src/functional_test.rs b/src/functional_test.rs index 493f73c31..ff36369ea 100644 --- a/src/functional_test.rs +++ b/src/functional_test.rs @@ -22,6 +22,7 @@ fn test_indexing() { let schema = schema_builder.build(); let index = Index::create_from_tempdir(schema).unwrap(); + let reader = index.reader().unwrap(); let mut rng = thread_rng(); @@ -36,8 +37,8 @@ fn test_indexing() { index_writer.commit().expect("Commit failed"); committed_docs.extend(&uncommitted_docs); uncommitted_docs.clear(); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + reader.reload().unwrap(); + let searcher = reader.searcher(); // check that everything is correct. check_index_content(&searcher, &committed_docs); } else { diff --git a/src/indexer/index_writer.rs b/src/indexer/index_writer.rs index 44c9010b6..94961f5fc 100644 --- a/src/indexer/index_writer.rs +++ b/src/indexer/index_writer.rs @@ -44,8 +44,8 @@ pub const HEAP_SIZE_MAX: usize = u32::max_value() as usize - MARGIN_IN_BYTES; // reaches `PIPELINE_MAX_SIZE_IN_DOCS` const PIPELINE_MAX_SIZE_IN_DOCS: usize = 10_000; -type DocumentSender = channel::Sender>; -type DocumentReceiver = channel::Receiver>; +type OperationSender = channel::Sender>; +type OperationReceiver = channel::Receiver>; /// Split the thread memory budget into /// - the heap size @@ -85,8 +85,8 @@ pub struct IndexWriter { workers_join_handle: Vec>>, - document_receiver: DocumentReceiver, - document_sender: DocumentSender, + operation_receiver: OperationReceiver, + operation_sender: OperationSender, segment_updater: SegmentUpdater, @@ -133,7 +133,7 @@ pub fn open_index_writer( let err_msg = format!("The heap size per thread cannot exceed {}", HEAP_SIZE_MAX); return Err(TantivyError::InvalidArgument(err_msg)); } - let (document_sender, document_receiver): (DocumentSender, DocumentReceiver) = + let (document_sender, document_receiver): (OperationSender, OperationReceiver) = channel::bounded(PIPELINE_MAX_SIZE_IN_DOCS); let delete_queue = DeleteQueue::new(); @@ -151,8 +151,8 @@ pub fn open_index_writer( heap_size_in_bytes_per_thread, index: index.clone(), - document_receiver, - document_sender, + operation_receiver: document_receiver, + operation_sender: document_sender, segment_updater, @@ -335,7 +335,7 @@ impl IndexWriter { pub fn wait_merging_threads(mut self) -> Result<()> { // this will stop the indexing thread, // dropping the last reference to the segment_updater. - drop(self.document_sender); + drop(self.operation_sender); let former_workers_handles = mem::replace(&mut self.workers_join_handle, vec![]); for join_handle in former_workers_handles { @@ -384,7 +384,7 @@ impl IndexWriter { /// The thread consumes documents from the pipeline. /// fn add_indexing_worker(&mut self) -> Result<()> { - let document_receiver_clone = self.document_receiver.clone(); + let document_receiver_clone = self.operation_receiver.clone(); let mut segment_updater = self.segment_updater.clone(); let generation = self.generation; @@ -479,11 +479,11 @@ impl IndexWriter { /// when no documents are remaining. /// /// Returns the former segment_ready channel. - fn recreate_document_channel(&mut self) -> DocumentReceiver { - let (document_sender, document_receiver): (DocumentSender, DocumentReceiver) = + fn recreate_document_channel(&mut self) -> OperationReceiver { + let (document_sender, document_receiver): (OperationSender, OperationReceiver) = channel::bounded(PIPELINE_MAX_SIZE_IN_DOCS); - mem::replace(&mut self.document_sender, document_sender); - mem::replace(&mut self.document_receiver, document_receiver) + mem::replace(&mut self.operation_sender, document_sender); + mem::replace(&mut self.operation_receiver, document_receiver) } /// Rollback to the last commit @@ -501,7 +501,7 @@ impl IndexWriter { // segment updates will be ignored. self.segment_updater.kill(); - let document_receiver = self.document_receiver.clone(); + let document_receiver = self.operation_receiver.clone(); // take the directory lock to create a new index_writer. let directory_lock = self @@ -648,7 +648,7 @@ impl IndexWriter { pub fn add_document(&mut self, document: Document) -> u64 { let opstamp = self.stamper.stamp(); let add_operation = AddOperation { opstamp, document }; - let send_result = self.document_sender.send(vec![add_operation]); + let send_result = self.operation_sender.send(vec![add_operation]); if let Err(e) = send_result { panic!("Failed to index document. Sending to indexing channel failed. This probably means all of the indexing threads have panicked. {:?}", e); } @@ -709,7 +709,7 @@ impl IndexWriter { } } } - let send_result = self.document_sender.send(adds); + let send_result = self.operation_sender.send(adds); if let Err(e) = send_result { panic!("Failed to index document. Sending to indexing channel failed. This probably means all of the indexing threads have panicked. {:?}", e); }; @@ -728,8 +728,9 @@ mod tests { use error::*; use indexer::NoMergePolicy; use query::TermQuery; - use schema::{self, Document, IndexRecordOption}; + use schema::{self, IndexRecordOption}; use Index; + use ReloadPolicy; use Term; #[test] @@ -757,6 +758,11 @@ mod tests { let mut schema_builder = schema::Schema::builder(); let text_field = schema_builder.add_text_field("text", schema::TEXT); let index = Index::create_in_ram(schema_builder.build()); + let reader = index + .reader_builder() + .reload_policy(ReloadPolicy::Manual) + .try_into() + .unwrap(); let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); let a_term = Term::from_field_text(text_field, "a"); let b_term = Term::from_field_text(text_field, "b"); @@ -769,7 +775,7 @@ mod tests { index_writer.run(operations); index_writer.commit().expect("failed to commit"); - index.load_searchers().expect("failed to load searchers"); + reader.reload().expect("failed to load searchers"); let a_term = Term::from_field_text(text_field, "a"); let b_term = Term::from_field_text(text_field, "b"); @@ -777,7 +783,7 @@ mod tests { let a_query = TermQuery::new(a_term, IndexRecordOption::Basic); let b_query = TermQuery::new(b_term, IndexRecordOption::Basic); - let searcher = index.searcher(); + let searcher = reader.searcher(); let a_docs = searcher .search(&a_query, &TopDocs::with_limit(1)) @@ -864,9 +870,13 @@ mod tests { let mut schema_builder = schema::Schema::builder(); let text_field = schema_builder.add_text_field("text", schema::TEXT); let index = Index::create_in_ram(schema_builder.build()); - + let reader = index + .reader_builder() + .reload_policy(ReloadPolicy::Manual) + .try_into() + .unwrap(); let num_docs_containing = |s: &str| { - let searcher = index.searcher(); + let searcher = reader.searcher(); let term = Term::from_field_text(text_field, s); searcher.doc_freq(&term) }; @@ -876,7 +886,6 @@ mod tests { let mut index_writer = index.writer(3_000_000).unwrap(); index_writer.add_document(doc!(text_field=>"a")); index_writer.rollback().unwrap(); - assert_eq!(index_writer.commit_opstamp(), 0u64); assert_eq!(num_docs_containing("a"), 0); { @@ -884,13 +893,13 @@ mod tests { index_writer.add_document(doc!(text_field=>"c")); } assert!(index_writer.commit().is_ok()); - index.load_searchers().unwrap(); + reader.reload().unwrap(); assert_eq!(num_docs_containing("a"), 0); assert_eq!(num_docs_containing("b"), 1); assert_eq!(num_docs_containing("c"), 1); } - index.load_searchers().unwrap(); - index.searcher(); + reader.reload().unwrap(); + reader.searcher(); } #[test] @@ -898,32 +907,33 @@ mod tests { let mut schema_builder = schema::Schema::builder(); let text_field = schema_builder.add_text_field("text", schema::TEXT); let index = Index::create_in_ram(schema_builder.build()); + let reader = index + .reader_builder() + .reload_policy(ReloadPolicy::Manual) + .try_into() + .unwrap(); let num_docs_containing = |s: &str| { - let searcher = index.searcher(); let term_a = Term::from_field_text(text_field, s); - searcher.doc_freq(&term_a) + reader.searcher().doc_freq(&term_a) }; { // writing the segment let mut index_writer = index.writer(12_000_000).unwrap(); // create 8 segments with 100 tiny docs for _doc in 0..100 { - let mut doc = Document::default(); - doc.add_text(text_field, "a"); - index_writer.add_document(doc); + index_writer.add_document(doc!(text_field=>"a")); } index_writer.commit().expect("commit failed"); for _doc in 0..100 { - let mut doc = Document::default(); - doc.add_text(text_field, "a"); - index_writer.add_document(doc); + index_writer.add_document(doc!(text_field=>"a")); } - // this should create 8 segments and trigger a merge. + // this should create 8 segments and trigger a merge. index_writer.commit().expect("commit failed"); index_writer .wait_merging_threads() .expect("waiting merging thread failed"); - index.load_searchers().unwrap(); + + reader.reload().unwrap(); assert_eq!(num_docs_containing("a"), 200); assert!(index.searchable_segments().unwrap().len() < 8); @@ -990,11 +1000,15 @@ mod tests { } index_writer.commit().unwrap(); } - index.load_searchers().unwrap(); let num_docs_containing = |s: &str| { - let searcher = index.searcher(); let term_a = Term::from_field_text(text_field, s); - searcher.doc_freq(&term_a) + index + .reader_builder() + .reload_policy(ReloadPolicy::Manual) + .try_into() + .unwrap() + .searcher() + .doc_freq(&term_a) }; assert_eq!(num_docs_containing("a"), 0); assert_eq!(num_docs_containing("b"), 100); @@ -1026,11 +1040,9 @@ mod tests { index_writer.add_document(doc!(text_field => "b")); } assert!(index_writer.commit().is_err()); - index.load_searchers().unwrap(); let num_docs_containing = |s: &str| { - let searcher = index.searcher(); let term_a = Term::from_field_text(text_field, s); - searcher.doc_freq(&term_a) + index.reader().unwrap().searcher().doc_freq(&term_a) }; assert_eq!(num_docs_containing("a"), 100); assert_eq!(num_docs_containing("b"), 0); diff --git a/src/indexer/merger.rs b/src/indexer/merger.rs index 37c07b7a6..97048546b 100644 --- a/src/indexer/merger.rs +++ b/src/indexer/merger.rs @@ -194,17 +194,17 @@ impl IndexMerger { fast_field_serializer, )?; } - FieldType::U64(ref options) | FieldType::I64(ref options) | FieldType::Date(ref options) => { - match options.get_fastfield_cardinality() { - Some(Cardinality::SingleValue) => { - self.write_single_fast_field(field, fast_field_serializer)?; - } - Some(Cardinality::MultiValues) => { - self.write_multi_fast_field(field, fast_field_serializer)?; - } - None => {} + FieldType::U64(ref options) + | FieldType::I64(ref options) + | FieldType::Date(ref options) => match options.get_fastfield_cardinality() { + Some(Cardinality::SingleValue) => { + self.write_single_fast_field(field, fast_field_serializer)?; } - } + Some(Cardinality::MultiValues) => { + self.write_multi_fast_field(field, fast_field_serializer)?; + } + None => {} + }, FieldType::Str(_) => { // We don't handle str fast field for the moment // They can be implemented using what is done @@ -676,8 +676,8 @@ mod tests { let score_field = schema_builder.add_u64_field("score", score_fieldtype); let bytes_score_field = schema_builder.add_bytes_field("score_bytes"); let index = Index::create_in_ram(schema_builder.build()); + let reader = index.reader().unwrap(); let curr_time = chrono::Utc::now(); - let add_score_bytes = |doc: &mut Document, score: u32| { let mut bytes = Vec::new(); bytes @@ -748,8 +748,8 @@ mod tests { index_writer.wait_merging_threads().unwrap(); } { - index.load_searchers().unwrap(); - let searcher = index.searcher(); + reader.reload().unwrap(); + let searcher = reader.searcher(); let get_doc_ids = |terms: Vec| { let query = BooleanQuery::new_multiterms_query(terms); let top_docs = searcher.search(&query, &TestCollector).unwrap(); @@ -780,10 +780,7 @@ mod tests { ); assert_eq!( get_doc_ids(vec![Term::from_field_date(date_field, &curr_time)]), - vec![ - DocAddress(0, 0), - DocAddress(0, 3) - ] + vec![DocAddress(0, 0), DocAddress(0, 3)] ); } { @@ -848,7 +845,7 @@ mod tests { let bytes_score_field = schema_builder.add_bytes_field("score_bytes"); let index = Index::create_in_ram(schema_builder.build()); let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); - + let reader = index.reader().unwrap(); let search_term = |searcher: &Searcher, term: Term| { let collector = FastFieldTestCollector::for_field(score_field); let bytes_collector = BytesFastFieldTestCollector::for_field(bytes_score_field); @@ -885,8 +882,8 @@ mod tests { bytes_score_field => vec![0u8, 0, 0, 3], )); index_writer.commit().expect("committed"); - index.load_searchers().unwrap(); - let ref searcher = *index.searcher(); + reader.reload().unwrap(); + let searcher = reader.searcher(); assert_eq!(searcher.num_docs(), 2); assert_eq!(searcher.segment_readers()[0].num_docs(), 2); assert_eq!(searcher.segment_readers()[0].max_doc(), 3); @@ -932,8 +929,8 @@ mod tests { bytes_score_field => vec![0u8, 0, 27, 88], )); index_writer.commit().expect("committed"); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + reader.reload().unwrap(); + let searcher = reader.searcher(); assert_eq!(searcher.segment_readers().len(), 2); assert_eq!(searcher.num_docs(), 3); @@ -994,8 +991,8 @@ mod tests { .expect("Failed to initiate merge") .wait() .expect("Merging failed"); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + reader.reload().unwrap(); + let searcher = reader.searcher(); assert_eq!(searcher.segment_readers().len(), 1); assert_eq!(searcher.num_docs(), 3); assert_eq!(searcher.segment_readers()[0].num_docs(), 3); @@ -1040,8 +1037,8 @@ mod tests { index_writer.delete_term(Term::from_field_text(text_field, "c")); index_writer.commit().unwrap(); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + reader.reload().unwrap(); + let searcher = reader.searcher(); assert_eq!(searcher.segment_readers().len(), 1); assert_eq!(searcher.num_docs(), 2); assert_eq!(searcher.segment_readers()[0].num_docs(), 2); @@ -1091,9 +1088,9 @@ mod tests { .expect("Failed to initiate merge") .wait() .expect("Merging failed"); - index.load_searchers().unwrap(); + reader.reload().unwrap(); - let ref searcher = *index.searcher(); + let searcher = reader.searcher(); assert_eq!(searcher.segment_readers().len(), 1); assert_eq!(searcher.num_docs(), 2); assert_eq!(searcher.segment_readers()[0].num_docs(), 2); @@ -1141,9 +1138,9 @@ mod tests { let segment_ids = index .searchable_segment_ids() .expect("Searchable segments failed."); - index.load_searchers().unwrap(); + reader.reload().unwrap(); - let ref searcher = *index.searcher(); + let searcher = reader.searcher(); assert!(segment_ids.is_empty()); assert!(searcher.segment_readers().is_empty()); assert_eq!(searcher.num_docs(), 0); @@ -1155,6 +1152,7 @@ mod tests { let mut schema_builder = schema::Schema::builder(); let facet_field = schema_builder.add_facet_field("facet"); let index = Index::create_in_ram(schema_builder.build()); + let reader = index.reader().unwrap(); { let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); let index_doc = |index_writer: &mut IndexWriter, doc_facets: &[&str]| { @@ -1184,9 +1182,9 @@ mod tests { index_writer.commit().expect("committed"); } - index.load_searchers().unwrap(); + reader.reload().unwrap(); let test_searcher = |expected_num_docs: usize, expected: &[(&str, u64)]| { - let searcher = index.searcher(); + let searcher = reader.searcher(); let mut facet_collector = FacetCollector::for_field(facet_field); facet_collector.add_facet(Facet::from("/top")); let (count, facet_counts) = searcher @@ -1228,7 +1226,7 @@ mod tests { .wait() .expect("Merging failed"); index_writer.wait_merging_threads().unwrap(); - index.load_searchers().unwrap(); + reader.reload().unwrap(); test_searcher( 11, &[ @@ -1249,7 +1247,7 @@ mod tests { let facet_term = Term::from_facet(facet_field, &facet); index_writer.delete_term(facet_term); index_writer.commit().unwrap(); - index.load_searchers().unwrap(); + reader.reload().unwrap(); test_searcher( 9, &[ @@ -1274,8 +1272,8 @@ mod tests { index_writer.commit().expect("commit failed"); index_writer.add_document(doc!(int_field => 1u64)); index_writer.commit().expect("commit failed"); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); assert_eq!(searcher.num_docs(), 2); index_writer.delete_term(Term::from_field_u64(int_field, 1)); let segment_ids = index @@ -1286,10 +1284,10 @@ mod tests { .expect("Failed to initiate merge") .wait() .expect("Merging failed"); - index.load_searchers().unwrap(); + reader.reload().unwrap(); // commit has not been called yet. The document should still be // there. - assert_eq!(index.searcher().num_docs(), 2); + assert_eq!(reader.searcher().num_docs(), 2); } #[test] @@ -1300,7 +1298,7 @@ mod tests { .set_indexed(); let int_field = schema_builder.add_u64_field("intvals", int_options); let index = Index::create_in_ram(schema_builder.build()); - + let reader = index.reader().unwrap(); { let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); let mut doc = Document::default(); @@ -1321,8 +1319,8 @@ mod tests { .expect("Merging failed"); // assert delete has not been committed - index.load_searchers().unwrap(); - let searcher = index.searcher(); + reader.reload().expect("failed to load searcher 1"); + let searcher = reader.searcher(); assert_eq!(searcher.num_docs(), 2); index_writer.commit().unwrap(); @@ -1330,13 +1328,13 @@ mod tests { index_writer.wait_merging_threads().unwrap(); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + reader.reload().unwrap(); + let searcher = reader.searcher(); assert_eq!(searcher.num_docs(), 0); } #[test] - fn test_merge_multivalued_int_fields() { + fn test_merge_multivalued_int_fields_simple() { let mut schema_builder = schema::Schema::builder(); let int_options = IntOptions::default() .set_fast(Cardinality::MultiValues) @@ -1353,7 +1351,6 @@ mod tests { } index_writer.add_document(doc); }; - index_doc(&mut index_writer, &[1, 2]); index_doc(&mut index_writer, &[1, 2, 3]); index_doc(&mut index_writer, &[4, 5]); @@ -1362,19 +1359,14 @@ mod tests { index_doc(&mut index_writer, &[3]); index_doc(&mut index_writer, &[17]); index_writer.commit().expect("committed"); - index_doc(&mut index_writer, &[20]); index_writer.commit().expect("committed"); - index_doc(&mut index_writer, &[28, 27]); index_doc(&mut index_writer, &[1_000]); - index_writer.commit().expect("committed"); } - index.load_searchers().unwrap(); - - let searcher = index.searcher(); - + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let mut vals: Vec = Vec::new(); { @@ -1440,13 +1432,14 @@ mod tests { .expect("Failed to initiate merge") .wait() .expect("Merging failed"); - index_writer.wait_merging_threads().unwrap(); + index_writer + .wait_merging_threads() + .expect("Wait for merging threads"); } - - index.load_searchers().unwrap(); + reader.reload().expect("Load searcher"); { - let searcher = index.searcher(); + let searcher = reader.searcher(); println!( "{:?}", searcher diff --git a/src/indexer/segment_updater.rs b/src/indexer/segment_updater.rs index 2580bedda..3dc373795 100644 --- a/src/indexer/segment_updater.rs +++ b/src/indexer/segment_updater.rs @@ -565,9 +565,8 @@ mod tests { index_writer.delete_term(term); assert!(index_writer.commit().is_ok()); } - - index.load_searchers().unwrap(); - assert_eq!(index.searcher().num_docs(), 302); + let reader = index.reader().unwrap(); + assert_eq!(reader.searcher().num_docs(), 302); { index_writer @@ -575,9 +574,9 @@ mod tests { .expect("waiting for merging threads"); } - index.load_searchers().unwrap(); - assert_eq!(index.searcher().segment_readers().len(), 1); - assert_eq!(index.searcher().num_docs(), 302); + reader.reload().unwrap(); + assert_eq!(reader.searcher().segment_readers().len(), 1); + assert_eq!(reader.searcher().num_docs(), 302); } #[test] @@ -636,18 +635,18 @@ mod tests { .expect("waiting for merging threads"); } - index.load_searchers().unwrap(); - assert_eq!(index.searcher().num_docs(), 0); + let reader = index.reader().unwrap(); + assert_eq!(reader.searcher().num_docs(), 0); let seg_ids = index .searchable_segment_ids() .expect("Searchable segments failed."); assert!(seg_ids.is_empty()); - index.load_searchers().unwrap(); - assert_eq!(index.searcher().num_docs(), 0); + reader.reload().unwrap(); + assert_eq!(reader.searcher().num_docs(), 0); // empty segments should be erased assert!(index.searchable_segment_metas().unwrap().is_empty()); - assert!(index.searcher().segment_readers().is_empty()); + assert!(reader.searcher().segment_readers().is_empty()); } } diff --git a/src/lib.rs b/src/lib.rs index 6beb6f32d..70c9e78fd 100755 --- a/src/lib.rs +++ b/src/lib.rs @@ -75,9 +75,9 @@ //! //! // # Searching //! -//! index.load_searchers()?; +//! let reader = index.reader()?; //! -//! let searcher = index.searcher(); +//! let searcher = reader.searcher(); //! //! let query_parser = QueryParser::for_index(&index, vec![title, body]); //! @@ -186,8 +186,8 @@ pub use error::TantivyError; pub use error::TantivyError as Error; extern crate census; -extern crate owned_read; pub extern crate chrono; +extern crate owned_read; /// Tantivy result. pub type Result = std::result::Result; @@ -215,6 +215,9 @@ pub mod space_usage; pub mod store; pub mod termdict; +mod reader; + +pub use self::reader::{IndexReader, IndexReaderBuilder, ReloadPolicy}; mod snippet; pub use self::snippet::{Snippet, SnippetGenerator}; @@ -303,6 +306,7 @@ mod tests { use Index; use IndexWriter; use Postings; + use ReloadPolicy; pub fn assert_nearly_equals(expected: f32, val: f32) { assert!( @@ -391,8 +395,8 @@ mod tests { index_writer.commit().unwrap(); } { - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let term_a = Term::from_field_text(text_field, "a"); assert_eq!(searcher.doc_freq(&term_a), 3); let term_b = Term::from_field_text(text_field, "b"); @@ -419,8 +423,8 @@ mod tests { index_writer.commit().unwrap(); } { - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let index_reader = index.reader().unwrap(); + let searcher = index_reader.searcher(); let reader = searcher.segment_reader(0); { let fieldnorm_reader = reader.get_fieldnorms_reader(text_field); @@ -455,8 +459,8 @@ mod tests { index_writer.commit().unwrap(); } { - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let segment_reader: &SegmentReader = searcher.segment_reader(0); let fieldnorms_reader = segment_reader.get_fieldnorms_reader(text_field); assert_eq!(fieldnorms_reader.fieldnorm(0), 3); @@ -484,6 +488,11 @@ mod tests { let term_c = Term::from_field_text(text_field, "c"); let schema = schema_builder.build(); let index = Index::create_in_ram(schema); + let reader = index + .reader_builder() + .reload_policy(ReloadPolicy::Manual) + .try_into() + .unwrap(); { // writing the segment let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); @@ -505,10 +514,10 @@ mod tests { index_writer.commit().unwrap(); } { - index.load_searchers().unwrap(); - let searcher = index.searcher(); - let reader = searcher.segment_reader(0); - let inverted_index = reader.inverted_index(text_field); + reader.reload().unwrap(); + let searcher = reader.searcher(); + let segment_reader = searcher.segment_reader(0); + let inverted_index = segment_reader.inverted_index(text_field); assert!(inverted_index .read_postings(&term_abcd, IndexRecordOption::WithFreqsAndPositions) .is_none()); @@ -516,19 +525,19 @@ mod tests { let mut postings = inverted_index .read_postings(&term_a, IndexRecordOption::WithFreqsAndPositions) .unwrap(); - assert!(advance_undeleted(&mut postings, reader)); + assert!(advance_undeleted(&mut postings, segment_reader)); assert_eq!(postings.doc(), 5); - assert!(!advance_undeleted(&mut postings, reader)); + assert!(!advance_undeleted(&mut postings, segment_reader)); } { let mut postings = inverted_index .read_postings(&term_b, IndexRecordOption::WithFreqsAndPositions) .unwrap(); - assert!(advance_undeleted(&mut postings, reader)); + assert!(advance_undeleted(&mut postings, segment_reader)); assert_eq!(postings.doc(), 3); - assert!(advance_undeleted(&mut postings, reader)); + assert!(advance_undeleted(&mut postings, segment_reader)); assert_eq!(postings.doc(), 4); - assert!(!advance_undeleted(&mut postings, reader)); + assert!(!advance_undeleted(&mut postings, segment_reader)); } } { @@ -541,10 +550,10 @@ mod tests { index_writer.rollback().unwrap(); } { - index.load_searchers().unwrap(); - let searcher = index.searcher(); - let reader = searcher.segment_reader(0); - let inverted_index = reader.inverted_index(term_abcd.field()); + reader.reload().unwrap(); + let searcher = reader.searcher(); + let seg_reader = searcher.segment_reader(0); + let inverted_index = seg_reader.inverted_index(term_abcd.field()); assert!(inverted_index .read_postings(&term_abcd, IndexRecordOption::WithFreqsAndPositions) @@ -553,19 +562,19 @@ mod tests { let mut postings = inverted_index .read_postings(&term_a, IndexRecordOption::WithFreqsAndPositions) .unwrap(); - assert!(advance_undeleted(&mut postings, reader)); + assert!(advance_undeleted(&mut postings, seg_reader)); assert_eq!(postings.doc(), 5); - assert!(!advance_undeleted(&mut postings, reader)); + assert!(!advance_undeleted(&mut postings, seg_reader)); } { let mut postings = inverted_index .read_postings(&term_b, IndexRecordOption::WithFreqsAndPositions) .unwrap(); - assert!(advance_undeleted(&mut postings, reader)); + assert!(advance_undeleted(&mut postings, seg_reader)); assert_eq!(postings.doc(), 3); - assert!(advance_undeleted(&mut postings, reader)); + assert!(advance_undeleted(&mut postings, seg_reader)); assert_eq!(postings.doc(), 4); - assert!(!advance_undeleted(&mut postings, reader)); + assert!(!advance_undeleted(&mut postings, seg_reader)); } } { @@ -578,10 +587,10 @@ mod tests { index_writer.commit().unwrap(); } { - index.load_searchers().unwrap(); - let searcher = index.searcher(); - let reader = searcher.segment_reader(0); - let inverted_index = reader.inverted_index(term_abcd.field()); + reader.reload().unwrap(); + let searcher = reader.searcher(); + let segment_reader = searcher.segment_reader(0); + let inverted_index = segment_reader.inverted_index(term_abcd.field()); assert!(inverted_index .read_postings(&term_abcd, IndexRecordOption::WithFreqsAndPositions) .is_none()); @@ -589,25 +598,25 @@ mod tests { let mut postings = inverted_index .read_postings(&term_a, IndexRecordOption::WithFreqsAndPositions) .unwrap(); - assert!(!advance_undeleted(&mut postings, reader)); + assert!(!advance_undeleted(&mut postings, segment_reader)); } { let mut postings = inverted_index .read_postings(&term_b, IndexRecordOption::WithFreqsAndPositions) .unwrap(); - assert!(advance_undeleted(&mut postings, reader)); + assert!(advance_undeleted(&mut postings, segment_reader)); assert_eq!(postings.doc(), 3); - assert!(advance_undeleted(&mut postings, reader)); + assert!(advance_undeleted(&mut postings, segment_reader)); assert_eq!(postings.doc(), 4); - assert!(!advance_undeleted(&mut postings, reader)); + assert!(!advance_undeleted(&mut postings, segment_reader)); } { let mut postings = inverted_index .read_postings(&term_c, IndexRecordOption::WithFreqsAndPositions) .unwrap(); - assert!(advance_undeleted(&mut postings, reader)); + assert!(advance_undeleted(&mut postings, segment_reader)); assert_eq!(postings.doc(), 4); - assert!(!advance_undeleted(&mut postings, reader)); + assert!(!advance_undeleted(&mut postings, segment_reader)); } } } @@ -622,8 +631,8 @@ mod tests { let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); index_writer.add_document(doc!(field=>1u64)); index_writer.commit().unwrap(); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let term = Term::from_field_u64(field, 1u64); let mut postings = searcher .segment_reader(0) @@ -646,8 +655,8 @@ mod tests { let negative_val = -1i64; index_writer.add_document(doc!(value_field => negative_val)); index_writer.commit().unwrap(); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let term = Term::from_field_i64(value_field, negative_val); let mut postings = searcher .segment_reader(0) @@ -669,8 +678,8 @@ mod tests { let mut index_writer = index.writer_with_num_threads(2, 6_000_000).unwrap(); index_writer.add_document(doc!(text_field=>"a")); assert!(index_writer.commit().is_ok()); - assert!(index.load_searchers().is_ok()); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let segment_reader = searcher.segment_reader(0); segment_reader.inverted_index(absent_field); //< should not panic } @@ -681,6 +690,11 @@ mod tests { let text_field = schema_builder.add_text_field("text", TEXT); let schema = schema_builder.build(); let index = Index::create_in_ram(schema); + let reader = index + .reader_builder() + .reload_policy(ReloadPolicy::Manual) + .try_into() + .unwrap(); // writing the segment let mut index_writer = index.writer_with_num_threads(2, 6_000_000).unwrap(); @@ -706,8 +720,8 @@ mod tests { remove_document(&mut index_writer, "38"); remove_document(&mut index_writer, "34"); index_writer.commit().unwrap(); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + reader.reload().unwrap(); + let searcher = reader.searcher(); assert_eq!(searcher.num_docs(), 6); } @@ -727,8 +741,8 @@ mod tests { index_writer.commit().unwrap(); } { - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let index_reader = index.reader().unwrap(); + let searcher = index_reader.searcher(); let reader = searcher.segment_reader(0); let inverted_index = reader.inverted_index(text_field); let term_abcd = Term::from_field_text(text_field, "abcd"); @@ -752,7 +766,7 @@ mod tests { let text_field = schema_builder.add_text_field("text", TEXT); let schema = schema_builder.build(); let index = Index::create_in_ram(schema); - + let reader = index.reader().unwrap(); { // writing the segment let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); @@ -762,8 +776,8 @@ mod tests { index_writer.commit().unwrap(); } { - index.load_searchers().unwrap(); - let searcher = index.searcher(); + reader.reload().unwrap(); + let searcher = reader.searcher(); let get_doc_ids = |terms: Vec| { let query = BooleanQuery::new_multiterms_query(terms); let topdocs = searcher.search(&query, &TestCollector).unwrap(); @@ -805,25 +819,22 @@ mod tests { let text_field = schema_builder.add_text_field("text", TEXT); let schema = schema_builder.build(); let index = Index::create_in_ram(schema); - + let reader = index + .reader_builder() + .reload_policy(ReloadPolicy::Manual) + .try_into() + .unwrap(); + assert_eq!(reader.searcher().num_docs(), 0u64); { // writing the segment let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); - { - let doc = doc!(text_field=>"af b"); - index_writer.add_document(doc); - } - { - let doc = doc!(text_field=>"a b c"); - index_writer.add_document(doc); - } - { - let doc = doc!(text_field=>"a b c d"); - index_writer.add_document(doc); - } + index_writer.add_document(doc!(text_field=>"af b")); + index_writer.add_document(doc!(text_field=>"a b c")); + index_writer.add_document(doc!(text_field=>"a b c d")); index_writer.commit().unwrap(); } - index.searcher(); + reader.reload().unwrap(); + assert_eq!(reader.searcher().num_docs(), 3u64); } #[test] @@ -860,9 +871,8 @@ mod tests { index_writer.add_document(document); index_writer.commit().unwrap(); } - - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let segment_reader: &SegmentReader = searcher.segment_reader(0); { let fast_field_reader_res = segment_reader.fast_field_reader::(text_field); diff --git a/src/postings/mod.rs b/src/postings/mod.rs index e134e7a9e..e571e26df 100644 --- a/src/postings/mod.rs +++ b/src/postings/mod.rs @@ -100,9 +100,8 @@ pub mod tests { } index_writer.add_document(doc!(title => r#"abc be be be be abc"#)); index_writer.commit().unwrap(); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let inverted_index = searcher.segment_reader(0u32).inverted_index(title); let term = Term::from_field_text(title, "abc"); @@ -292,9 +291,8 @@ pub mod tests { } assert!(index_writer.commit().is_ok()); } - index.load_searchers().unwrap(); let term_a = Term::from_field_text(text_field, "a"); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let segment_reader = searcher.segment_reader(0); let mut postings = segment_reader .inverted_index(text_field) @@ -331,10 +329,9 @@ pub mod tests { } assert!(index_writer.commit().is_ok()); } - index.load_searchers().unwrap(); index }; - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let segment_reader = searcher.segment_reader(0); // check that the basic usage works @@ -402,8 +399,7 @@ pub mod tests { index_writer.delete_term(term_0); assert!(index_writer.commit().is_ok()); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let segment_reader = searcher.segment_reader(0); // make sure seeking still works @@ -450,12 +446,9 @@ pub mod tests { { let mut index_writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); index_writer.delete_term(term_1); - assert!(index_writer.commit().is_ok()); } - index.load_searchers().unwrap(); - - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); // finally, check that it's empty { @@ -511,7 +504,6 @@ pub mod tests { } assert!(index_writer.commit().is_ok()); } - index.load_searchers().unwrap(); index }; } diff --git a/src/postings/postings_writer.rs b/src/postings/postings_writer.rs index 97de4cd10..61bcc3537 100644 --- a/src/postings/postings_writer.rs +++ b/src/postings/postings_writer.rs @@ -33,9 +33,10 @@ fn posting_from_field_entry(field_entry: &FieldEntry) -> Box { } }) .unwrap_or_else(|| SpecializedPostingsWriter::::new_boxed()), - FieldType::U64(_) | FieldType::I64(_) | FieldType::Date(_) | FieldType::HierarchicalFacet => { - SpecializedPostingsWriter::::new_boxed() - } + FieldType::U64(_) + | FieldType::I64(_) + | FieldType::Date(_) + | FieldType::HierarchicalFacet => SpecializedPostingsWriter::::new_boxed(), FieldType::Bytes => { // FieldType::Bytes cannot actually be indexed. // TODO fix during the indexer refactoring described in #276 diff --git a/src/postings/segment_postings.rs b/src/postings/segment_postings.rs index af26258f4..807265348 100644 --- a/src/postings/segment_postings.rs +++ b/src/postings/segment_postings.rs @@ -773,8 +773,7 @@ mod tests { last_doc = doc + 1; } index_writer.commit().unwrap(); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let segment_reader = searcher.segment_reader(0); let inverted_index = segment_reader.inverted_index(int_field); let term = Term::from_field_u64(int_field, 0u64); @@ -842,8 +841,7 @@ mod tests { index_writer.add_document(doc); } index_writer.commit().unwrap(); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let segment_reader = searcher.segment_reader(0); let mut block_segments; diff --git a/src/query/all_query.rs b/src/query/all_query.rs index e6468e2d7..8b2b64233 100644 --- a/src/query/all_query.rs +++ b/src/query/all_query.rs @@ -101,8 +101,9 @@ mod tests { index_writer.commit().unwrap(); index_writer.add_document(doc!(field=>"ccc")); index_writer.commit().unwrap(); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + reader.reload().unwrap(); + let searcher = reader.searcher(); let weight = AllQuery.weight(&searcher, false).unwrap(); { let reader = searcher.segment_reader(0); diff --git a/src/query/boolean_query/mod.rs b/src/query/boolean_query/mod.rs index b450f7f0a..16f37de26 100644 --- a/src/query/boolean_query/mod.rs +++ b/src/query/boolean_query/mod.rs @@ -51,7 +51,6 @@ mod tests { } assert!(index_writer.commit().is_ok()); } - index.load_searchers().unwrap(); (index, text_field) } @@ -60,7 +59,8 @@ mod tests { let (index, text_field) = aux_test_helper(); let query_parser = QueryParser::for_index(&index, vec![text_field]); let query = query_parser.parse_query("(+a +b) d").unwrap(); - assert_eq!(query.count(&*index.searcher()).unwrap(), 3); + let searcher = index.reader().unwrap().searcher(); + assert_eq!(query.count(&searcher).unwrap(), 3); } #[test] @@ -68,7 +68,7 @@ mod tests { let (index, text_field) = aux_test_helper(); let query_parser = QueryParser::for_index(&index, vec![text_field]); let query = query_parser.parse_query("+a").unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let weight = query.weight(&searcher, true).unwrap(); let scorer = weight.scorer(searcher.segment_reader(0u32)).unwrap(); assert!(scorer.is::()); @@ -78,7 +78,7 @@ mod tests { pub fn test_boolean_termonly_intersection() { let (index, text_field) = aux_test_helper(); let query_parser = QueryParser::for_index(&index, vec![text_field]); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); { let query = query_parser.parse_query("+a +b +c").unwrap(); let weight = query.weight(&searcher, true).unwrap(); @@ -97,7 +97,7 @@ mod tests { pub fn test_boolean_reqopt() { let (index, text_field) = aux_test_helper(); let query_parser = QueryParser::for_index(&index, vec![text_field]); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); { let query = query_parser.parse_query("+a b").unwrap(); let weight = query.weight(&searcher, true).unwrap(); @@ -126,10 +126,13 @@ mod tests { query }; + let reader = index.reader().unwrap(); + let matching_docs = |boolean_query: &Query| { - let searcher = index.searcher(); - let test_docs = searcher.search(boolean_query, &TestCollector).unwrap(); - test_docs + reader + .searcher() + .search(boolean_query, &TestCollector) + .unwrap() .docs() .iter() .cloned() @@ -185,10 +188,12 @@ mod tests { let query: Box = Box::new(term_query); query }; - + let reader = index.reader().unwrap(); let score_docs = |boolean_query: &Query| { - let searcher = index.searcher(); - let fruit = searcher.search(boolean_query, &TestCollector).unwrap(); + let fruit = reader + .searcher() + .search(boolean_query, &TestCollector) + .unwrap(); fruit.scores().to_vec() }; diff --git a/src/query/fuzzy_query.rs b/src/query/fuzzy_query.rs index 6539929bf..ea54b789b 100644 --- a/src/query/fuzzy_query.rs +++ b/src/query/fuzzy_query.rs @@ -52,9 +52,8 @@ lazy_static! { /// )); /// index_writer.commit().unwrap(); /// } -/// -/// index.load_searchers()?; -/// let searcher = index.searcher(); +/// let reader = index.reader()?; +/// let searcher = reader.searcher(); /// /// { /// @@ -141,8 +140,8 @@ mod test { )); index_writer.commit().unwrap(); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); { let term = Term::from_field_text(country_field, "japon"); diff --git a/src/query/phrase_query/mod.rs b/src/query/phrase_query/mod.rs index 90ae26451..a08e505a4 100644 --- a/src/query/phrase_query/mod.rs +++ b/src/query/phrase_query/mod.rs @@ -31,7 +31,6 @@ mod tests { } assert!(index_writer.commit().is_ok()); } - index.load_searchers().unwrap(); index } @@ -46,8 +45,7 @@ mod tests { ]); let schema = index.schema(); let text_field = schema.get_field("text").unwrap(); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let test_query = |texts: Vec<&str>| { let terms: Vec = texts .iter() @@ -90,8 +88,7 @@ mod tests { index_writer.add_document(doc!(text_field=>"a b c")); assert!(index_writer.commit().is_ok()); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let phrase_query = PhraseQuery::new(vec![ Term::from_field_text(text_field, "a"), Term::from_field_text(text_field, "b"), @@ -115,8 +112,7 @@ mod tests { let index = create_index(&["a b c", "a b c a b"]); let schema = index.schema(); let text_field = schema.get_field("text").unwrap(); - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let test_query = |texts: Vec<&str>| { let terms: Vec = texts .iter() @@ -148,8 +144,7 @@ mod tests { assert!(index_writer.commit().is_ok()); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let test_query = |texts: Vec<&str>| { let terms: Vec = texts .iter() @@ -177,8 +172,7 @@ mod tests { index_writer.add_document(doc!(text_field=>"a b c d e f g h")); assert!(index_writer.commit().is_ok()); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let test_query = |texts: Vec<(usize, &str)>| { let terms: Vec<(usize, Term)> = texts .iter() diff --git a/src/query/query_parser/query_parser.rs b/src/query/query_parser/query_parser.rs index 25b08ae1f..b6ac926fc 100644 --- a/src/query/query_parser/query_parser.rs +++ b/src/query/query_parser/query_parser.rs @@ -239,12 +239,13 @@ impl QueryParser { let term = Term::from_field_i64(field, val); Ok(vec![(0, term)]) } - FieldType::Date(_) => { - match chrono::DateTime::parse_from_rfc3339(phrase) { - Ok(x) => Ok(vec![(0, Term::from_field_date(field, &x.with_timezone(&chrono::Utc)))]), - Err(e) => Err(QueryParserError::DateFormatError(e)) - } - } + FieldType::Date(_) => match chrono::DateTime::parse_from_rfc3339(phrase) { + Ok(x) => Ok(vec![( + 0, + Term::from_field_date(field, &x.with_timezone(&chrono::Utc)), + )]), + Err(e) => Err(QueryParserError::DateFormatError(e)), + }, FieldType::U64(_) => { let val: u64 = u64::from_str(phrase)?; let term = Term::from_field_u64(field, val); @@ -791,7 +792,9 @@ mod test { query_parser.parse_query("date:18a"), Err(QueryParserError::DateFormatError(_)) ); - assert!(query_parser.parse_query("date:\"1985-04-12T23:20:50.52Z\"").is_ok()); + assert!(query_parser + .parse_query("date:\"1985-04-12T23:20:50.52Z\"") + .is_ok()); } #[test] diff --git a/src/query/range_query.rs b/src/query/range_query.rs index f111e90e1..3aa996520 100644 --- a/src/query/range_query.rs +++ b/src/query/range_query.rs @@ -61,8 +61,8 @@ fn map_bound TTo>( /// # } /// # index_writer.commit().unwrap(); /// # } -/// # index.load_searchers()?; -/// let searcher = index.searcher(); +/// # let reader = index.reader()?; +/// let searcher = reader.searcher(); /// /// let docs_in_the_sixties = RangeQuery::new_u64(year_field, 1960..1970); /// @@ -316,8 +316,8 @@ mod tests { } index_writer.commit().unwrap(); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let docs_in_the_sixties = RangeQuery::new_u64(year_field, 1960u64..1970u64); @@ -355,8 +355,8 @@ mod tests { index_writer.commit().unwrap(); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let count_multiples = |range_query: RangeQuery| searcher.search(&range_query, &Count).unwrap(); diff --git a/src/query/regex_query.rs b/src/query/regex_query.rs index 3d3254b2f..ec7dcceb7 100644 --- a/src/query/regex_query.rs +++ b/src/query/regex_query.rs @@ -44,8 +44,8 @@ use Searcher; /// index_writer.commit().unwrap(); /// } /// -/// index.load_searchers()?; -/// let searcher = index.searcher(); +/// let reader = index.reader()?; +/// let searcher = reader.searcher(); /// /// let term = Term::from_field_text(title, "Diary"); /// let query = RegexQuery::new("d[ai]{2}ry".to_string(), title); @@ -108,8 +108,8 @@ mod test { )); index_writer.commit().unwrap(); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); { let regex_query = RegexQuery::new("jap[ao]n".to_string(), country_field); let scored_docs = searcher diff --git a/src/query/term_query/mod.rs b/src/query/term_query/mod.rs index edc4af411..2a85e9383 100644 --- a/src/query/term_query/mod.rs +++ b/src/query/term_query/mod.rs @@ -32,9 +32,7 @@ mod tests { } assert!(index_writer.commit().is_ok()); } - - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let term_query = TermQuery::new( Term::from_field_text(text_field, "a"), IndexRecordOption::Basic, @@ -65,8 +63,7 @@ mod tests { index_writer.add_document(doc!(left_field => "left4 left1")); index_writer.commit().unwrap(); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); { let term = Term::from_field_text(left_field, "left2"); let term_query = TermQuery::new(term, IndexRecordOption::WithFreqs); diff --git a/src/query/term_query/term_query.rs b/src/query/term_query/term_query.rs index 8ddf42762..6dc52acb2 100644 --- a/src/query/term_query/term_query.rs +++ b/src/query/term_query/term_query.rs @@ -48,9 +48,8 @@ use Term; /// )); /// index_writer.commit()?; /// } -/// -/// index.load_searchers()?; -/// let searcher = index.searcher(); +/// let reader = index.reader()?; +/// let searcher = reader.searcher(); /// /// let query = TermQuery::new( /// Term::from_field_text(title, "diary"), diff --git a/src/reader/mod.rs b/src/reader/mod.rs new file mode 100644 index 000000000..67d0ba52c --- /dev/null +++ b/src/reader/mod.rs @@ -0,0 +1,187 @@ +mod pool; + +use self::pool::{LeasedItem, Pool}; +use core::Segment; +use directory::Directory; +use directory::WatchHandle; +use directory::META_LOCK; +use std::sync::Arc; +use Index; +use Result; +use Searcher; +use SegmentReader; + +/// Defines when a new version of the index should be reloaded. +/// +/// Regardless of whether you search and index in the same process, tantivy does not necessarily +/// reflects the change that are commited to your index. `ReloadPolicy` precisely helps you define +/// when you want your index to be reloaded. +#[derive(Clone, Copy)] +pub enum ReloadPolicy { + /// The index is entirely reloaded manually. + /// All updates of the index should be manual. + /// + /// No change is reflected automatically. You are required to call `.load_seacher()` manually. + Manual, + /// The index is reloaded within milliseconds after a new commit is available. + /// This is made possible by watching changes in the `meta.json` file. + OnCommit, // TODO add NEAR_REAL_TIME(target_ms) +} + +/// `IndexReader` builder +/// +/// It makes it possible to set the following values. +/// +/// - `num_searchers` (by default, the number of detected CPU threads): +/// +/// When `num_searchers` queries are requested at the same time, the `num_searchers` will block +/// until the one of the searcher in-use gets released. +/// - `reload_policy` (by default `ReloadPolicy::OnCommit`): +/// +/// See [`ReloadPolicy`](./enum.ReloadPolicy.html) for more details. +#[derive(Clone)] +pub struct IndexReaderBuilder { + num_searchers: usize, + reload_policy: ReloadPolicy, + index: Index, +} + +impl IndexReaderBuilder { + pub(crate) fn new(index: Index) -> IndexReaderBuilder { + IndexReaderBuilder { + num_searchers: num_cpus::get(), + reload_policy: ReloadPolicy::OnCommit, + index, + } + } + + /// Builds the reader. + /// + /// 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. + /// TODO(pmasurel) Use the `TryInto` trait once it is available in stable. + pub fn try_into(self) -> Result { + let inner_reader = InnerIndexReader { + index: self.index, + num_searchers: self.num_searchers, + searcher_pool: Pool::new(), + }; + inner_reader.reload()?; + let inner_reader_arc = Arc::new(inner_reader); + let watch_handle_opt: Option; + match self.reload_policy { + ReloadPolicy::Manual => { + // No need to set anything... + watch_handle_opt = None; + } + ReloadPolicy::OnCommit => { + let inner_reader_arc_clone = inner_reader_arc.clone(); + let callback = move || { + if let Err(err) = inner_reader_arc_clone.reload() { + error!( + "Error while loading searcher after commit was detected. {:?}", + err + ); + } + }; + let watch_handle = inner_reader_arc.index.directory().watch(Box::new(callback)); + watch_handle_opt = Some(watch_handle); + } + } + Ok(IndexReader { + inner: inner_reader_arc, + watch_handle_opt, + }) + } + + /// Sets the reload_policy. + /// + /// See [`ReloadPolicy`](./enum.ReloadPolicy.html) for more details. + pub fn reload_policy(mut self, reload_policy: ReloadPolicy) -> IndexReaderBuilder { + self.reload_policy = reload_policy; + self + } + + /// Sets the number of `Searcher` in the searcher pool. + pub fn num_searchers(mut self, num_searchers: usize) -> IndexReaderBuilder { + self.num_searchers = num_searchers; + self + } +} + +struct InnerIndexReader { + num_searchers: usize, + searcher_pool: Pool, + index: Index, +} + +impl InnerIndexReader { + fn reload(&self) -> Result<()> { + let segment_readers: Vec = { + let _meta_lock = self.index.directory().acquire_lock(&META_LOCK)?; + let searchable_segments = self.searchable_segments()?; + searchable_segments + .iter() + .map(SegmentReader::open) + .collect::>()? + }; + let schema = self.index.schema(); + let searchers = (0..self.num_searchers) + .map(|_| Searcher::new(schema.clone(), self.index.clone(), segment_readers.clone())) + .collect(); + self.searcher_pool.publish_new_generation(searchers); + Ok(()) + } + + /// Returns the list of segments that are searchable + fn searchable_segments(&self) -> Result> { + self.index.searchable_segments() + } + + fn searcher(&self) -> LeasedItem { + self.searcher_pool.acquire() + } +} + +/// `IndexReader` is your entry point to read and search the index. +/// +/// It controls when a new version of the index should be loaded and lends +/// you instances of `Searcher` for the last loaded version. +/// +/// `Clone` does not clone the different pool of searcher. `IndexReader` +/// just wraps and `Arc`. +#[derive(Clone)] +pub struct IndexReader { + inner: Arc, + watch_handle_opt: Option, +} + +impl IndexReader { + /// Update searchers so that they reflect the state of the last + /// `.commit()`. + /// + /// If you set up the `OnCommit` `ReloadPolicy` (which is the default) + /// every commit should be rapidly reflected on your `IndexReader` and you should + /// not need to call `reload()` at all. + /// + /// This automatic reload can take 10s of milliseconds to kick in however, and in unit tests + /// it can be nice to deterministically force the reload of searchers. + pub fn reload(&self) -> Result<()> { + self.inner.reload() + } + + /// Returns a searcher + /// + /// This method should be called every single time a search + /// query is performed. + /// The searchers are taken from a pool of `num_searchers` searchers. + /// If no searcher is available + /// this may block. + /// + /// The same searcher must be used for a given query, as it ensures + /// the use of a consistent segment set. + pub fn searcher(&self) -> LeasedItem { + self.inner.searcher() + } +} diff --git a/src/core/pool.rs b/src/reader/pool.rs similarity index 100% rename from src/core/pool.rs rename to src/reader/pool.rs diff --git a/src/schema/document.rs b/src/schema/document.rs index 2c535446a..bec54177a 100644 --- a/src/schema/document.rs +++ b/src/schema/document.rs @@ -90,7 +90,7 @@ impl Document { /// Add a date field pub fn add_date(&mut self, field: Field, value: &DateTime) { - self.add(FieldValue::new(field, Value::Date(DateTime::from(*value)))); + self.add(FieldValue::new(field, Value::Date(*value))); } /// Add a bytes field diff --git a/src/schema/field_entry.rs b/src/schema/field_entry.rs index fc5972b94..89a8b251d 100644 --- a/src/schema/field_entry.rs +++ b/src/schema/field_entry.rs @@ -87,7 +87,9 @@ impl FieldEntry { pub fn is_indexed(&self) -> bool { match self.field_type { FieldType::Str(ref options) => options.get_indexing_options().is_some(), - FieldType::U64(ref options) | FieldType::I64(ref options) | FieldType::Date(ref options) => options.is_indexed(), + FieldType::U64(ref options) + | FieldType::I64(ref options) + | FieldType::Date(ref options) => options.is_indexed(), FieldType::HierarchicalFacet => true, FieldType::Bytes => false, } @@ -104,7 +106,9 @@ impl FieldEntry { /// Returns true iff the field is stored pub fn is_stored(&self) -> bool { match self.field_type { - FieldType::U64(ref options) | FieldType::I64(ref options) | FieldType::Date(ref options) => options.is_stored(), + FieldType::U64(ref options) + | FieldType::I64(ref options) + | FieldType::Date(ref options) => options.is_stored(), FieldType::Str(ref options) => options.is_stored(), // TODO make stored hierarchical facet optional FieldType::HierarchicalFacet => true, diff --git a/src/schema/field_type.rs b/src/schema/field_type.rs index 79c2656a6..a67451d7a 100644 --- a/src/schema/field_type.rs +++ b/src/schema/field_type.rs @@ -95,7 +95,9 @@ impl FieldType { FieldType::Str(ref text_options) => text_options .get_indexing_options() .map(|indexing_options| indexing_options.index_option()), - FieldType::U64(ref int_options) | FieldType::I64(ref int_options) | FieldType::Date(ref int_options) => { + FieldType::U64(ref int_options) + | FieldType::I64(ref int_options) + | FieldType::Date(ref int_options) => { if int_options.is_indexed() { Some(IndexRecordOption::Basic) } else { @@ -116,9 +118,9 @@ impl FieldType { match *json { JsonValue::String(ref field_text) => match *self { FieldType::Str(_) => Ok(Value::Str(field_text.clone())), - FieldType::U64(_) | FieldType::I64(_) | FieldType::Date(_) => Err(ValueParsingError::TypeError( - format!("Expected an integer, got {:?}", json), - )), + FieldType::U64(_) | FieldType::I64(_) | FieldType::Date(_) => Err( + ValueParsingError::TypeError(format!("Expected an integer, got {:?}", json)), + ), FieldType::HierarchicalFacet => Ok(Value::Facet(Facet::from(field_text))), FieldType::Bytes => decode(field_text).map(Value::Bytes).map_err(|_| { ValueParsingError::InvalidBase64(format!( diff --git a/src/schema/flags.rs b/src/schema/flags.rs index e766b4c27..104df2b33 100644 --- a/src/schema/flags.rs +++ b/src/schema/flags.rs @@ -43,7 +43,11 @@ pub const FAST: SchemaFlagList = SchemaFlagList { }; impl BitOr> for SchemaFlagList - where Head: Clone, OldHead: Clone, OldTail: Clone { +where + Head: Clone, + OldHead: Clone, + OldTail: Clone, +{ type Output = SchemaFlagList>; fn bitor(self, head: SchemaFlagList) -> Self::Output { @@ -54,7 +58,7 @@ impl BitOr> for SchemaFlagList< } } -impl> BitOr for SchemaFlagList { +impl> BitOr for SchemaFlagList { type Output = IntOptions; fn bitor(self, rhs: IntOptions) -> Self::Output { @@ -62,7 +66,7 @@ impl> BitOr for SchemaFlagList { } } -impl> BitOr for SchemaFlagList { +impl> BitOr for SchemaFlagList { type Output = TextOptions; fn bitor(self, rhs: TextOptions) -> Self::Output { diff --git a/src/schema/int_options.rs b/src/schema/int_options.rs index b1599bd75..a95f236c3 100644 --- a/src/schema/int_options.rs +++ b/src/schema/int_options.rs @@ -1,4 +1,4 @@ -use schema::flags::{SchemaFlagList, FastFlag, IndexedFlag, StoredFlag}; +use schema::flags::{FastFlag, IndexedFlag, SchemaFlagList, StoredFlag}; use std::ops::BitOr; /// Express whether a field is single-value or multi-valued. diff --git a/src/schema/schema.rs b/src/schema/schema.rs index 994b71c9a..7d2be9ac9 100644 --- a/src/schema/schema.rs +++ b/src/schema/schema.rs @@ -97,7 +97,7 @@ impl SchemaBuilder { pub fn add_date_field>( &mut self, field_name_str: &str, - field_options: T + field_options: T, ) -> Field { let field_name = String::from(field_name_str); let field_entry = FieldEntry::new_date(field_name, field_options.into()); diff --git a/src/schema/value.rs b/src/schema/value.rs index 0090b4435..cb27c64b3 100644 --- a/src/schema/value.rs +++ b/src/schema/value.rs @@ -107,10 +107,10 @@ impl Value { } } - /// Returns the Date-value, provided the value is of the `Date` type. - /// - /// # Panics - /// If the value is not of type `Date` + /// Returns the Date-value, provided the value is of the `Date` type. + /// + /// # Panics + /// If the value is not of type `Date` pub fn date_value(&self) -> &DateTime { match *self { Value::Date(ref value) => value, @@ -138,7 +138,9 @@ impl From for Value { } impl From for Value { - fn from(date_time: DateTime) -> Value { Value::Date(date_time) } + fn from(date_time: DateTime) -> Value { + Value::Date(date_time) + } } impl<'a> From<&'a str> for Value { @@ -161,10 +163,10 @@ impl From> for Value { mod binary_serialize { use super::Value; + use chrono::{TimeZone, Utc}; use common::BinarySerializable; use schema::Facet; use std::io::{self, Read, Write}; - use chrono::{Utc, TimeZone}; const TEXT_CODE: u8 = 0; const U64_CODE: u8 = 1; @@ -217,7 +219,7 @@ mod binary_serialize { let value = i64::deserialize(reader)?; Ok(Value::I64(value)) } - DATE_CODE=> { + DATE_CODE => { let timestamp = i64::deserialize(reader)?; Ok(Value::Date(Utc.timestamp(timestamp, 0))) } diff --git a/src/snippet/mod.rs b/src/snippet/mod.rs index 8a3895acb..3fa605b1c 100644 --- a/src/snippet/mod.rs +++ b/src/snippet/mod.rs @@ -241,8 +241,8 @@ fn select_best_fragment_combination(fragments: &[FragmentCandidate], text: &str) /// # let query_parser = QueryParser::for_index(&index, vec![text_field]); /// // ... /// let query = query_parser.parse_query("haleurs flamands").unwrap(); -/// # index.load_searchers()?; -/// # let searcher = index.searcher(); +/// # let reader = index.reader()?; +/// # let searcher = reader.searcher(); /// let mut snippet_generator = SnippetGenerator::create(&searcher, &*query, text_field)?; /// snippet_generator.set_max_num_chars(100); /// let snippet = snippet_generator.snippet_from_doc(&doc); @@ -528,9 +528,8 @@ Survey in 2016, 2017, and 2018."#; index_writer.add_document(doc!(text_field => "a")); index_writer.add_document(doc!(text_field => "a b")); index_writer.commit().unwrap(); - index.load_searchers().unwrap(); } - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let query_parser = QueryParser::for_index(&index, vec![text_field]); { let query = query_parser.parse_query("e").unwrap(); @@ -587,8 +586,7 @@ Survey in 2016, 2017, and 2018."#; } index_writer.commit().unwrap(); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let query_parser = QueryParser::for_index(&index, vec![text_field]); let query = query_parser.parse_query("rust design").unwrap(); let mut snippet_generator = diff --git a/src/space_usage/mod.rs b/src/space_usage/mod.rs index f361d6860..8ffb841f0 100644 --- a/src/space_usage/mod.rs +++ b/src/space_usage/mod.rs @@ -304,9 +304,8 @@ mod test { fn test_empty() { let schema = Schema::builder().build(); let index = Index::create_in_ram(schema.clone()); - - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let searcher_space_usage = searcher.space_usage(); assert_eq!(0, searcher_space_usage.total()); } @@ -344,8 +343,8 @@ mod test { index_writer.commit().unwrap(); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let searcher_space_usage = searcher.space_usage(); assert!(searcher_space_usage.total() > 0); assert_eq!(1, searcher_space_usage.segments().len()); @@ -384,8 +383,8 @@ mod test { index_writer.commit().unwrap(); } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let searcher_space_usage = searcher.space_usage(); assert!(searcher_space_usage.total() > 0); assert_eq!(1, searcher_space_usage.segments().len()); @@ -423,9 +422,8 @@ mod test { index_writer.add_document(doc!(name => "hello hi goodbye")); index_writer.commit().unwrap(); } - - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let searcher_space_usage = searcher.space_usage(); assert!(searcher_space_usage.total() > 0); assert_eq!(1, searcher_space_usage.segments().len()); @@ -471,9 +469,8 @@ mod test { index_writer2.commit().unwrap(); } - index.load_searchers().unwrap(); - - let searcher = index.searcher(); + let reader = index.reader().unwrap(); + let searcher = reader.searcher(); let searcher_space_usage = searcher.space_usage(); assert!(searcher_space_usage.total() > 0); assert_eq!(1, searcher_space_usage.segments().len()); diff --git a/src/termdict/mod.rs b/src/termdict/mod.rs index f3157118c..89a98f072 100644 --- a/src/termdict/mod.rs +++ b/src/termdict/mod.rs @@ -159,8 +159,7 @@ mod tests { index_writer.commit().unwrap(); } } - index.load_searchers().unwrap(); - let searcher = index.searcher(); + let searcher = index.reader().unwrap().searcher(); let field_searcher = searcher.field(text_field); let mut term_it = field_searcher.terms(); From 5768d93171561667d270ca6b57877771cd268345 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Wed, 20 Mar 2019 08:54:19 +0900 Subject: [PATCH 18/26] Rename try to attempt as try is becoming a keyword in rust --- src/query/query_parser/query_grammar.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/query/query_parser/query_grammar.rs b/src/query/query_parser/query_grammar.rs index 3a3ac4256..2b6fda541 100644 --- a/src/query/query_parser/query_grammar.rs +++ b/src/query/query_parser/query_grammar.rs @@ -52,7 +52,7 @@ parser! { field_name: None, phrase, }); - try(term_query) + attempt(term_query) .or(term_default_field) .map(UserInputLeaf::from) } @@ -83,12 +83,12 @@ parser! { let lower_bound = { let excl = (char('{'), term_val()).map(|(_, w)| UserInputBound::Exclusive(w)); let incl = (char('['), term_val()).map(|(_, w)| UserInputBound::Inclusive(w)); - try(excl).or(incl) + attempt(excl).or(incl) }; let upper_bound = { let excl = (term_val(), char('}')).map(|(w, _)| UserInputBound::Exclusive(w)); let incl = (term_val(), char(']')).map(|(w, _)| UserInputBound::Inclusive(w)); - try(excl).or(incl) + attempt(excl).or(incl) }; ( optional((field(), char(':')).map(|x| x.0)), @@ -112,11 +112,11 @@ parser! { .or((char('+'), leaf()).map(|(_, expr)| expr.unary(Occur::Must) )) .or((char('('), parse_to_ast(), char(')')).map(|(_, expr, _)| expr)) .or(char('*').map(|_| UserInputAST::from(UserInputLeaf::All) )) - .or(try( + .or(attempt( (string("NOT"), spaces1(), leaf()).map(|(_, _, expr)| expr.unary(Occur::MustNot)) ) ) - .or(try( + .or(attempt( range().map(UserInputAST::from) ) ) @@ -160,7 +160,7 @@ parser! { where [I: Stream] { ( - try( + attempt( chainl1( leaf().map(Element::SingleEl), binary_operand().map(|op: BinaryOperand| From 22cf1004bdf8bd25bc35fbc6da8fa4260998fe4a Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Wed, 20 Mar 2019 08:54:52 +0900 Subject: [PATCH 19/26] Reenabled test on android --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index a2c77ef68..ea3d037ab 100644 --- a/.travis.yml +++ b/.travis.yml @@ -29,7 +29,7 @@ addons: matrix: include: # Android - - env: TARGET=aarch64-linux-android DISABLE_TESTS=1 + - env: TARGET=aarch64-linux-android #- env: TARGET=arm-linux-androideabi DISABLE_TESTS=1 #- env: TARGET=armv7-linux-androideabi DISABLE_TESTS=1 #- env: TARGET=i686-linux-android DISABLE_TESTS=1 From 6ea34b3d53ab6c5d537c7b2ffb076bb05074959d Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Wed, 20 Mar 2019 09:39:24 +0900 Subject: [PATCH 20/26] Fix version --- Cargo.toml | 2 +- README.md | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index 1e04cf4fa..f8bc05554 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "tantivy" -version = "0.9.0-dev" +version = "0.9.0" authors = ["Paul Masurel "] license = "MIT" categories = ["database-implementations", "data-structures"] diff --git a/README.md b/README.md index 06e4e8fa3..bd381f87f 100644 --- a/README.md +++ b/README.md @@ -30,6 +30,7 @@ Tantivy is, in fact, strongly inspired by Lucene's design. # Features - Full-text search +- Configurable tokenizer. (stemming available for 17 latin languages. Third party support for Chinese ([tantivy-jieba](https://crates.io/crates/tantivy-jieba) and [cang-jie](https://crates.io/crates/cang-jie)) and [Japanese](https://crates.io/crates/tantivy-tokenizer-tiny-segmenter) - Fast (check out the :racehorse: :sparkles: [benchmark](https://tantivy-search.github.io/bench/) :sparkles: :racehorse:) - Tiny startup time (<10ms), perfect for command line tools - BM25 scoring (the same as lucene) @@ -41,6 +42,7 @@ Tantivy is, in fact, strongly inspired by Lucene's design. - SIMD integer compression when the platform/CPU includes the SSE2 instruction set. - Single valued and multivalued u64 and i64 fast fields (equivalent of doc values in Lucene) - `&[u8]` fast fields +- Text, i64, u64, dates and hierarchical facet fields - LZ4 compressed document store - Range queries - Faceted search From ee6e273365b5cb3e4644a364f6230ea707b65f94 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Wed, 20 Mar 2019 10:01:30 +0900 Subject: [PATCH 21/26] cleanup for nodefaultfeatures --- src/common/mod.rs | 7 -- src/core/index.rs | 127 ++++++++++++----------- src/directory/managed_directory.rs | 159 +++++++++++++++-------------- src/directory/mmap_directory.rs | 7 +- src/functional_test.rs | 1 - src/lib.rs | 1 + 6 files changed, 157 insertions(+), 145 deletions(-) diff --git a/src/common/mod.rs b/src/common/mod.rs index 27228b057..6a1026e54 100644 --- a/src/common/mod.rs +++ b/src/common/mod.rs @@ -13,8 +13,6 @@ pub use self::serialize::{BinarySerializable, FixedSize}; pub use self::vint::{read_u32_vint, serialize_vint_u32, write_u32_vint, VInt}; pub use byteorder::LittleEndian as Endianness; -use std::io; - /// Computes the number of bits that will be used for bitpacking. /// /// In general the target is the minimum number of bits @@ -52,11 +50,6 @@ pub(crate) fn is_power_of_2(n: usize) -> bool { (n > 0) && (n & (n - 1) == 0) } -/// Create a default io error given a string. -pub(crate) fn make_io_err(msg: String) -> io::Error { - io::Error::new(io::ErrorKind::Other, msg) -} - /// Has length trait pub trait HasLen { /// Return length diff --git a/src/core/index.rs b/src/core/index.rs index 62a31cc95..5e8e08200 100644 --- a/src/core/index.rs +++ b/src/core/index.rs @@ -6,8 +6,6 @@ use core::SegmentId; use core::SegmentMeta; use core::META_FILEPATH; use directory::ManagedDirectory; -#[cfg(feature = "mmap")] -use directory::MmapDirectory; use directory::INDEX_WRITER_LOCK; use directory::{Directory, RAMDirectory}; use error::DataCorruption; @@ -24,12 +22,15 @@ use schema::Schema; use serde_json; use std::borrow::BorrowMut; use std::fmt; -use std::path::Path; use std::sync::Arc; use tokenizer::BoxedTokenizer; use tokenizer::TokenizerManager; use IndexWriter; use Result; +#[cfg(feature = "mmap")] +use std::path::Path; +#[cfg(feature = "mmap")] +use directory::MmapDirectory; fn load_metas(directory: &Directory) -> Result { let meta_data = directory.atomic_read(&META_FILEPATH)?; @@ -355,10 +356,8 @@ mod tests { use directory::RAMDirectory; use schema::Field; use schema::{Schema, INDEXED, TEXT}; - use std::path::PathBuf; use std::thread; use std::time::Duration; - use tempdir::TempDir; use Index; use IndexReader; use IndexWriter; @@ -444,62 +443,72 @@ mod tests { test_index_on_commit_reload_policy_aux(field, &mut writer, &reader); } - #[test] - fn test_index_on_commit_reload_policy_mmap() { - let schema = throw_away_schema(); - let field = schema.get_field("num_likes").unwrap(); - let tempdir = TempDir::new("index").unwrap(); - let tempdir_path = PathBuf::from(tempdir.path()); - let index = Index::create_in_dir(&tempdir_path, schema).unwrap(); - let mut writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); - writer.commit().unwrap(); - let reader = index - .reader_builder() - .reload_policy(ReloadPolicy::OnCommit) - .try_into() - .unwrap(); - assert_eq!(reader.searcher().num_docs(), 0); - test_index_on_commit_reload_policy_aux(field, &mut writer, &reader); + + #[cfg(feature="mmap")] + mod mmap_specific { + + use std::path::PathBuf; + use tempdir::TempDir; + use super::*; + + #[test] + fn test_index_on_commit_reload_policy_mmap() { + let schema = throw_away_schema(); + let field = schema.get_field("num_likes").unwrap(); + let tempdir = TempDir::new("index").unwrap(); + let tempdir_path = PathBuf::from(tempdir.path()); + let index = Index::create_in_dir(&tempdir_path, schema).unwrap(); + let mut writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); + writer.commit().unwrap(); + let reader = index + .reader_builder() + .reload_policy(ReloadPolicy::OnCommit) + .try_into() + .unwrap(); + assert_eq!(reader.searcher().num_docs(), 0); + test_index_on_commit_reload_policy_aux(field, &mut writer, &reader); + } + + #[test] + fn test_index_manual_policy_mmap() { + let schema = throw_away_schema(); + let field = schema.get_field("num_likes").unwrap(); + let index = Index::create_from_tempdir(schema).unwrap(); + let mut writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); + writer.commit().unwrap(); + let reader = index + .reader_builder() + .reload_policy(ReloadPolicy::Manual) + .try_into() + .unwrap(); + assert_eq!(reader.searcher().num_docs(), 0); + writer.add_document(doc!(field=>1u64)); + writer.commit().unwrap(); + thread::sleep(Duration::from_millis(500)); + assert_eq!(reader.searcher().num_docs(), 0); + reader.reload().unwrap(); + assert_eq!(reader.searcher().num_docs(), 1); + } + + #[test] + fn test_index_on_commit_reload_policy_different_directories() { + let schema = throw_away_schema(); + let field = schema.get_field("num_likes").unwrap(); + let tempdir = TempDir::new("index").unwrap(); + let tempdir_path = PathBuf::from(tempdir.path()); + let write_index = Index::create_in_dir(&tempdir_path, schema).unwrap(); + let read_index = Index::open_in_dir(&tempdir_path).unwrap(); + let reader = read_index + .reader_builder() + .reload_policy(ReloadPolicy::OnCommit) + .try_into() + .unwrap(); + assert_eq!(reader.searcher().num_docs(), 0); + let mut writer = write_index.writer_with_num_threads(1, 3_000_000).unwrap(); + test_index_on_commit_reload_policy_aux(field, &mut writer, &reader); + } } - #[test] - fn test_index_manual_policy_mmap() { - let schema = throw_away_schema(); - let field = schema.get_field("num_likes").unwrap(); - let index = Index::create_from_tempdir(schema).unwrap(); - let mut writer = index.writer_with_num_threads(1, 3_000_000).unwrap(); - writer.commit().unwrap(); - let reader = index - .reader_builder() - .reload_policy(ReloadPolicy::Manual) - .try_into() - .unwrap(); - assert_eq!(reader.searcher().num_docs(), 0); - writer.add_document(doc!(field=>1u64)); - writer.commit().unwrap(); - thread::sleep(Duration::from_millis(500)); - assert_eq!(reader.searcher().num_docs(), 0); - reader.reload().unwrap(); - assert_eq!(reader.searcher().num_docs(), 1); - } - - #[test] - fn test_index_on_commit_reload_policy_different_directories() { - let schema = throw_away_schema(); - let field = schema.get_field("num_likes").unwrap(); - let tempdir = TempDir::new("index").unwrap(); - let tempdir_path = PathBuf::from(tempdir.path()); - let write_index = Index::create_in_dir(&tempdir_path, schema).unwrap(); - let read_index = Index::open_in_dir(&tempdir_path).unwrap(); - let reader = read_index - .reader_builder() - .reload_policy(ReloadPolicy::OnCommit) - .try_into() - .unwrap(); - assert_eq!(reader.searcher().num_docs(), 0); - let mut writer = write_index.writer_with_num_threads(1, 3_000_000).unwrap(); - test_index_on_commit_reload_policy_aux(field, &mut writer, &reader); - } fn test_index_on_commit_reload_policy_aux( field: Field, diff --git a/src/directory/managed_directory.rs b/src/directory/managed_directory.rs index ddd8dfbba..b2f2bfc34 100644 --- a/src/directory/managed_directory.rs +++ b/src/directory/managed_directory.rs @@ -260,95 +260,100 @@ impl Clone for ManagedDirectory { #[cfg(test)] mod tests { - use super::*; #[cfg(feature = "mmap")] - use directory::MmapDirectory; - use std::io::Write; - use std::path::Path; - use tempdir::TempDir; + mod mmap_specific { - lazy_static! { - static ref TEST_PATH1: &'static Path = Path::new("some_path_for_test"); - static ref TEST_PATH2: &'static Path = Path::new("some_path_for_test2"); - } + use super::super::*; + use std::path::Path; + use tempdir::TempDir; - #[test] - #[cfg(feature = "mmap")] - fn test_managed_directory() { - let tempdir = TempDir::new("index").unwrap(); - let tempdir_path = PathBuf::from(tempdir.path()); - { - let mmap_directory = MmapDirectory::open(&tempdir_path).unwrap(); - let mut managed_directory = ManagedDirectory::wrap(mmap_directory).unwrap(); + + lazy_static! { + static ref TEST_PATH1: &'static Path = Path::new("some_path_for_test"); + static ref TEST_PATH2: &'static Path = Path::new("some_path_for_test2"); + } + + use directory::MmapDirectory; + use std::io::Write; + + #[test] + fn test_managed_directory() { + let tempdir = TempDir::new("index").unwrap(); + let tempdir_path = PathBuf::from(tempdir.path()); { - let mut write_file = managed_directory.open_write(*TEST_PATH1).unwrap(); - write_file.flush().unwrap(); + let mmap_directory = MmapDirectory::open(&tempdir_path).unwrap(); + let mut managed_directory = ManagedDirectory::wrap(mmap_directory).unwrap(); + { + let mut write_file = managed_directory.open_write(*TEST_PATH1).unwrap(); + write_file.flush().unwrap(); + } + { + managed_directory + .atomic_write(*TEST_PATH2, &vec![0u8, 1u8]) + .unwrap(); + } + { + assert!(managed_directory.exists(*TEST_PATH1)); + assert!(managed_directory.exists(*TEST_PATH2)); + } + { + let living_files: HashSet = + [TEST_PATH1.to_owned()].into_iter().cloned().collect(); + managed_directory.garbage_collect(|| living_files); + } + { + assert!(managed_directory.exists(*TEST_PATH1)); + assert!(!managed_directory.exists(*TEST_PATH2)); + } } { - managed_directory - .atomic_write(*TEST_PATH2, &vec![0u8, 1u8]) - .unwrap(); - } - { - assert!(managed_directory.exists(*TEST_PATH1)); - assert!(managed_directory.exists(*TEST_PATH2)); - } - { - let living_files: HashSet = - [TEST_PATH1.to_owned()].into_iter().cloned().collect(); - managed_directory.garbage_collect(|| living_files); - } - { - assert!(managed_directory.exists(*TEST_PATH1)); - assert!(!managed_directory.exists(*TEST_PATH2)); + let mmap_directory = MmapDirectory::open(&tempdir_path).unwrap(); + let mut managed_directory = ManagedDirectory::wrap(mmap_directory).unwrap(); + { + assert!(managed_directory.exists(*TEST_PATH1)); + assert!(!managed_directory.exists(*TEST_PATH2)); + } + { + let living_files: HashSet = HashSet::new(); + managed_directory.garbage_collect(|| living_files); + } + { + assert!(!managed_directory.exists(*TEST_PATH1)); + assert!(!managed_directory.exists(*TEST_PATH2)); + } } } - { + + #[test] + fn test_managed_directory_gc_while_mmapped() { + let tempdir = TempDir::new("index").unwrap(); + let tempdir_path = PathBuf::from(tempdir.path()); + let living_files = HashSet::new(); + let mmap_directory = MmapDirectory::open(&tempdir_path).unwrap(); let mut managed_directory = ManagedDirectory::wrap(mmap_directory).unwrap(); - { - assert!(managed_directory.exists(*TEST_PATH1)); - assert!(!managed_directory.exists(*TEST_PATH2)); - } - { - let living_files: HashSet = HashSet::new(); - managed_directory.garbage_collect(|| living_files); - } - { - assert!(!managed_directory.exists(*TEST_PATH1)); - assert!(!managed_directory.exists(*TEST_PATH2)); - } - } - } - - #[test] - #[cfg(feature = "mmap ")] - fn test_managed_directory_gc_while_mmapped() { - let tempdir = TempDir::new("index").unwrap(); - let tempdir_path = PathBuf::from(tempdir.path()); - let living_files = HashSet::new(); - - let mmap_directory = MmapDirectory::open(&tempdir_path).unwrap(); - let mut managed_directory = ManagedDirectory::wrap(mmap_directory).unwrap(); - managed_directory - .atomic_write(*TEST_PATH1, &vec![0u8, 1u8]) - .unwrap(); - assert!(managed_directory.exists(*TEST_PATH1)); - - let _mmap_read = managed_directory.open_read(*TEST_PATH1).unwrap(); - managed_directory.garbage_collect(|| living_files.clone()); - if cfg!(target_os = "windows") { - // On Windows, gc should try and fail the file as it is mmapped. + managed_directory + .atomic_write(*TEST_PATH1, &vec![0u8, 1u8]) + .unwrap(); assert!(managed_directory.exists(*TEST_PATH1)); - // unmap should happen here. - drop(_mmap_read); - // The file should still be in the list of managed file and - // eventually be deleted once mmap is released. - managed_directory.garbage_collect(|| living_files); - assert!(!managed_directory.exists(*TEST_PATH1)); - } else { - assert!(!managed_directory.exists(*TEST_PATH1)); + + let _mmap_read = managed_directory.open_read(*TEST_PATH1).unwrap(); + managed_directory.garbage_collect(|| living_files.clone()); + if cfg!(target_os = "windows") { + // On Windows, gc should try and fail the file as it is mmapped. + assert!(managed_directory.exists(*TEST_PATH1)); + // unmap should happen here. + drop(_mmap_read); + // The file should still be in the list of managed file and + // eventually be deleted once mmap is released. + managed_directory.garbage_collect(|| living_files); + assert!(!managed_directory.exists(*TEST_PATH1)); + } else { + assert!(!managed_directory.exists(*TEST_PATH1)); + } } + + } } diff --git a/src/directory/mmap_directory.rs b/src/directory/mmap_directory.rs index 70c277d56..0f1f19384 100644 --- a/src/directory/mmap_directory.rs +++ b/src/directory/mmap_directory.rs @@ -6,7 +6,6 @@ use self::notify::RawEvent; use self::notify::RecursiveMode; use self::notify::Watcher; use atomicwrites; -use common::make_io_err; use core::META_FILEPATH; use directory::error::LockError; use directory::error::{DeleteError, IOError, OpenDirectoryError, OpenReadError, OpenWriteError}; @@ -37,6 +36,12 @@ use std::sync::Weak; use std::thread; use tempdir::TempDir; + +/// Create a default io error given a string. +pub(crate) fn make_io_err(msg: String) -> io::Error { + io::Error::new(io::ErrorKind::Other, msg) +} + /// Returns None iff the file exists, can be read, but is empty (and hence /// cannot be mmapped) fn open_mmap(full_path: &Path) -> result::Result, OpenReadError> { diff --git a/src/functional_test.rs b/src/functional_test.rs index ff36369ea..1c2a6501c 100644 --- a/src/functional_test.rs +++ b/src/functional_test.rs @@ -13,7 +13,6 @@ fn check_index_content(searcher: &Searcher, vals: &HashSet) { #[test] #[ignore] -#[cfg(feature = "mmap")] fn test_indexing() { let mut schema_builder = Schema::builder(); diff --git a/src/lib.rs b/src/lib.rs index 70c9e78fd..3a4a676dd 100755 --- a/src/lib.rs +++ b/src/lib.rs @@ -174,6 +174,7 @@ extern crate downcast_rs; #[macro_use] extern crate fail; +#[cfg(feature = "mmap")] #[cfg(test)] mod functional_test; From 83eb0d0cb7ed839f720c206ecef4b16e7e220de1 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Wed, 20 Mar 2019 10:21:15 +0900 Subject: [PATCH 22/26] Disabling tests on Android --- .travis.yml | 4 +-- src/core/index.rs | 14 ++++----- src/directory/managed_directory.rs | 2 -- src/directory/mmap_directory.rs | 1 - src/postings/segment_postings.rs | 46 +++++++++++++++--------------- 5 files changed, 31 insertions(+), 36 deletions(-) diff --git a/.travis.yml b/.travis.yml index ea3d037ab..ae81a31f4 100644 --- a/.travis.yml +++ b/.travis.yml @@ -29,7 +29,7 @@ addons: matrix: include: # Android - - env: TARGET=aarch64-linux-android + - env: TARGET=aarch64-linux-android DISABLE_TESTS #- env: TARGET=arm-linux-androideabi DISABLE_TESTS=1 #- env: TARGET=armv7-linux-androideabi DISABLE_TESTS=1 #- env: TARGET=i686-linux-android DISABLE_TESTS=1 @@ -77,4 +77,4 @@ before_cache: notifications: email: - on_success: never \ No newline at end of file + on_success: never diff --git a/src/core/index.rs b/src/core/index.rs index 5e8e08200..8e1709ce4 100644 --- a/src/core/index.rs +++ b/src/core/index.rs @@ -6,6 +6,8 @@ use core::SegmentId; use core::SegmentMeta; use core::META_FILEPATH; use directory::ManagedDirectory; +#[cfg(feature = "mmap")] +use directory::MmapDirectory; use directory::INDEX_WRITER_LOCK; use directory::{Directory, RAMDirectory}; use error::DataCorruption; @@ -22,15 +24,13 @@ use schema::Schema; use serde_json; use std::borrow::BorrowMut; use std::fmt; +#[cfg(feature = "mmap")] +use std::path::Path; use std::sync::Arc; use tokenizer::BoxedTokenizer; use tokenizer::TokenizerManager; use IndexWriter; use Result; -#[cfg(feature = "mmap")] -use std::path::Path; -#[cfg(feature = "mmap")] -use directory::MmapDirectory; fn load_metas(directory: &Directory) -> Result { let meta_data = directory.atomic_read(&META_FILEPATH)?; @@ -443,13 +443,12 @@ mod tests { test_index_on_commit_reload_policy_aux(field, &mut writer, &reader); } - - #[cfg(feature="mmap")] + #[cfg(feature = "mmap")] mod mmap_specific { + use super::*; use std::path::PathBuf; use tempdir::TempDir; - use super::*; #[test] fn test_index_on_commit_reload_policy_mmap() { @@ -509,7 +508,6 @@ mod tests { } } - fn test_index_on_commit_reload_policy_aux( field: Field, writer: &mut IndexWriter, diff --git a/src/directory/managed_directory.rs b/src/directory/managed_directory.rs index b2f2bfc34..8faef439d 100644 --- a/src/directory/managed_directory.rs +++ b/src/directory/managed_directory.rs @@ -267,7 +267,6 @@ mod tests { use std::path::Path; use tempdir::TempDir; - lazy_static! { static ref TEST_PATH1: &'static Path = Path::new("some_path_for_test"); static ref TEST_PATH2: &'static Path = Path::new("some_path_for_test2"); @@ -353,7 +352,6 @@ mod tests { } } - } } diff --git a/src/directory/mmap_directory.rs b/src/directory/mmap_directory.rs index 0f1f19384..f8f0810d0 100644 --- a/src/directory/mmap_directory.rs +++ b/src/directory/mmap_directory.rs @@ -36,7 +36,6 @@ use std::sync::Weak; use std::thread; use tempdir::TempDir; - /// Create a default io error given a string. pub(crate) fn make_io_err(msg: String) -> io::Error { io::Error::new(io::ErrorKind::Other, msg) diff --git a/src/postings/segment_postings.rs b/src/postings/segment_postings.rs index 807265348..e6eca614b 100644 --- a/src/postings/segment_postings.rs +++ b/src/postings/segment_postings.rs @@ -153,6 +153,25 @@ fn search_within_block(block_docs: &[u32], target: u32) -> usize { } impl DocSet for SegmentPostings { + // goes to the next element. + // next needs to be called a first time to point to the correct element. + #[inline] + fn advance(&mut self) -> bool { + if self.position_computer.is_some() { + let term_freq = self.term_freq() as usize; + self.position_computer.as_mut().unwrap().add_skip(term_freq); + } + self.cur += 1; + if self.cur >= self.block_cursor.block_len() { + self.cur = 0; + if !self.block_cursor.advance() { + self.cur = COMPRESSION_BLOCK_SIZE; + return false; + } + } + true + } + fn skip_next(&mut self, target: DocId) -> SkipResult { if !self.advance() { return SkipResult::End; @@ -235,29 +254,6 @@ impl DocSet for SegmentPostings { } } - // goes to the next element. - // next needs to be called a first time to point to the correct element. - #[inline] - fn advance(&mut self) -> bool { - if self.position_computer.is_some() { - let term_freq = self.term_freq() as usize; - self.position_computer.as_mut().unwrap().add_skip(term_freq); - } - self.cur += 1; - if self.cur >= self.block_cursor.block_len() { - self.cur = 0; - if !self.block_cursor.advance() { - self.cur = COMPRESSION_BLOCK_SIZE; - return false; - } - } - true - } - - fn size_hint(&self) -> u32 { - self.len() as u32 - } - /// Return the current document's `DocId`. #[inline] fn doc(&self) -> DocId { @@ -269,6 +265,10 @@ impl DocSet for SegmentPostings { docs[self.cur] } + fn size_hint(&self) -> u32 { + self.len() as u32 + } + fn append_to_bitset(&mut self, bitset: &mut BitSet) { // finish the current block if self.advance() { From a8cc5208f17973b1ccca025e8e34e45ca30e627a Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Wed, 20 Mar 2019 22:10:05 +0900 Subject: [PATCH 23/26] Linear simd (#519) * linear simd search within block --- CHANGELOG.md | 2 +- src/common/mod.rs | 13 ++ src/indexer/merger.rs | 10 ++ src/indexer/segment_updater.rs | 1 + src/postings/block_search.rs | 229 +++++++++++++++++++++++++++++++ src/postings/compression/mod.rs | 21 +-- src/postings/mod.rs | 5 +- src/postings/segment_postings.rs | 109 +-------------- src/query/intersection.rs | 112 +++++++-------- 9 files changed, 329 insertions(+), 173 deletions(-) create mode 100644 src/postings/block_search.rs diff --git a/CHANGELOG.md b/CHANGELOG.md index d692de690..cee32c998 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -15,7 +15,7 @@ previous index format.* for int fields. (@fulmicoton) - Added DateTime field (@barrotsteindev) - Added IndexReader. By default, index is reloaded automatically upon new commits (@fulmicoton) - +- SIMD linear search within blocks (@fulmicoton) Tantivy 0.8.2 ===================== diff --git a/src/common/mod.rs b/src/common/mod.rs index 6a1026e54..78ec3c423 100644 --- a/src/common/mod.rs +++ b/src/common/mod.rs @@ -13,6 +13,12 @@ pub use self::serialize::{BinarySerializable, FixedSize}; pub use self::vint::{read_u32_vint, serialize_vint_u32, write_u32_vint, VInt}; pub use byteorder::LittleEndian as Endianness; + +/// Segment's max doc must be `< MAX_DOC_LIMIT`. +/// +/// We do not allow segments with more than +pub const MAX_DOC_LIMIT: u32 = 1 << 31; + /// Computes the number of bits that will be used for bitpacking. /// /// In general the target is the minimum number of bits @@ -127,4 +133,11 @@ pub(crate) mod test { assert_eq!(compute_num_bits(256), 9u8); assert_eq!(compute_num_bits(5_000_000_000), 33u8); } + + #[test] + fn test_max_doc() { + // this is the first time I write a unit test for a constant. + assert!(((super::MAX_DOC_LIMIT - 1) as i32) >= 0); + assert!((super::MAX_DOC_LIMIT as i32) < 0); + } } diff --git a/src/indexer/merger.rs b/src/indexer/merger.rs index 97048546b..404fd9bd4 100644 --- a/src/indexer/merger.rs +++ b/src/indexer/merger.rs @@ -1,3 +1,4 @@ +use common::MAX_DOC_LIMIT; use core::Segment; use core::SegmentReader; use core::SerializableSegment; @@ -23,6 +24,7 @@ use termdict::TermMerger; use termdict::TermOrdinal; use DocId; use Result; +use TantivyError; fn compute_total_num_tokens(readers: &[SegmentReader], field: Field) -> u64 { let mut total_tokens = 0u64; @@ -150,6 +152,14 @@ impl IndexMerger { readers.push(reader); } } + if max_doc >= MAX_DOC_LIMIT { + let err_msg = format!( + "The segment resulting from this merge would have {} docs,\ + which exceeds the limit {}.", + max_doc, MAX_DOC_LIMIT + ); + return Err(TantivyError::InvalidArgument(err_msg)); + } Ok(IndexMerger { schema, readers, diff --git a/src/indexer/segment_updater.rs b/src/indexer/segment_updater.rs index 3dc373795..316dbd68a 100644 --- a/src/indexer/segment_updater.rs +++ b/src/indexer/segment_updater.rs @@ -420,6 +420,7 @@ impl SegmentUpdater { }) .collect::>(); merge_candidates.extend(committed_merge_candidates.into_iter()); + for merge_operation in merge_candidates { match self.start_merge_impl(merge_operation) { Ok(merge_future) => { diff --git a/src/postings/block_search.rs b/src/postings/block_search.rs new file mode 100644 index 000000000..825eb9b7d --- /dev/null +++ b/src/postings/block_search.rs @@ -0,0 +1,229 @@ +/// This modules define the logic used to search for a doc in a given +/// block. (at most 128 docs) +/// +/// Searching within a block is a hotspot when running intersection. +/// so it was worth defining it in its own module. + +#[cfg(target_arch = "x86_64")] +mod sse2 { + use postings::compression::COMPRESSION_BLOCK_SIZE; + use std::arch::x86_64::__m128i as DataType; + use std::arch::x86_64::_mm_add_epi32 as op_add; + use std::arch::x86_64::_mm_cmplt_epi32 as op_lt; + use std::arch::x86_64::_mm_load_si128 as op_load; // requires 128-bits alignment + use std::arch::x86_64::_mm_set1_epi32 as set1; + use std::arch::x86_64::_mm_setzero_si128 as set0; + use std::arch::x86_64::_mm_sub_epi32 as op_sub; + use std::arch::x86_64::{_mm_cvtsi128_si32, _mm_shuffle_epi32}; + + const MASK1: i32 = 78; + const MASK2: i32 = 177; + + /// Performs an exhaustive linear search over the + /// + /// There is no early exit here. We simply count the + /// number of elements that are `< target`. + pub fn linear_search_sse2_128(arr: &[u32], target: u32) -> usize { + unsafe { + let ptr = arr.as_ptr() as *const DataType; + let vkey = set1(target as i32); + let mut cnt = set0(); + // We work over 4 `__m128i` at a time. + // A single `__m128i` actual contains 4 `u32`. + for i in 0..(COMPRESSION_BLOCK_SIZE as isize) / (4 * 4) { + let cmp1 = op_lt(op_load(ptr.offset(i * 4)), vkey); + let cmp2 = op_lt(op_load(ptr.offset(i * 4 + 1)), vkey); + let cmp3 = op_lt(op_load(ptr.offset(i * 4 + 2)), vkey); + let cmp4 = op_lt(op_load(ptr.offset(i * 4 + 3)), vkey); + let sum = op_add(op_add(cmp1, cmp2), op_add(cmp3, cmp4)); + cnt = op_sub(cnt, sum); + } + cnt = op_add(cnt, _mm_shuffle_epi32(cnt, MASK1)); + cnt = op_add(cnt, _mm_shuffle_epi32(cnt, MASK2)); + _mm_cvtsi128_si32(cnt) as usize + } + } + + #[cfg(test)] + mod test { + use super::linear_search_sse2_128; + + #[test] + fn test_linear_search_sse2_128_u32() { + for i in 0..23 { + dbg!(i); + let arr: Vec = (0..128).map(|el| el * 2 + 1 << 18).collect(); + assert_eq!(linear_search_sse2_128(&arr, arr[64] + 1), 65); + } + } + } +} + +/// This `linear search` browser exhaustively through the array. +/// but the early exit is very difficult to predict. +/// +/// Coupled with `exponential search` this function is likely +/// to be called with the same `len` +fn linear_search(arr: &[u32], target: u32) -> usize { + arr.iter().map(|&el| if el < target { 1 } else { 0 }).sum() +} + +fn exponential_search(arr: &[u32], target: u32) -> (usize, usize) { + let end = arr.len(); + let mut begin = 0; + for &pivot in &[1, 3, 7, 15, 31, 63] { + if pivot >= end { + break; + } + if arr[pivot] > target { + return (begin, pivot); + } + begin = pivot; + } + (begin, end) +} + +fn galloping(block_docs: &[u32], target: u32) -> usize { + let (start, end) = exponential_search(&block_docs, target); + start + linear_search(&block_docs[start..end], target) +} + +/// Tantivy may rely on SIMD instructions to search for a specific document within +/// a given block. +#[derive(Clone, Copy, PartialEq)] +pub enum BlockSearcher { + #[cfg(target_arch = "x86_64")] + SSE2, + Scalar, +} + +impl BlockSearcher { + /// Search the first index containing an element greater or equal to + /// the target. + /// + /// The results should be equivalent to + /// ```ignore + /// block[..] + // .iter() + // .take_while(|&&val| val < target) + // .count() + /// ``` + /// + /// The `start` argument is just used to hint that the response is + /// greater than beyond `start`. The implementation may or may not use + /// it for optimization. + /// + /// # Assumption + /// + /// The array len is > start. + /// The block is sorted + /// The target is assumed greater or equal to the `arr[start]`. + /// The target is assumed smaller or equal to the last element of the block. + /// + /// Currently the scalar implementation starts by an exponential search, and + /// then operates a linear search in the result subarray. + /// + /// If SSE2 instructions are available in the `(platform, running CPU)`, + /// then we use a different implementation that does an exhaustive linear search over + /// the full block whenever the block is full (`len == 128`). It is surprisingly faster, most likely because of the lack + /// of branch. + pub fn search_in_block(&self, block_docs: &[u32], start: usize, target: u32) -> usize { + #[cfg(target_arch = "x86_64")] + { + use postings::compression::COMPRESSION_BLOCK_SIZE; + if *self == BlockSearcher::SSE2 { + if block_docs.len() == COMPRESSION_BLOCK_SIZE { + return sse2::linear_search_sse2_128(block_docs, target); + } + } + } + start + galloping(&block_docs[start..], target) + } +} + +impl Default for BlockSearcher { + fn default() -> BlockSearcher { + #[cfg(target_arch = "x86_64")] + { + if is_x86_feature_detected!("sse2") { + return BlockSearcher::SSE2; + } + } + BlockSearcher::Scalar + } +} + +#[cfg(test)] +mod tests { + use super::exponential_search; + use super::linear_search; + use super::BlockSearcher; + + #[test] + fn test_linear_search() { + let len: usize = 50; + let arr: Vec = (0..len).map(|el| 1u32 + (el as u32) * 2).collect(); + for target in 1..*arr.last().unwrap() { + let res = linear_search(&arr[..], target); + if res > 0 { + assert!(arr[res - 1] < target); + } + if res < len { + assert!(arr[res] >= target); + } + } + } + + #[test] + fn test_exponentiel_search() { + assert_eq!(exponential_search(&[1, 2], 0), (0, 1)); + assert_eq!(exponential_search(&[1, 2], 1), (0, 1)); + assert_eq!( + exponential_search(&[1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11], 7), + (3, 7) + ); + } + + fn util_test_search_in_block(block_searcher: BlockSearcher, block: &[u32], target: u32) { + let cursor = search_in_block_trivial_but_slow(block, target); + for i in 0..cursor { + assert_eq!(block_searcher.search_in_block(block, i, target), cursor); + } + } + + fn util_test_search_in_block_all(block_searcher: BlockSearcher, block: &[u32]) { + use std::collections::HashSet; + let mut targets = HashSet::new(); + for (i, val) in block.iter().cloned().enumerate() { + if i > 0 { + targets.insert(val - 1); + } + targets.insert(val); + } + for target in targets { + util_test_search_in_block(block_searcher, block, target); + } + } + + fn search_in_block_trivial_but_slow(block: &[u32], target: u32) -> usize { + block.iter().take_while(|&&val| val < target).count() + } + + fn test_search_in_block_util(block_searcher: BlockSearcher) { + for len in 1u32..128u32 { + let v: Vec = (0..len).map(|i| i * 2).collect(); + util_test_search_in_block_all(block_searcher, &v[..]); + } + } + + #[test] + fn test_search_in_block_scalar() { + test_search_in_block_util(BlockSearcher::Scalar); + } + + #[cfg(target_arch = "x86_64")] + #[test] + fn test_search_in_block_sse2() { + test_search_in_block_util(BlockSearcher::SSE2); + } +} diff --git a/src/postings/compression/mod.rs b/src/postings/compression/mod.rs index 4da8c1c25..f35b6cde4 100644 --- a/src/postings/compression/mod.rs +++ b/src/postings/compression/mod.rs @@ -43,9 +43,14 @@ impl BlockEncoder { } } +/// We ensure that the OutputBuffer is align on 128 bits +/// in order to run SSE2 linear search on it. +#[repr(align(128))] +struct OutputBuffer([u32; COMPRESSION_BLOCK_SIZE + 1]); + pub struct BlockDecoder { bitpacker: BitPacker4x, - pub output: [u32; COMPRESSION_BLOCK_SIZE + 1], + output: OutputBuffer, pub output_len: usize, } @@ -59,7 +64,7 @@ impl BlockDecoder { output[COMPRESSION_BLOCK_SIZE] = 0u32; BlockDecoder { bitpacker: BitPacker4x::new(), - output, + output: OutputBuffer(output), output_len: 0, } } @@ -72,23 +77,23 @@ impl BlockDecoder { ) -> usize { self.output_len = COMPRESSION_BLOCK_SIZE; self.bitpacker - .decompress_sorted(offset, &compressed_data, &mut self.output, num_bits) + .decompress_sorted(offset, &compressed_data, &mut self.output.0, num_bits) } pub fn uncompress_block_unsorted(&mut self, compressed_data: &[u8], num_bits: u8) -> usize { self.output_len = COMPRESSION_BLOCK_SIZE; self.bitpacker - .decompress(&compressed_data, &mut self.output, num_bits) + .decompress(&compressed_data, &mut self.output.0, num_bits) } #[inline] pub fn output_array(&self) -> &[u32] { - &self.output[..self.output_len] + &self.output.0[..self.output_len] } #[inline] pub fn output(&self, idx: usize) -> u32 { - self.output[idx] + self.output.0[idx] } } @@ -159,12 +164,12 @@ impl VIntDecoder for BlockDecoder { num_els: usize, ) -> usize { self.output_len = num_els; - vint::uncompress_sorted(compressed_data, &mut self.output[..num_els], offset) + vint::uncompress_sorted(compressed_data, &mut self.output.0[..num_els], offset) } fn uncompress_vint_unsorted<'a>(&mut self, compressed_data: &'a [u8], num_els: usize) -> usize { self.output_len = num_els; - vint::uncompress_unsorted(compressed_data, &mut self.output[..num_els]) + vint::uncompress_unsorted(compressed_data, &mut self.output.0[..num_els]) } } diff --git a/src/postings/mod.rs b/src/postings/mod.rs index e571e26df..3e1c03411 100644 --- a/src/postings/mod.rs +++ b/src/postings/mod.rs @@ -2,6 +2,7 @@ Postings module (also called inverted index) */ +mod block_search; pub(crate) mod compression; /// Postings module /// @@ -16,6 +17,8 @@ mod skip; mod stacker; mod term_info; +pub(crate) use self::block_search::BlockSearcher; + pub(crate) use self::postings_writer::MultiFieldPostingsWriter; pub use self::serializer::{FieldSerializer, InvertedIndexSerializer}; @@ -104,9 +107,7 @@ pub mod tests { let searcher = index.reader().unwrap().searcher(); let inverted_index = searcher.segment_reader(0u32).inverted_index(title); let term = Term::from_field_text(title, "abc"); - let mut positions = Vec::new(); - { let mut postings = inverted_index .read_postings(&term, IndexRecordOption::WithFreqsAndPositions) diff --git a/src/postings/segment_postings.rs b/src/postings/segment_postings.rs index e6eca614b..b12c0b716 100644 --- a/src/postings/segment_postings.rs +++ b/src/postings/segment_postings.rs @@ -7,6 +7,7 @@ use positions::PositionReader; use postings::compression::compressed_block_size; use postings::compression::{BlockDecoder, VIntDecoder, COMPRESSION_BLOCK_SIZE}; use postings::serializer::PostingsSerializer; +use postings::BlockSearcher; use postings::FreqReadingOption; use postings::Postings; use postings::SkipReader; @@ -60,6 +61,7 @@ pub struct SegmentPostings { block_cursor: BlockSegmentPostings, cur: usize, position_computer: Option, + block_searcher: BlockSearcher, } impl SegmentPostings { @@ -70,6 +72,7 @@ impl SegmentPostings { block_cursor: empty_block_cursor, cur: COMPRESSION_BLOCK_SIZE, position_computer: None, + block_searcher: BlockSearcher::default(), } } @@ -117,41 +120,11 @@ impl SegmentPostings { block_cursor: segment_block_postings, cur: COMPRESSION_BLOCK_SIZE, // cursor within the block position_computer: positions_stream_opt.map(PositionComputer::new), + block_searcher: BlockSearcher::default(), } } } -fn linear_search(arr: &[u32], target: u32) -> usize { - arr.iter().map(|&el| if el < target { 1 } else { 0 }).sum() -} - -fn exponential_search(arr: &[u32], target: u32) -> (usize, usize) { - let end = arr.len(); - let mut begin = 0; - for &pivot in &[1, 3, 7, 15, 31, 63] { - if pivot >= end { - break; - } - if arr[pivot] > target { - return (begin, pivot); - } - begin = pivot; - } - (begin, end) -} - -/// Search the first index containing an element greater or equal to the target. -/// -/// # Assumption -/// -/// The array is assumed non empty. -/// The target is assumed greater or equal to the first element. -/// The target is assumed smaller or equal to the last element. -fn search_within_block(block_docs: &[u32], target: u32) -> usize { - let (start, end) = exponential_search(block_docs, target); - start + linear_search(&block_docs[start..end], target) -} - impl DocSet for SegmentPostings { // goes to the next element. // next needs to be called a first time to point to the correct element. @@ -230,9 +203,8 @@ impl DocSet for SegmentPostings { // we're in the right block now, start with an exponential search let block_docs = self.block_cursor.docs(); let new_cur = self - .cur - .wrapping_add(search_within_block(&block_docs[self.cur..], target)); - + .block_searcher + .search_in_block(&block_docs, self.cur, target); if need_positions { sum_freqs_skipped += self.block_cursor.freqs()[self.cur..new_cur] .iter() @@ -614,10 +586,6 @@ impl<'b> Streamer<'b> for BlockSegmentPostings { #[cfg(test)] mod tests { - - use super::exponential_search; - use super::linear_search; - use super::search_within_block; use super::BlockSegmentPostings; use super::BlockSegmentPostingsSkipResult; use super::SegmentPostings; @@ -632,21 +600,6 @@ mod tests { use DocId; use SkipResult; - #[test] - fn test_linear_search() { - let len: usize = 50; - let arr: Vec = (0..len).map(|el| 1u32 + (el as u32) * 2).collect(); - for target in 1..*arr.last().unwrap() { - let res = linear_search(&arr[..], target); - if res > 0 { - assert!(arr[res - 1] < target); - } - if res < len { - assert!(arr[res] >= target); - } - } - } - #[test] fn test_empty_segment_postings() { let mut postings = SegmentPostings::empty(); @@ -662,56 +615,6 @@ mod tests { assert_eq!(postings.doc_freq(), 0); } - fn search_within_block_trivial_but_slow(block: &[u32], target: u32) -> usize { - block - .iter() - .cloned() - .enumerate() - .filter(|&(_, ref val)| *val >= target) - .next() - .unwrap() - .0 - } - - #[test] - fn test_exponentiel_search() { - assert_eq!(exponential_search(&[1, 2], 0), (0, 1)); - assert_eq!(exponential_search(&[1, 2], 1), (0, 1)); - assert_eq!( - exponential_search(&[1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11], 7), - (3, 7) - ); - } - - fn util_test_search_within_block(block: &[u32], target: u32) { - assert_eq!( - search_within_block(block, target), - search_within_block_trivial_but_slow(block, target) - ); - } - - fn util_test_search_within_block_all(block: &[u32]) { - use std::collections::HashSet; - let mut targets = HashSet::new(); - for (i, val) in block.iter().cloned().enumerate() { - if i > 0 { - targets.insert(val - 1); - } - targets.insert(val); - } - for target in targets { - util_test_search_within_block(block, target); - } - } - - #[test] - fn test_search_within_block() { - for len in 1u32..128u32 { - let v: Vec = (0..len).map(|i| i * 2).collect(); - util_test_search_within_block_all(&v[..]); - } - } - #[test] fn test_block_segment_postings() { let mut block_segments = build_block_postings(&(0..100_000).collect::>()); diff --git a/src/query/intersection.rs b/src/query/intersection.rs index 1067fe46d..a35a8ef6c 100644 --- a/src/query/intersection.rs +++ b/src/query/intersection.rs @@ -14,41 +14,35 @@ use Score; /// specialized implementation if the two /// shortest scorers are `TermScorer`s. pub fn intersect_scorers(mut scorers: Vec>) -> Box { + if scorers.is_empty() { + return Box::new(EmptyScorer); + } + if scorers.len() == 1 { + return scorers.pop().unwrap(); + } + // We know that we have at least 2 elements. let num_docsets = scorers.len(); scorers.sort_by(|left, right| right.size_hint().cmp(&left.size_hint())); - let rarest_opt = scorers.pop(); - let second_rarest_opt = scorers.pop(); + let left = scorers.pop().unwrap(); + let right = scorers.pop().unwrap(); scorers.reverse(); - match (rarest_opt, second_rarest_opt) { - (None, None) => Box::new(EmptyScorer), - (Some(single_docset), None) => single_docset, - (Some(left), Some(right)) => { - { - let all_term_scorers = [&left, &right] - .iter() - .all(|&scorer| scorer.is::()); - if all_term_scorers { - let left = *(left.downcast::().map_err(|_| ()).unwrap()); - let right = *(right.downcast::().map_err(|_| ()).unwrap()); - return Box::new(Intersection { - left, - right, - others: scorers, - num_docsets, - }); - } - } - Box::new(Intersection { - left, - right, - others: scorers, - num_docsets, - }) - } - _ => { - unreachable!(); - } + let all_term_scorers = [&left, &right] + .iter() + .all(|&scorer| scorer.is::()); + if all_term_scorers { + return Box::new(Intersection { + left: *(left.downcast::().map_err(|_| ()).unwrap()), + right: *(right.downcast::().map_err(|_| ()).unwrap()), + others: scorers, + num_docsets, + }); } + Box::new(Intersection { + left, + right, + others: scorers, + num_docsets, + }) } /// Creates a `DocSet` that iterator through the intersection of two `DocSet`s. @@ -124,7 +118,6 @@ impl DocSet for Intersection { break; @@ -140,35 +133,36 @@ impl DocSet for Intersection {} - SkipResult::OverStep => { - // this is not in the intersection, - // let's update our candidate. - candidate = docset.doc(); - match left.skip_next(candidate) { - SkipResult::Reached => { - other_candidate_ord = ord; - } - SkipResult::OverStep => { - candidate = left.doc(); - other_candidate_ord = usize::max_value(); - } - SkipResult::End => { - return false; - } + if ord == other_candidate_ord { + continue; + } + // `candidate_ord` is already at the + // right position. + // + // Calling `skip_next` would advance this docset + // and miss it. + match docset.skip_next(candidate) { + SkipResult::Reached => {} + SkipResult::OverStep => { + // this is not in the intersection, + // let's update our candidate. + candidate = docset.doc(); + match left.skip_next(candidate) { + SkipResult::Reached => { + other_candidate_ord = ord; + } + SkipResult::OverStep => { + candidate = left.doc(); + other_candidate_ord = usize::max_value(); + } + SkipResult::End => { + return false; } - continue 'outer; - } - SkipResult::End => { - return false; } + continue 'outer; + } + SkipResult::End => { + return false; } } } From 6c0e621fdb1a3ada05e9ae0d195337e9163aaf96 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Thu, 21 Mar 2019 09:35:04 +0900 Subject: [PATCH 24/26] Added bench info in README --- README.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/README.md b/README.md index bd381f87f..4822a1688 100644 --- a/README.md +++ b/README.md @@ -27,6 +27,14 @@ to build such a search engine. Tantivy is, in fact, strongly inspired by Lucene's design. +# Benchmark + +Tantivy is typically faster than Lucene, but the results will depend on +the nature of the queries in your workload. + +The following [benchmark](https://tantivy-search.github.io/bench/) break downs +performance for different type of queries / collection. + # Features - Full-text search From bfa61d2f2f0838ad8ad9f6624aaf402214ccb722 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Fri, 22 Mar 2019 09:51:00 +0900 Subject: [PATCH 25/26] Added patreon button --- README.md | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 4822a1688..591f06438 100644 --- a/README.md +++ b/README.md @@ -17,7 +17,7 @@ [![](https://sourcerer.io/fame/fulmicoton/tantivy-search/tantivy/images/6)](https://sourcerer.io/fame/fulmicoton/tantivy-search/tantivy/links/6) [![](https://sourcerer.io/fame/fulmicoton/tantivy-search/tantivy/images/7)](https://sourcerer.io/fame/fulmicoton/tantivy-search/tantivy/links/7) - +Become a Patron! **Tantivy** is a **full text search engine library** written in rust. @@ -95,6 +95,17 @@ To check out and run tests, you can simply run : Some tests will not run with just `cargo test` because of `fail-rs`. To run the tests exhaustively, run `./run-tests.sh`. -# Contribute +# How can I support this project ? -Send me an email (paul.masurel at gmail.com) if you want to contribute to tantivy. +There are many ways to support this project. + +- If you use tantivy, share on gitter or by email (paul.masurel@gmail.com) +how you use it. +- Report bugs +- Write a blog +- Complete documentation +- Contribute code +- Drop a word on on [![Say Thanks!](https://img.shields.io/badge/Say%20Thanks-!-1EAEDB.svg)](https://saythanks.io/to/fulmicoton) or become a patron! + + + (join gitter or paul.masurel@gmail.com) From e3abb4481b03a9dba2d2bc130e2ea3bc7c878321 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Fri, 22 Mar 2019 09:58:28 +0900 Subject: [PATCH 26/26] broken link --- README.md | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/README.md b/README.md index 591f06438..dc441cb09 100644 --- a/README.md +++ b/README.md @@ -17,7 +17,8 @@ [![](https://sourcerer.io/fame/fulmicoton/tantivy-search/tantivy/images/6)](https://sourcerer.io/fame/fulmicoton/tantivy-search/tantivy/links/6) [![](https://sourcerer.io/fame/fulmicoton/tantivy-search/tantivy/images/7)](https://sourcerer.io/fame/fulmicoton/tantivy-search/tantivy/links/7) -Become a Patron! +[![Become a patron](https://c5.patreon.com/external/logo/become_a_patron_button.png)](https://www.patreon.com/fulmicoton) + **Tantivy** is a **full text search engine library** written in rust. @@ -99,13 +100,10 @@ To run the tests exhaustively, run `./run-tests.sh`. There are many ways to support this project. -- If you use tantivy, share on gitter or by email (paul.masurel@gmail.com) -how you use it. +- If you use tantivy, tell us about your experience on [gitter](https://gitter.im/tantivy-search/tantivy) or by email (paul.masurel@gmail.com) - Report bugs -- Write a blog +- Write a blog post - Complete documentation -- Contribute code -- Drop a word on on [![Say Thanks!](https://img.shields.io/badge/Say%20Thanks-!-1EAEDB.svg)](https://saythanks.io/to/fulmicoton) or become a patron! - - - (join gitter or paul.masurel@gmail.com) +- Contribute code (you can join [our gitter](https://gitter.im/tantivy-search/tantivy) ) +- Talk about tantivy around you +- Drop a word on on [![Say Thanks!](https://img.shields.io/badge/Say%20Thanks-!-1EAEDB.svg)](https://saythanks.io/to/fulmicoton) or even [![Become a patron](https://c5.patreon.com/external/logo/become_a_patron_button.png)](https://www.patreon.com/fulmicoton)