Switch from error-chain to failure crate (#376)

* Switch from error-chain to failure crate

* Added deprecated alias for

* Started editing the changeld
This commit is contained in:
Dru Sellers
2018-08-19 19:40:45 -05:00
committed by Paul Masurel
parent a0a284fe91
commit ef3a16a129
16 changed files with 142 additions and 146 deletions

View File

@@ -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

View File

@@ -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"

View File

@@ -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<IndexMeta> {
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

View File

@@ -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<FacetReader> {
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<DocId> = searcher.segment_reader(0)
.doc_ids_alive()
.collect();
let docs: Vec<DocId> = searcher.segment_reader(0).doc_ids_alive().collect();
assert_eq!(vec![0u32, 2u32], docs);
}
}

View File

@@ -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<PathBuf> = 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 {

View File

@@ -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<FastFieldNotAvailableError> for Error {
fn from(fastfield_error: FastFieldNotAvailableError) -> Error {
ErrorKind::FastFieldError(fastfield_error).into()
impl From<FastFieldNotAvailableError> for TantivyError {
fn from(fastfield_error: FastFieldNotAvailableError) -> TantivyError {
TantivyError::FastFieldError(fastfield_error).into()
}
}
impl From<IOError> for Error {
fn from(io_error: IOError) -> Error {
ErrorKind::IOError(io_error).into()
impl From<IOError> for TantivyError {
fn from(io_error: IOError) -> TantivyError {
TantivyError::IOError(io_error).into()
}
}
impl From<io::Error> for Error {
fn from(io_error: io::Error) -> Error {
ErrorKind::IOError(io_error.into()).into()
impl From<io::Error> for TantivyError {
fn from(io_error: io::Error) -> TantivyError {
TantivyError::IOError(io_error.into()).into()
}
}
impl From<query::QueryParserError> for Error {
fn from(parsing_error: query::QueryParserError) -> Error {
ErrorKind::InvalidArgument(format!("Query is invalid. {:?}", parsing_error)).into()
impl From<query::QueryParserError> for TantivyError {
fn from(parsing_error: query::QueryParserError) -> TantivyError {
TantivyError::InvalidArgument(format!("Query is invalid. {:?}", parsing_error)).into()
}
}
impl<Guard> From<PoisonError<Guard>> for Error {
fn from(_: PoisonError<Guard>) -> Error {
ErrorKind::Poisoned.into()
impl<Guard> From<PoisonError<Guard>> for TantivyError {
fn from(_: PoisonError<Guard>) -> TantivyError {
TantivyError::Poisoned.into()
}
}
impl From<OpenReadError> for Error {
fn from(error: OpenReadError) -> Error {
impl From<OpenReadError> 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<schema::DocParsingError> for Error {
fn from(error: schema::DocParsingError) -> Error {
ErrorKind::InvalidArgument(format!("Failed to parse document {:?}", error)).into()
impl From<schema::DocParsingError> for TantivyError {
fn from(error: schema::DocParsingError) -> TantivyError {
TantivyError::InvalidArgument(format!("Failed to parse document {:?}", error)).into()
}
}
impl From<OpenWriteError> for Error {
fn from(error: OpenWriteError) -> Error {
impl From<OpenWriteError> 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<OpenDirectoryError> for Error {
fn from(error: OpenDirectoryError) -> Error {
impl From<OpenDirectoryError> 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<serde_json::Error> for Error {
fn from(error: serde_json::Error) -> Error {
impl From<serde_json::Error> 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()
}
}

View File

@@ -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,
}

View File

@@ -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"),
}
}

View File

@@ -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;

View File

@@ -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)
}

View File

@@ -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: 'static + Send, F: 'static + Send + FnOnce(SegmentUpdater) -> T>(
&self,
f: F,
) -> CpuFuture<T, Error> {
) -> CpuFuture<T, TantivyError> {
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()?;

View File

@@ -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<T> = std::result::Result<T, Error>;
pub type Result<T> = std::result::Result<T, error::TantivyError>;
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()
}

View File

@@ -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

View File

@@ -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<Term> {
self.phrase_terms.iter().map(|(_, term)| term.clone()).collect::<Vec<Term>>()
}
self.phrase_terms
.iter()
.map(|(_, term)| term.clone())
.collect::<Vec<Term>>()
}
}
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(),

View File

@@ -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,

View File

@@ -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<AutomatonWeight<Regex>> {
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))
}