mirror of
https://github.com/quickwit-oss/tantivy.git
synced 2026-01-08 18:12:55 +00:00
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:
committed by
Paul Masurel
parent
a0a284fe91
commit
ef3a16a129
@@ -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
|
||||
|
||||
@@ -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"
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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 {
|
||||
|
||||
151
src/error.rs
151
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<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()
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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,
|
||||
}
|
||||
|
||||
@@ -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"),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
@@ -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()?;
|
||||
|
||||
23
src/lib.rs
23
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<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()
|
||||
}
|
||||
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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(),
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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))
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user