From ef3a16a129699ce1286fccc12ebd0eb5387ae5ab Mon Sep 17 00:00:00 2001 From: Dru Sellers Date: Sun, 19 Aug 2018 19:40:45 -0500 Subject: [PATCH] Switch from error-chain to failure crate (#376) * Switch from error-chain to failure crate * Added deprecated alias for * Started editing the changeld --- CHANGELOG.md | 8 ++ Cargo.toml | 2 +- src/core/index.rs | 7 +- src/core/segment_reader.rs | 10 +- src/directory/managed_directory.rs | 5 +- src/error.rs | 151 +++++++++++-------------- src/fastfield/error.rs | 3 +- src/indexer/index_writer.rs | 19 ++-- src/indexer/merger.rs | 2 +- src/indexer/segment_manager.rs | 6 +- src/indexer/segment_updater.rs | 14 +-- src/lib.rs | 23 ++-- src/query/phrase_query/mod.rs | 7 +- src/query/phrase_query/phrase_query.rs | 23 ++-- src/query/range_query.rs | 4 +- src/query/regex_query.rs | 4 +- 16 files changed, 142 insertions(+), 146 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0be47a0b7..ef956ac52 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,11 @@ + +Tantivy 0.7 +===================== +- Skip data for doc ids and positions (@fulmicoton), + greatly improving performance +- Tantivy error now rely on the failure crate (@drusellers) + + Tantivy 0.6.1 ========================= - Bugfix #324. GC removing was removing file that were still in useful diff --git a/Cargo.toml b/Cargo.toml index e0ba03e6a..ab767d3fd 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -38,7 +38,6 @@ crossbeam = "0.4" crossbeam-channel = "0.2" futures = "0.1" futures-cpupool = "0.1" -error-chain = "0.8" owning_ref = "0.4" stable_deref_trait = "1.0.0" rust-stemmers = "1" @@ -48,6 +47,7 @@ bitpacking = "0.5" census = "0.1" fnv = "1.0.6" owned-read = "0.4" +failure = "0.1" [target.'cfg(windows)'.dependencies] winapi = "0.2" diff --git a/src/core/index.rs b/src/core/index.rs index c88292ba1..efdfedc5f 100644 --- a/src/core/index.rs +++ b/src/core/index.rs @@ -1,5 +1,5 @@ use core::SegmentId; -use error::{ErrorKind, ResultExt}; +use error::TantivyError; use schema::Schema; use serde_json; use std::borrow::BorrowMut; @@ -17,10 +17,10 @@ use core::IndexMeta; use core::SegmentMeta; use core::SegmentReader; use core::META_FILEPATH; -use directory::{ManagedDirectory, DirectoryClone}; #[cfg(feature = "mmap")] use directory::MmapDirectory; use directory::{Directory, RAMDirectory}; +use directory::{DirectoryClone, ManagedDirectory}; use indexer::index_writer::open_index_writer; use indexer::index_writer::HEAP_SIZE_MIN; use indexer::segment_updater::save_new_metas; @@ -33,7 +33,8 @@ use IndexWriter; fn load_metas(directory: &Directory) -> Result { let meta_data = directory.atomic_read(&META_FILEPATH)?; let meta_string = String::from_utf8_lossy(&meta_data); - serde_json::from_str(&meta_string).chain_err(|| ErrorKind::CorruptedFile(META_FILEPATH.clone())) + serde_json::from_str(&meta_string) + .map_err(|_| TantivyError::CorruptedFile(META_FILEPATH.clone())) } /// Search Index diff --git a/src/core/segment_reader.rs b/src/core/segment_reader.rs index f0edb86b3..37b950332 100644 --- a/src/core/segment_reader.rs +++ b/src/core/segment_reader.rs @@ -5,7 +5,7 @@ use core::Segment; use core::SegmentComponent; use core::SegmentId; use core::SegmentMeta; -use error::ErrorKind; +use error::TantivyError; use fastfield::DeleteBitSet; use fastfield::FacetReader; use fastfield::FastFieldReader; @@ -171,7 +171,7 @@ impl SegmentReader { pub fn facet_reader(&self, field: Field) -> Result { let field_entry = self.schema.get_field_entry(field); if field_entry.field_type() != &FieldType::HierarchicalFacet { - return Err(ErrorKind::InvalidArgument(format!( + return Err(TantivyError::InvalidArgument(format!( "The field {:?} is not a \ hierarchical facet.", field_entry @@ -179,7 +179,7 @@ impl SegmentReader { } let term_ords_reader = self.multi_fast_field_reader(field)?; let termdict_source = self.termdict_composite.open_read(field).ok_or_else(|| { - ErrorKind::InvalidArgument(format!( + TantivyError::InvalidArgument(format!( "The field \"{}\" is a hierarchical \ but this segment does not seem to have the field term \ dictionary.", @@ -462,9 +462,7 @@ mod test { index.load_searchers().unwrap(); let searcher = index.searcher(); - let docs: Vec = searcher.segment_reader(0) - .doc_ids_alive() - .collect(); + let docs: Vec = searcher.segment_reader(0).doc_ids_alive().collect(); assert_eq!(vec![0u32, 2u32], docs); } } diff --git a/src/directory/managed_directory.rs b/src/directory/managed_directory.rs index 2977337c6..cf59b9ace 100644 --- a/src/directory/managed_directory.rs +++ b/src/directory/managed_directory.rs @@ -1,7 +1,7 @@ use core::MANAGED_FILEPATH; use directory::error::{DeleteError, IOError, OpenReadError, OpenWriteError}; use directory::{ReadOnlySource, WritePtr}; -use error::{ErrorKind, Result, ResultExt}; +use error::TantivyError; use serde_json; use std::collections::HashSet; use std::io; @@ -11,6 +11,7 @@ use std::result; use std::sync::RwLockWriteGuard; use std::sync::{Arc, RwLock}; use Directory; +use Result; /// Wrapper of directories that keeps track of files created by Tantivy. /// @@ -51,7 +52,7 @@ impl ManagedDirectory { Ok(data) => { let managed_files_json = String::from_utf8_lossy(&data); let managed_files: HashSet = serde_json::from_str(&managed_files_json) - .chain_err(|| ErrorKind::CorruptedFile(MANAGED_FILEPATH.clone()))?; + .map_err(|_| TantivyError::CorruptedFile(MANAGED_FILEPATH.clone()))?; Ok(ManagedDirectory { directory: Box::new(directory), meta_informations: Arc::new(RwLock::new(MetaInformation { diff --git a/src/error.rs b/src/error.rs index 4ec4bfe25..8fa5cb1ce 100644 --- a/src/error.rs +++ b/src/error.rs @@ -10,129 +10,114 @@ use serde_json; use std::path::PathBuf; use std::sync::PoisonError; -error_chain!( - errors { - /// Path does not exist. - PathDoesNotExist(buf: PathBuf) { - description("path does not exist") - display("path does not exist: '{:?}'", buf) - } - /// File already exists, this is a problem when we try to write into a new file. - FileAlreadyExists(buf: PathBuf) { - description("file already exists") - display("file already exists: '{:?}'", buf) - } - /// IO Error. - IOError(err: IOError) { - description("an IO error occurred") - display("an IO error occurred: '{}'", err) - } - /// The data within is corrupted. - /// - /// For instance, it contains invalid JSON. - CorruptedFile(buf: PathBuf) { - description("file contains corrupted data") - display("file contains corrupted data: '{:?}'", buf) - } - /// A thread holding the locked panicked and poisoned the lock. - Poisoned { - description("a thread holding the locked panicked and poisoned the lock") - } - /// Invalid argument was passed by the user. - InvalidArgument(arg: String) { - description("an invalid argument was passed") - display("an invalid argument was passed: '{}'", arg) - } - /// An Error happened in one of the thread. - ErrorInThread(err: String) { - description("an error occurred in a thread") - display("an error occurred in a thread: '{}'", err) - } - /// An Error appeared related to the schema. - SchemaError(message: String) { - description("the schema is not matching expectations.") - display("Schema error: '{}'", message) - } - /// Tried to access a fastfield reader for a field not configured accordingly. - FastFieldError(err: FastFieldNotAvailableError) { - description("fast field not available") - display("fast field not available: '{:?}'", err) - } - } -); +/// The library's failure based error enum +#[derive(Debug, Fail)] +pub enum TantivyError { + /// Path does not exist. + #[fail(display = "path does not exist: '{:?}'", _0)] + PathDoesNotExist(PathBuf), + /// File already exists, this is a problem when we try to write into a new file. + #[fail(display = "file already exists: '{:?}'", _0)] + FileAlreadyExists(PathBuf), + /// IO Error. + #[fail(display = "an IO error occurred: '{}'", _0)] + IOError(#[cause] IOError), + /// The data within is corrupted. + /// + /// For instance, it contains invalid JSON. + #[fail(display = "file contains corrupted data: '{:?}'", _0)] + CorruptedFile(PathBuf), + /// A thread holding the locked panicked and poisoned the lock. + #[fail(display = "a thread holding the locked panicked and poisoned the lock")] + Poisoned, + /// Invalid argument was passed by the user. + #[fail(display = "an invalid argument was passed: '{}'", _0)] + InvalidArgument(String), + /// An Error happened in one of the thread. + #[fail(display = "an error occurred in a thread: '{}'", _0)] + ErrorInThread(String), + /// An Error appeared related to the schema. + #[fail(display = "Schema error: '{}'", _0)] + SchemaError(String), + /// Tried to access a fastfield reader for a field not configured accordingly. + #[fail(display = "fast field not available: '{:?}'", _0)] + FastFieldError(#[cause] FastFieldNotAvailableError), +} -impl From for Error { - fn from(fastfield_error: FastFieldNotAvailableError) -> Error { - ErrorKind::FastFieldError(fastfield_error).into() +impl From for TantivyError { + fn from(fastfield_error: FastFieldNotAvailableError) -> TantivyError { + TantivyError::FastFieldError(fastfield_error).into() } } -impl From for Error { - fn from(io_error: IOError) -> Error { - ErrorKind::IOError(io_error).into() +impl From for TantivyError { + fn from(io_error: IOError) -> TantivyError { + TantivyError::IOError(io_error).into() } } -impl From for Error { - fn from(io_error: io::Error) -> Error { - ErrorKind::IOError(io_error.into()).into() +impl From for TantivyError { + fn from(io_error: io::Error) -> TantivyError { + TantivyError::IOError(io_error.into()).into() } } -impl From for Error { - fn from(parsing_error: query::QueryParserError) -> Error { - ErrorKind::InvalidArgument(format!("Query is invalid. {:?}", parsing_error)).into() +impl From for TantivyError { + fn from(parsing_error: query::QueryParserError) -> TantivyError { + TantivyError::InvalidArgument(format!("Query is invalid. {:?}", parsing_error)).into() } } -impl From> for Error { - fn from(_: PoisonError) -> Error { - ErrorKind::Poisoned.into() +impl From> for TantivyError { + fn from(_: PoisonError) -> TantivyError { + TantivyError::Poisoned.into() } } -impl From for Error { - fn from(error: OpenReadError) -> Error { +impl From for TantivyError { + fn from(error: OpenReadError) -> TantivyError { match error { OpenReadError::FileDoesNotExist(filepath) => { - ErrorKind::PathDoesNotExist(filepath).into() + TantivyError::PathDoesNotExist(filepath).into() } - OpenReadError::IOError(io_error) => ErrorKind::IOError(io_error).into(), + OpenReadError::IOError(io_error) => TantivyError::IOError(io_error).into(), } } } -impl From for Error { - fn from(error: schema::DocParsingError) -> Error { - ErrorKind::InvalidArgument(format!("Failed to parse document {:?}", error)).into() +impl From for TantivyError { + fn from(error: schema::DocParsingError) -> TantivyError { + TantivyError::InvalidArgument(format!("Failed to parse document {:?}", error)).into() } } -impl From for Error { - fn from(error: OpenWriteError) -> Error { +impl From for TantivyError { + fn from(error: OpenWriteError) -> TantivyError { match error { - OpenWriteError::FileAlreadyExists(filepath) => ErrorKind::FileAlreadyExists(filepath), - OpenWriteError::IOError(io_error) => ErrorKind::IOError(io_error), + OpenWriteError::FileAlreadyExists(filepath) => { + TantivyError::FileAlreadyExists(filepath) + } + OpenWriteError::IOError(io_error) => TantivyError::IOError(io_error), }.into() } } -impl From for Error { - fn from(error: OpenDirectoryError) -> Error { +impl From for TantivyError { + fn from(error: OpenDirectoryError) -> TantivyError { match error { OpenDirectoryError::DoesNotExist(directory_path) => { - ErrorKind::PathDoesNotExist(directory_path).into() + TantivyError::PathDoesNotExist(directory_path).into() } - OpenDirectoryError::NotADirectory(directory_path) => ErrorKind::InvalidArgument( + OpenDirectoryError::NotADirectory(directory_path) => TantivyError::InvalidArgument( format!("{:?} is not a directory", directory_path), ).into(), } } } -impl From for Error { - fn from(error: serde_json::Error) -> Error { +impl From for TantivyError { + fn from(error: serde_json::Error) -> TantivyError { let io_err = io::Error::from(error); - ErrorKind::IOError(io_err.into()).into() + TantivyError::IOError(io_err.into()).into() } } diff --git a/src/fastfield/error.rs b/src/fastfield/error.rs index a05ef2284..df6c2febe 100644 --- a/src/fastfield/error.rs +++ b/src/fastfield/error.rs @@ -4,7 +4,8 @@ use std::result; /// `FastFieldNotAvailableError` is returned when the /// user requested for a fast field reader, and the field was not /// defined in the schema as a fast field. -#[derive(Debug)] +#[derive(Debug, Fail)] +#[fail(display = "field not available: '{:?}'", field_name)] pub struct FastFieldNotAvailableError { field_name: String, } diff --git a/src/indexer/index_writer.rs b/src/indexer/index_writer.rs index 59cfb6661..982140fbc 100644 --- a/src/indexer/index_writer.rs +++ b/src/indexer/index_writer.rs @@ -2,15 +2,15 @@ use super::operation::AddOperation; use super::segment_updater::SegmentUpdater; use super::PreparedCommit; use bit_set::BitSet; -use crossbeam_channel as channel; use core::Index; use core::Segment; use core::SegmentComponent; use core::SegmentId; use core::SegmentMeta; use core::SegmentReader; +use crossbeam_channel as channel; use docset::DocSet; -use error::{Error, ErrorKind, Result, ResultExt}; +use error::TantivyError; use fastfield::write_delete_bitset; use futures::sync::oneshot::Receiver; use indexer::delete_queue::{DeleteCursor, DeleteQueue}; @@ -29,6 +29,7 @@ use std::mem; use std::mem::swap; use std::thread; use std::thread::JoinHandle; +use Result; // Size of the margin for the heap. A segment is closed when the remaining memory // in the heap goes below MARGIN_IN_BYTES. @@ -122,11 +123,11 @@ pub fn open_index_writer( "The heap size per thread needs to be at least {}.", HEAP_SIZE_MIN ); - bail!(ErrorKind::InvalidArgument(err_msg)); + return Err(TantivyError::InvalidArgument(err_msg)); } if heap_size_in_bytes_per_thread >= HEAP_SIZE_MAX { let err_msg = format!("The heap size per thread cannot exceed {}", HEAP_SIZE_MAX); - bail!(ErrorKind::InvalidArgument(err_msg)); + return Err(TantivyError::InvalidArgument(err_msg)); } let (document_sender, document_receiver): (DocumentSender, DocumentReceiver) = channel::bounded(PIPELINE_MAX_SIZE_IN_DOCS); @@ -334,13 +335,15 @@ impl IndexWriter { join_handle .join() .expect("Indexing Worker thread panicked") - .chain_err(|| ErrorKind::ErrorInThread("Error in indexing worker thread.".into()))?; + .map_err(|_| { + TantivyError::ErrorInThread("Error in indexing worker thread.".into()) + })?; } drop(self.workers_join_handle); let result = self.segment_updater .wait_merging_thread() - .chain_err(|| ErrorKind::ErrorInThread("Failed to join merging thread.".into())); + .map_err(|_| TantivyError::ErrorInThread("Failed to join merging thread.".into())); if let Err(ref e) = result { error!("Some merging thread failed {:?}", e); @@ -559,7 +562,7 @@ impl IndexWriter { for worker_handle in former_workers_join_handle { let indexing_worker_result = worker_handle .join() - .map_err(|e| Error::from_kind(ErrorKind::ErrorInThread(format!("{:?}", e))))?; + .map_err(|e| TantivyError::ErrorInThread(format!("{:?}", e)))?; indexing_worker_result?; // add a new worker for the next generation. @@ -654,7 +657,7 @@ mod tests { let index = Index::create_in_ram(schema_builder.build()); let _index_writer = index.writer(40_000_000).unwrap(); match index.writer(40_000_000) { - Err(Error(ErrorKind::FileAlreadyExists(_), _)) => {} + Err(TantivyError::FileAlreadyExists(_)) => {} _ => panic!("Expected FileAlreadyExists error"), } } diff --git a/src/indexer/merger.rs b/src/indexer/merger.rs index 1a5d4c026..e79551a4c 100644 --- a/src/indexer/merger.rs +++ b/src/indexer/merger.rs @@ -2,7 +2,6 @@ use core::Segment; use core::SegmentReader; use core::SerializableSegment; use docset::DocSet; -use error::Result; use fastfield::DeleteBitSet; use fastfield::FastFieldReader; use fastfield::FastFieldSerializer; @@ -23,6 +22,7 @@ use store::StoreWriter; use termdict::TermMerger; use termdict::TermOrdinal; use DocId; +use Result; fn compute_total_num_tokens(readers: &[SegmentReader], field: Field) -> u64 { let mut total_tokens = 0u64; diff --git a/src/indexer/segment_manager.rs b/src/indexer/segment_manager.rs index b82af0823..18175c774 100644 --- a/src/indexer/segment_manager.rs +++ b/src/indexer/segment_manager.rs @@ -2,8 +2,7 @@ use super::segment_register::SegmentRegister; use core::SegmentId; use core::SegmentMeta; use core::{LOCKFILE_FILEPATH, META_FILEPATH}; -use error::ErrorKind; -use error::Result as TantivyResult; +use error::TantivyError; use indexer::delete_queue::DeleteCursor; use indexer::SegmentEntry; use std::collections::hash_set::HashSet; @@ -11,6 +10,7 @@ use std::fmt::{self, Debug, Formatter}; use std::path::PathBuf; use std::sync::RwLock; use std::sync::{RwLockReadGuard, RwLockWriteGuard}; +use Result as TantivyResult; #[derive(Default)] struct SegmentRegisters { @@ -141,7 +141,7 @@ impl SegmentManager { let error_msg = "Merge operation sent for segments that are not \ all uncommited or commited." .to_string(); - bail!(ErrorKind::InvalidArgument(error_msg)) + return Err(TantivyError::InvalidArgument(error_msg)); } Ok(segment_entries) } diff --git a/src/indexer/segment_updater.rs b/src/indexer/segment_updater.rs index faaef38c0..732270ea1 100644 --- a/src/indexer/segment_updater.rs +++ b/src/indexer/segment_updater.rs @@ -7,7 +7,7 @@ use core::SegmentMeta; use core::SerializableSegment; use core::META_FILEPATH; use directory::{Directory, DirectoryClone}; -use error::{Error, ErrorKind, Result, ResultExt}; +use error::TantivyError; use futures::oneshot; use futures::sync::oneshot::Receiver; use futures::Future; @@ -34,6 +34,7 @@ use std::sync::Arc; use std::sync::RwLock; use std::thread; use std::thread::JoinHandle; +use Result; /// Save the index meta file. /// This operation is atomic : @@ -114,12 +115,9 @@ fn perform_merge( // ... we just serialize this index merger in our new segment // to merge the two segments. - let segment_serializer = SegmentSerializer::for_segment(&mut merged_segment) - .chain_err(|| "Creating index serializer failed")?; + let segment_serializer = SegmentSerializer::for_segment(&mut merged_segment)?; - let num_docs = merger - .write(segment_serializer) - .chain_err(|| "Serializing merged index failed")?; + let num_docs = merger.write(segment_serializer)?; let segment_meta = SegmentMeta::new(merged_segment.id(), num_docs); @@ -186,7 +184,7 @@ impl SegmentUpdater { fn run_async T>( &self, f: F, - ) -> CpuFuture { + ) -> CpuFuture { let me_clone = self.clone(); self.0.pool.spawn_fn(move || Ok(f(me_clone))) } @@ -463,7 +461,7 @@ impl SegmentUpdater { merging_thread_handle .join() .map(|_| ()) - .map_err(|_| ErrorKind::ErrorInThread("Merging thread failed.".into()))?; + .map_err(|_| TantivyError::ErrorInThread("Merging thread failed.".into()))?; } // Our merging thread may have queued their completed self.run_async(move |_| {}).wait()?; diff --git a/src/lib.rs b/src/lib.rs index c01226c55..985d68a84 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -7,7 +7,7 @@ #![allow(new_without_default)] #![allow(decimal_literal_representation)] #![warn(missing_docs)] -#![recursion_limit="80"] +#![recursion_limit = "80"] //! # `tantivy` //! @@ -124,7 +124,7 @@ extern crate serde_json; extern crate log; #[macro_use] -extern crate error_chain; +extern crate failure; #[cfg(feature = "mmap")] extern crate atomicwrites; @@ -179,13 +179,16 @@ mod functional_test; #[macro_use] mod macros; -pub use error::{Error, ErrorKind, ResultExt}; +pub use error::TantivyError; + +#[deprecated(since="0.7.0", note="please use `tantivy::TantivyError` instead")] +pub use error::TantivyError as Error; extern crate census; extern crate owned_read; /// Tantivy result. -pub type Result = std::result::Result; +pub type Result = std::result::Result; mod common; mod core; @@ -199,8 +202,8 @@ pub mod collector; pub mod directory; pub mod fastfield; pub mod fieldnorm; -pub mod postings; pub(crate) mod positions; +pub mod postings; pub mod query; pub mod schema; pub mod store; @@ -286,13 +289,13 @@ mod tests { use core::SegmentReader; use docset::DocSet; use query::BooleanQuery; + use rand::distributions::Bernoulli; use rand::distributions::Range; use rand::{Rng, SeedableRng, XorShiftRng}; use schema::*; use Index; use IndexWriter; use Postings; - use rand::distributions::Bernoulli; pub fn assert_nearly_equals(expected: f32, val: f32) { assert!( @@ -321,13 +324,7 @@ mod tests { .sample_iter(&Bernoulli::new(ratio)) .take(n as usize) .enumerate() - .filter_map(|(val, keep)| { - if keep { - Some(val as u32) - } else { - None - } - }) + .filter_map(|(val, keep)| if keep { Some(val as u32) } else { None }) .collect() } diff --git a/src/query/phrase_query/mod.rs b/src/query/phrase_query/mod.rs index f4f974388..303301b0d 100644 --- a/src/query/phrase_query/mod.rs +++ b/src/query/phrase_query/mod.rs @@ -12,7 +12,7 @@ mod tests { use super::*; use collector::tests::TestCollector; use core::Index; - use error::ErrorKind; + use error::TantivyError; use schema::{SchemaBuilder, Term, TEXT}; use tests::assert_nearly_equals; @@ -92,10 +92,9 @@ mod tests { Term::from_field_text(text_field, "b"), ]); let mut test_collector = TestCollector::default(); - if let &ErrorKind::SchemaError(ref msg) = searcher + if let TantivyError::SchemaError(ref msg) = searcher .search(&phrase_query, &mut test_collector) .unwrap_err() - .kind() { assert_eq!( "Applied phrase query on field \"text\", which does not have positions indexed", @@ -191,7 +190,7 @@ mod tests { let mut test_collector = TestCollector::default(); let terms: Vec<(usize, Term)> = texts .iter() - .map(|(offset, text)| (*offset, Term::from_field_text(text_field, text)) ) + .map(|(offset, text)| (*offset, Term::from_field_text(text_field, text))) .collect(); let phrase_query = PhraseQuery::new_with_offset(terms); searcher diff --git a/src/query/phrase_query/phrase_query.rs b/src/query/phrase_query/phrase_query.rs index 9cabe8cc4..e501711ed 100644 --- a/src/query/phrase_query/phrase_query.rs +++ b/src/query/phrase_query/phrase_query.rs @@ -1,6 +1,6 @@ use super::PhraseWeight; use core::searcher::Searcher; -use error::ErrorKind; +use error::TantivyError; use query::bm25::BM25Weight; use query::Query; use query::Weight; @@ -38,11 +38,10 @@ impl PhraseQuery { PhraseQuery::new_with_offset(terms_with_offset) } - /// Creates a new `PhraseQuery` given a list of terms and there offsets. /// /// Can be used to provide custom offset for each term. - pub fn new_with_offset(mut terms: Vec<(usize, Term)>) ->PhraseQuery { + pub fn new_with_offset(mut terms: Vec<(usize, Term)>) -> PhraseQuery { assert!( terms.len() > 1, "A phrase query is required to have strictly more than one term." @@ -66,9 +65,11 @@ impl PhraseQuery { /// `Term`s in the phrase without the associated offsets. pub fn phrase_terms(&self) -> Vec { - self.phrase_terms.iter().map(|(_, term)| term.clone()).collect::>() - } - + self.phrase_terms + .iter() + .map(|(_, term)| term.clone()) + .collect::>() + } } impl Query for PhraseQuery { @@ -85,15 +86,19 @@ impl Query for PhraseQuery { .unwrap_or(false); if !has_positions { let field_name = field_entry.name(); - bail!(ErrorKind::SchemaError(format!( + return Err(TantivyError::SchemaError(format!( "Applied phrase query on field {:?}, which does not have positions indexed", field_name - ))) + ))); } if scoring_enabled { let terms = self.phrase_terms(); let bm25_weight = BM25Weight::for_terms(searcher, &terms); - Ok(Box::new(PhraseWeight::new(self.phrase_terms.clone(), bm25_weight, true))) + Ok(Box::new(PhraseWeight::new( + self.phrase_terms.clone(), + bm25_weight, + true, + ))) } else { Ok(Box::new(PhraseWeight::new( self.phrase_terms.clone(), diff --git a/src/query/range_query.rs b/src/query/range_query.rs index 17d09657f..23efe1995 100644 --- a/src/query/range_query.rs +++ b/src/query/range_query.rs @@ -1,7 +1,7 @@ use common::BitSet; use core::Searcher; use core::SegmentReader; -use error::ErrorKind; +use error::TantivyError; use query::BitSetDocSet; use query::ConstScorer; use query::{Query, Scorer, Weight}; @@ -239,7 +239,7 @@ impl Query for RangeQuery { "Create a range query of the type {:?}, when the field given was of type {:?}", self.value_type, value_type ); - bail!(ErrorKind::SchemaError(err_msg)) + return Err(TantivyError::SchemaError(err_msg)); } Ok(Box::new(RangeWeight { field: self.field, diff --git a/src/query/regex_query.rs b/src/query/regex_query.rs index 8b930212a..9b02fc7cf 100644 --- a/src/query/regex_query.rs +++ b/src/query/regex_query.rs @@ -1,4 +1,4 @@ -use error::ErrorKind; +use error::TantivyError; use fst_regex::Regex; use query::{AutomatonWeight, Query, Weight}; use schema::Field; @@ -80,7 +80,7 @@ impl RegexQuery { fn specialized_weight(&self) -> Result> { let automaton = Regex::new(&self.regex_pattern) - .map_err(|_| ErrorKind::InvalidArgument(self.regex_pattern.clone()))?; + .map_err(|_| TantivyError::InvalidArgument(self.regex_pattern.clone()))?; Ok(AutomatonWeight::new(self.field.clone(), automaton)) }