From e79a316e41229e51bfff4034ad0085bd620438ed Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Thu, 18 May 2017 20:12:00 +0900 Subject: [PATCH 01/10] Issue 155 - Trying to avoid term lookup when merging terms + Adds a proper Streamer interface --- src/core/mod.rs | 2 - src/core/searcher.rs | 13 -- src/core/segment_reader.rs | 59 +++++--- src/core/term_iterator.rs | 182 ----------------------- src/datastruct/{ => fstmap}/fstmap.rs | 66 +++++---- src/datastruct/fstmap/fstmerger.rs | 206 ++++++++++++++++++++++++++ src/datastruct/fstmap/mod.rs | 9 ++ src/datastruct/fstmap/streamer.rs | 98 ++++++++++++ src/datastruct/mod.rs | 4 +- src/indexer/merger.rs | 137 +++++++++-------- src/indexer/segment_serializer.rs | 8 +- src/lib.rs | 2 - src/postings/serializer.rs | 2 +- src/query/boolean_query/mod.rs | 2 +- src/schema/field_entry.rs | 2 +- src/schema/field_type.rs | 23 ++- 16 files changed, 495 insertions(+), 320 deletions(-) delete mode 100644 src/core/term_iterator.rs rename src/datastruct/{ => fstmap}/fstmap.rs (72%) create mode 100644 src/datastruct/fstmap/fstmerger.rs create mode 100644 src/datastruct/fstmap/mod.rs create mode 100644 src/datastruct/fstmap/streamer.rs diff --git a/src/core/mod.rs b/src/core/mod.rs index 6f29254d9..dca8b5ccd 100644 --- a/src/core/mod.rs +++ b/src/core/mod.rs @@ -7,7 +7,6 @@ mod segment; mod index_meta; mod pool; mod segment_meta; -mod term_iterator; pub use self::searcher::Searcher; pub use self::segment_component::SegmentComponent; @@ -18,7 +17,6 @@ pub use self::segment::SerializableSegment; pub use self::index::Index; pub use self::segment_meta::SegmentMeta; pub use self::index_meta::IndexMeta; -pub use self::term_iterator::TermIterator; use std::path::PathBuf; diff --git a/src/core/searcher.rs b/src/core/searcher.rs index a869fa69c..76c318a2c 100644 --- a/src/core/searcher.rs +++ b/src/core/searcher.rs @@ -7,7 +7,6 @@ use query::Query; use DocId; use DocAddress; use schema::Term; -use core::TermIterator; use std::fmt; @@ -49,18 +48,6 @@ impl Searcher { .fold(0u32, |acc, val| acc + val) } - /// Returns a Stream over all of the sorted unique terms of - /// the searcher. - /// - /// This includes all of the fields from all of the segment_readers. - /// See [TermIterator](struct.TermIterator.html). - /// - /// # Warning - /// This API is very likely to change in the future. - pub fn terms(&self) -> TermIterator { - TermIterator::from(self.segment_readers()) - } - /// Return the list of segment readers pub fn segment_readers(&self) -> &[SegmentReader] { &self.segment_readers diff --git a/src/core/segment_reader.rs b/src/core/segment_reader.rs index 33b8f5a06..5a0a836e7 100644 --- a/src/core/segment_reader.rs +++ b/src/core/segment_reader.rs @@ -13,7 +13,7 @@ use directory::ReadOnlySource; use DocId; use std::str; use postings::TermInfo; -use datastruct::FstMap; +use datastruct::fstmap::FstMap; use std::sync::Arc; use std::fmt; use schema::Field; @@ -171,7 +171,7 @@ impl SegmentReader { } /// Return the term dictionary datastructure. - pub fn term_infos(&self) -> &FstMap { + pub fn term_infos<'b>(&'b self) -> &'b FstMap { &self.term_infos } @@ -201,39 +201,62 @@ impl SegmentReader { let field = term.field(); let field_entry = self.schema.get_field_entry(field); let term_info = get!(self.get_term_info(term)); - let offset = term_info.postings_offset as usize; - let postings_data = &self.postings_data[offset..]; - let freq_handler = match *field_entry.field_type() { + let possible_option = match *field_entry.field_type() { FieldType::Str(ref options) => { let indexing_options = options.get_indexing_options(); match option { - SegmentPostingsOption::NoFreq => FreqHandler::new_without_freq(), + SegmentPostingsOption::NoFreq => SegmentPostingsOption::NoFreq, SegmentPostingsOption::Freq => { if indexing_options.is_termfreq_enabled() { - FreqHandler::new_with_freq() + SegmentPostingsOption::Freq } else { - FreqHandler::new_without_freq() + SegmentPostingsOption::NoFreq } } SegmentPostingsOption::FreqAndPositions => { if indexing_options == TextIndexingOptions::TokenizedWithFreqAndPosition { - let offset = term_info.positions_offset as usize; - let offseted_position_data = &self.positions_data[offset..]; - FreqHandler::new_with_freq_and_position(offseted_position_data) + SegmentPostingsOption::FreqAndPositions } else if indexing_options.is_termfreq_enabled() { - FreqHandler::new_with_freq() + SegmentPostingsOption::Freq } else { - FreqHandler::new_without_freq() + SegmentPostingsOption::NoFreq } } } } - _ => FreqHandler::new_without_freq(), + _ => { SegmentPostingsOption::NoFreq }, }; - Some(SegmentPostings::from_data(term_info.doc_freq, - postings_data, - &self.delete_bitset, - freq_handler)) + Some(self.read_postings_from_terminfo(&term_info, possible_option)) + + // SegmentPostings::from_data(term_info.doc_freq, + // postings_data, + // &self.delete_bitset, + // freq_handler)) + } + + pub fn read_postings_from_terminfo(&self, + term_info: &TermInfo, + option: SegmentPostingsOption) + -> SegmentPostings { + let offset = term_info.postings_offset as usize; + let postings_data = &self.postings_data[offset..]; + let freq_handler = match option { + SegmentPostingsOption::NoFreq => { + FreqHandler::new_without_freq() + }, + SegmentPostingsOption::Freq => { + FreqHandler::new_with_freq() + } + SegmentPostingsOption::FreqAndPositions => { + let offset = term_info.positions_offset as usize; + let offseted_position_data = &self.positions_data[offset..]; + FreqHandler::new_with_freq_and_position(offseted_position_data) + } + }; + SegmentPostings::from_data(term_info.doc_freq, + postings_data, + &self.delete_bitset, + freq_handler) } diff --git a/src/core/term_iterator.rs b/src/core/term_iterator.rs deleted file mode 100644 index 143e8aa6b..000000000 --- a/src/core/term_iterator.rs +++ /dev/null @@ -1,182 +0,0 @@ -use fst::Streamer; -use std::mem; -use std::collections::BinaryHeap; -use fst::map::Keys; -use schema::Field; -use schema::Term; -use core::SegmentReader; -use std::cmp::Ordering; - - -#[derive(PartialEq, Eq, Debug)] -struct HeapItem { - term: Term, - segment_ord: usize, -} - -impl PartialOrd for HeapItem { - fn partial_cmp(&self, other: &HeapItem) -> Option { - Some(self.cmp(other)) - } -} - -impl Ord for HeapItem { - fn cmp(&self, other: &HeapItem) -> Ordering { - (&other.term, &other.segment_ord).cmp(&(&self.term, &self.segment_ord)) - } -} - -/// Given a list of sorted term streams, -/// returns an iterator over sorted unique terms. -/// -/// The item yield is actually a pair with -/// - the term -/// - a slice with the ordinal of the segments containing -/// the terms. -pub struct TermIterator<'a> { - key_streams: Vec>, - heap: BinaryHeap, - // Buffer hosting the list of segment ordinals containing - // the current term. - current_term: Term, - current_segment_ords: Vec, -} - -impl<'a> TermIterator<'a> { - fn new(key_streams: Vec>) -> TermIterator<'a> { - let key_streams_len = key_streams.len(); - TermIterator { - key_streams: key_streams, - heap: BinaryHeap::new(), - current_term: Term::from_field_text(Field(0), ""), - current_segment_ords: (0..key_streams_len).collect(), - } - } - - /// Advance the term iterator to the next term. - /// Returns true if there is indeed another term - /// False if there is none. - pub fn advance(&mut self) -> bool { - self.advance_segments(); - if let Some(mut head) = self.heap.pop() { - mem::swap(&mut self.current_term, &mut head.term); - self.current_segment_ords.push(head.segment_ord); - loop { - match self.heap.peek() { - Some(&ref next_heap_it) if next_heap_it.term == self.current_term => {} - _ => { - break; - } - } - let next_heap_it = self.heap.pop().unwrap(); // safe : we peeked beforehand - self.current_segment_ords.push(next_heap_it.segment_ord); - } - true - } else { - false - } - } - - - /// Returns the current term. - /// - /// This method may be called - /// iff advance() has been called before - /// and "true" was returned. - pub fn term(&self) -> &Term { - &self.current_term - } - - /// Returns the sorted list of segment ordinals - /// that include the current term. - /// - /// This method may be called - /// iff advance() has been called before - /// and "true" was returned. - pub fn segment_ords(&self) -> &[usize] { - &self.current_segment_ords[..] - } - - fn advance_segments(&mut self) { - for segment_ord in self.current_segment_ords.drain(..) { - if let Some(term) = self.key_streams[segment_ord].next() { - self.heap - .push(HeapItem { - term: Term::from_bytes(term), - segment_ord: segment_ord, - }); - } - } - } -} - -impl<'a, 'f> Streamer<'a> for TermIterator<'f> { - type Item = &'a Term; - - fn next(&'a mut self) -> Option { - if self.advance() { - Some(&self.current_term) - } else { - None - } - } -} - -impl<'a> From<&'a [SegmentReader]> for TermIterator<'a> { - fn from(segment_readers: &'a [SegmentReader]) -> TermIterator<'a> { - TermIterator::new(segment_readers - .iter() - .map(|reader| reader.term_infos().keys()) - .collect()) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use schema::{SchemaBuilder, Document, TEXT}; - use core::Index; - - #[test] - fn test_term_iterator() { - let mut schema_builder = SchemaBuilder::default(); - let text_field = schema_builder.add_text_field("text", TEXT); - let index = Index::create_in_ram(schema_builder.build()); - { - let mut index_writer = index.writer_with_num_threads(1, 40_000_000).unwrap(); - { - { - let mut doc = Document::default(); - doc.add_text(text_field, "a b d f"); - index_writer.add_document(doc); - } - index_writer.commit().unwrap(); - } - { - { - let mut doc = Document::default(); - doc.add_text(text_field, "a b c d f"); - index_writer.add_document(doc); - } - index_writer.commit().unwrap(); - } - { - { - let mut doc = Document::default(); - doc.add_text(text_field, "e f"); - index_writer.add_document(doc); - } - index_writer.commit().unwrap(); - } - } - index.load_searchers().unwrap(); - let searcher = index.searcher(); - let mut term_it = searcher.terms(); - let mut terms = String::new(); - while let Some(term) = term_it.next() { - terms.push_str(term.text()); - } - assert_eq!(terms, "abcdef"); - } - -} diff --git a/src/datastruct/fstmap.rs b/src/datastruct/fstmap/fstmap.rs similarity index 72% rename from src/datastruct/fstmap.rs rename to src/datastruct/fstmap/fstmap.rs index 79c35a4e2..54a17ef20 100644 --- a/src/datastruct/fstmap.rs +++ b/src/datastruct/fstmap/fstmap.rs @@ -1,12 +1,12 @@ -use std::io; -use std::io::Write; +use std::io::{self, Write}; use fst; use fst::raw::Fst; - +use super::{FstMapStreamerBuilder, FstMapStreamer}; use directory::ReadOnlySource; use common::BinarySerializable; use std::marker::PhantomData; + fn convert_fst_error(e: fst::Error) -> io::Error { io::Error::new(io::ErrorKind::Other, e) } @@ -19,7 +19,7 @@ pub struct FstMapBuilder { impl FstMapBuilder { pub fn new(w: W) -> io::Result> { - let fst_builder = try!(fst::MapBuilder::new(w).map_err(convert_fst_error)); + let fst_builder = fst::MapBuilder::new(w).map_err(convert_fst_error)?; Ok(FstMapBuilder { fst_builder: fst_builder, data: Vec::new(), @@ -35,33 +35,33 @@ impl FstMapBuilder { /// TODO see if I can bend Rust typesystem to enforce that /// in a nice way. pub fn insert_key(&mut self, key: &[u8]) -> io::Result<()> { - try!(self.fst_builder + self.fst_builder .insert(key, self.data.len() as u64) - .map_err(convert_fst_error)); + .map_err(convert_fst_error)?; Ok(()) } /// Horribly unsafe, nobody should ever do that... except me :) pub fn insert_value(&mut self, value: &V) -> io::Result<()> { - try!(value.serialize(&mut self.data)); + value.serialize(&mut self.data)?; Ok(()) } #[cfg(test)] pub fn insert(&mut self, key: &[u8], value: &V) -> io::Result<()> { - try!(self.fst_builder + self.fst_builder .insert(key, self.data.len() as u64) - .map_err(convert_fst_error)); - try!(value.serialize(&mut self.data)); + .map_err(convert_fst_error)?; + value.serialize(&mut self.data)?; Ok(()) } pub fn finish(self) -> io::Result { - let mut file = try!(self.fst_builder.into_inner().map_err(convert_fst_error)); + let mut file = self.fst_builder.into_inner().map_err(convert_fst_error)?; let footer_size = self.data.len() as u32; - try!(file.write_all(&self.data)); - try!((footer_size as u32).serialize(&mut file)); - try!(file.flush()); + file.write_all(&self.data)?; + (footer_size as u32).serialize(&mut file)?; + file.flush()?; Ok(file) } } @@ -76,37 +76,34 @@ pub struct FstMap { fn open_fst_index(source: ReadOnlySource) -> io::Result { Ok(fst::Map::from(match source { ReadOnlySource::Anonymous(data) => { - try!(Fst::from_shared_bytes(data.data, data.start, data.len) - .map_err(convert_fst_error)) + Fst::from_shared_bytes(data.data, data.start, data.len) + .map_err(convert_fst_error)? } ReadOnlySource::Mmap(mmap_readonly) => { - try!(Fst::from_mmap(mmap_readonly).map_err(convert_fst_error)) + Fst::from_mmap(mmap_readonly).map_err(convert_fst_error)? } })) } -impl FstMap { - pub fn keys(&self) -> fst::map::Keys { - self.fst_index.keys() - } +impl FstMap where V: BinarySerializable { pub fn from_source(source: ReadOnlySource) -> io::Result> { let total_len = source.len(); let length_offset = total_len - 4; let mut split_len_buffer: &[u8] = &source.as_slice()[length_offset..]; - let footer_size = try!(u32::deserialize(&mut split_len_buffer)) as usize; + let footer_size = u32::deserialize(&mut split_len_buffer)? as usize; let split_len = length_offset - footer_size; let fst_source = source.slice(0, split_len); let values_source = source.slice(split_len, length_offset); - let fst_index = try!(open_fst_index(fst_source)); + let fst_index = open_fst_index(fst_source)?; Ok(FstMap { fst_index: fst_index, values_mmap: values_source, _phantom_: PhantomData, - }) + }) } - fn read_value(&self, offset: u64) -> V { + pub(crate) fn read_value(&self, offset: u64) -> V { let buffer = self.values_mmap.as_slice(); let mut cursor = &buffer[(offset as usize)..]; V::deserialize(&mut cursor).expect("Data in FST is corrupted") @@ -117,6 +114,14 @@ impl FstMap { .get(key) .map(|offset| self.read_value(offset)) } + + pub fn stream(&self) -> FstMapStreamer { + self.range().into_stream() + } + + pub fn range(&self) -> FstMapStreamerBuilder { + FstMapStreamerBuilder::new(&self, self.fst_index.range()) + } } #[cfg(test)] @@ -124,7 +129,6 @@ mod tests { use super::*; use directory::{RAMDirectory, Directory}; use std::path::PathBuf; - use fst::Streamer; #[test] fn test_fstmap() { @@ -141,10 +145,12 @@ mod tests { let fstmap: FstMap = FstMap::from_source(source).unwrap(); assert_eq!(fstmap.get("abc"), Some(34u32)); assert_eq!(fstmap.get("abcd"), Some(346u32)); - let mut keys = fstmap.keys(); - assert_eq!(keys.next().unwrap(), "abc".as_bytes()); - assert_eq!(keys.next().unwrap(), "abcd".as_bytes()); - assert_eq!(keys.next(), None); + let mut stream = fstmap.stream(); + assert!(stream.advance()); + assert_eq!(stream.key(), "abc".as_bytes()); + assert!(stream.advance()); + assert_eq!(stream.key(), "abcd".as_bytes()); + assert!(!stream.advance()); } } diff --git a/src/datastruct/fstmap/fstmerger.rs b/src/datastruct/fstmap/fstmerger.rs new file mode 100644 index 000000000..90f2264d7 --- /dev/null +++ b/src/datastruct/fstmap/fstmerger.rs @@ -0,0 +1,206 @@ +use std::collections::BinaryHeap; +use core::SegmentReader; +use super::FstMapStreamer; +use common::BinarySerializable; +use postings::TermInfo; +use std::cmp::Ordering; + + +pub struct HeapItem<'a, V> where V: 'a + BinarySerializable { + pub streamer: FstMapStreamer<'a, V>, + pub segment_ord: usize, +} + +impl<'a, V> PartialEq for HeapItem<'a, V> where V: 'a + BinarySerializable { + fn eq(&self, other: &Self) -> bool { + self.segment_ord == other.segment_ord + } +} + +impl<'a, V> Eq for HeapItem<'a, V> where V: 'a + BinarySerializable { +} + +impl<'a, V> PartialOrd for HeapItem<'a, V> where V: 'a + BinarySerializable { + fn partial_cmp(&self, other: &HeapItem<'a, V>) -> Option { + Some(self.cmp(other)) + } +} + +impl<'a, V> Ord for HeapItem<'a, V> where V: 'a + BinarySerializable { + fn cmp(&self, other: &HeapItem<'a, V>) -> Ordering { + (&other.streamer.key(), &other.segment_ord).cmp(&(&self.streamer.key(), &self.segment_ord)) + } +} + +/// Given a list of sorted term streams, +/// returns an iterator over sorted unique terms. +/// +/// The item yield is actually a pair with +/// - the term +/// - a slice with the ordinal of the segments containing +/// the terms. +pub struct FstMerger<'a, V> where V: 'a + BinarySerializable { + heap: BinaryHeap>, + current_streamers: Vec>, +} + +impl<'a, V> FstMerger<'a, V> where V: 'a + BinarySerializable { + fn new(streams: Vec>) -> FstMerger<'a, V> { + FstMerger { + heap: BinaryHeap::new(), + current_streamers: streams + .into_iter() + .enumerate() + .map(|(ord, streamer)| { + HeapItem { + streamer: streamer, + segment_ord: ord, + } + }) + .collect(), + } + } + + fn advance_segments(&mut self) { + let streamers = &mut self.current_streamers; + let heap = &mut self.heap; + for mut heap_item in streamers.drain(..) { + if heap_item.streamer.advance() { + heap.push(heap_item); + } + } + } + + + /// Advance the term iterator to the next term. + /// Returns true if there is indeed another term + /// False if there is none. + pub fn advance(&mut self) -> bool { + self.advance_segments(); + if let Some(head) = self.heap.pop() { + self.current_streamers.push(head); + loop { + if let Some(next_streamer) = self.heap.peek() { + if self.current_streamers[0].streamer.key() != next_streamer.streamer.key() { + break; + } + } + else { break; } // no more streamer. + let next_heap_it = self.heap.pop().unwrap(); // safe : we peeked beforehand + self.current_streamers.push(next_heap_it); + } + true + } else { + false + } + } + + /// Returns the current term. + /// + /// This method may be called + /// iff advance() has been called before + /// and "true" was returned. + pub fn key(&self) -> &[u8] { + &self.current_streamers[0].streamer.key() + } + + /// Returns the sorted list of segment ordinals + /// that include the current term. + /// + /// This method may be called + /// iff advance() has been called before + /// and "true" was returned. + pub fn current_kvs(&self) -> &[HeapItem<'a, V>] { + &self.current_streamers[..] + } +} + + + + + /* + + + /// Returns the sorted list of segment ordinals + /// that include the current term. + /// + /// This method may be called + /// iff advance() has been called before + /// and "true" was returned. + pub fn segment_ords(&self) -> &[usize] { + &self.current_segment_ords[..] + } + */ + +/* +impl<'a, V> Streamer<'a> for FstMerger<'a, V> { + type Item = &'a Term; + + fn next(&'a mut self) -> Option { + if self.advance() { + Some(&self.current_term) + } else { + None + } + } +} +*/ + +impl<'a> From<&'a [SegmentReader]> for FstMerger<'a, TermInfo> where TermInfo: BinarySerializable { + fn from(segment_readers: &'a [SegmentReader]) -> FstMerger<'a, TermInfo> { + FstMerger::new(segment_readers + .iter() + .map(|reader| reader.term_infos().stream()) + .collect() ) + } +} + +// #[cfg(test)] +// mod tests { +// use super::*; +// use schema::{SchemaBuilder, Document, TEXT}; +// use core::Index; + +// #[test] +// fn test_term_iterator() { +// let mut schema_builder = SchemaBuilder::default(); +// let text_field = schema_builder.add_text_field("text", TEXT); +// let index = Index::create_in_ram(schema_builder.build()); +// { +// let mut index_writer = index.writer_with_num_threads(1, 40_000_000).unwrap(); +// { +// { +// let mut doc = Document::default(); +// doc.add_text(text_field, "a b d f"); +// index_writer.add_document(doc); +// } +// index_writer.commit().unwrap(); +// } +// { +// { +// let mut doc = Document::default(); +// doc.add_text(text_field, "a b c d f"); +// index_writer.add_document(doc); +// } +// index_writer.commit().unwrap(); +// } +// { +// { +// let mut doc = Document::default(); +// doc.add_text(text_field, "e f"); +// index_writer.add_document(doc); +// } +// index_writer.commit().unwrap(); +// } +// } +// index.load_searchers().unwrap(); +// let searcher = index.searcher(); +// let mut term_it = searcher.terms(); +// let mut terms = String::new(); +// while let Some(term) = term_it.next() { +// terms.push_str(term.text()); +// } +// assert_eq!(terms, "abcdef"); +// } + +// } diff --git a/src/datastruct/fstmap/mod.rs b/src/datastruct/fstmap/mod.rs new file mode 100644 index 000000000..701b5c47b --- /dev/null +++ b/src/datastruct/fstmap/mod.rs @@ -0,0 +1,9 @@ +mod fstmap; +mod streamer; +mod fstmerger; + +pub use self::fstmap::FstMap; +pub use self::fstmap::FstMapBuilder; +pub use self::streamer::FstMapStreamer; +pub use self::streamer::FstMapStreamerBuilder; +pub use self::fstmerger::FstMerger; \ No newline at end of file diff --git a/src/datastruct/fstmap/streamer.rs b/src/datastruct/fstmap/streamer.rs new file mode 100644 index 000000000..b91b13019 --- /dev/null +++ b/src/datastruct/fstmap/streamer.rs @@ -0,0 +1,98 @@ +use fst::{self, IntoStreamer, Streamer}; +use fst::map::{StreamBuilder, Stream}; +use common::BinarySerializable; +use super::FstMap; + +pub struct FstMapStreamerBuilder<'a, V> where V: 'a + BinarySerializable { + fst_map: &'a FstMap, + stream_builder: StreamBuilder<'a>, +} + +impl<'a, V> FstMapStreamerBuilder<'a, V> where V: 'a + BinarySerializable { + + pub fn ge>(mut self, bound: T) -> Self { + self.stream_builder = self.stream_builder.ge(bound); + self + } + + pub fn gt>(mut self, bound: T) -> Self { + self.stream_builder = self.stream_builder.gt(bound); + self + } + + pub fn le>(mut self, bound: T) -> Self { + self.stream_builder = self.stream_builder.le(bound); + self + } + + pub fn lt>(mut self, bound: T) -> Self { + self.stream_builder = self.stream_builder.lt(bound); + self + } + + pub fn into_stream(self) -> FstMapStreamer<'a, V> { + FstMapStreamer { + fst_map: self.fst_map, + stream: self.stream_builder.into_stream(), + buffer: Vec::with_capacity(100), + offset: 0u64, + } + } + + pub fn new(fst_map: &'a FstMap, stream_builder: StreamBuilder<'a>) -> FstMapStreamerBuilder<'a, V> { + FstMapStreamerBuilder { + fst_map: fst_map, + stream_builder: stream_builder, + } + } +} + + + + + +pub struct FstMapStreamer<'a, V> where V: 'a + BinarySerializable { + fst_map: &'a FstMap, + stream: Stream<'a>, + offset: u64, + buffer: Vec, +} + + +impl<'a, V> fst::Streamer<'a> for FstMapStreamer<'a, V> where V: 'a + BinarySerializable { + + type Item = &'a [u8]; + + fn next<'b>(&'b mut self) -> Option<&'b [u8]> { + if self.advance() { + Some(&self.buffer) + } + else { + None + } + } +} + +impl<'a, V> FstMapStreamer<'a, V> where V: 'a + BinarySerializable { + + pub fn advance(&mut self) -> bool { + if let Some((term, offset)) = self.stream.next() { + self.buffer.clear(); + self.buffer.extend_from_slice(term); + self.offset = offset; + true + } + else { + false + } + } + + pub fn key(&self) -> &[u8] { + &self.buffer + } + + pub fn value(&self) -> V { + self.fst_map.read_value(self.offset) + } +} + diff --git a/src/datastruct/mod.rs b/src/datastruct/mod.rs index f026afa7f..96cc8242f 100644 --- a/src/datastruct/mod.rs +++ b/src/datastruct/mod.rs @@ -1,7 +1,5 @@ -mod fstmap; +pub mod fstmap; mod skip; pub mod stacker; -pub use self::fstmap::FstMapBuilder; -pub use self::fstmap::FstMap; pub use self::skip::{SkipListBuilder, SkipList}; diff --git a/src/indexer/merger.rs b/src/indexer/merger.rs index 47593a57e..c94b21ea8 100644 --- a/src/indexer/merger.rs +++ b/src/indexer/merger.rs @@ -10,13 +10,15 @@ use fastfield::U64FastFieldReader; use itertools::Itertools; use postings::Postings; use postings::DocSet; -use core::TermIterator; use fastfield::DeleteBitSet; use schema::{Schema, Field}; +use datastruct::fstmap::FstMerger; use fastfield::FastFieldSerializer; use fastfield::FastFieldReader; use store::StoreWriter; use std::cmp::{min, max}; +use schema; +use postings::SegmentPostingsOption; pub struct IndexMerger { schema: Schema, @@ -129,8 +131,7 @@ impl IndexMerger { // used both to merge field norms and regular u64 fast fields. fn generic_write_fast_field(&self, fields: Vec, - field_reader_extractor: &Fn(&SegmentReader, Field) - -> Option, + field_reader_extractor: &Fn(&SegmentReader, Field) -> Option, fast_field_serializer: &mut FastFieldSerializer) -> Result<()> { @@ -161,9 +162,8 @@ impl IndexMerger { return Err(Error::SchemaError(error_msg)); } } - } - + if u64_readers.is_empty() { // we have actually zero documents. min_val = 0; @@ -172,24 +172,24 @@ impl IndexMerger { assert!(min_val <= max_val); - try!(fast_field_serializer.new_u64_fast_field(field, min_val, max_val)); + fast_field_serializer.new_u64_fast_field(field, min_val, max_val)?; for (max_doc, u64_reader, delete_bitset) in u64_readers { for doc_id in 0..max_doc { if !delete_bitset.is_deleted(doc_id) { let val = u64_reader.get(doc_id); - try!(fast_field_serializer.add_val(val)); + fast_field_serializer.add_val(val)?; } } } - try!(fast_field_serializer.close_field()); + fast_field_serializer.close_field()?; } Ok(()) } fn write_postings(&self, postings_serializer: &mut PostingsSerializer) -> Result<()> { - - let mut merged_terms = TermIterator::from(&self.readers[..]); + + let mut merged_terms = FstMerger::from(&self.readers[..]); let mut delta_position_computer = DeltaPositionComputer::new(); let mut max_doc = 0; @@ -211,7 +211,9 @@ impl IndexMerger { } let mut last_field: Option = None; - + let mut segment_postings_option = SegmentPostingsOption::FreqAndPositions; + let mut need_to_call_new_field = false; + while merged_terms.advance() { // Create the total list of doc ids // by stacking the doc ids from the different segment. @@ -223,59 +225,70 @@ impl IndexMerger { // - Segment 2's doc ids become [seg0.max_doc + seg1.max_doc, // seg0.max_doc + seg1.max_doc + seg2.max_doc] // ... - let term = merged_terms.term(); - let mut term_written = false; - let segment_postings = merged_terms - .segment_ords() - .iter() - .cloned() - .flat_map(|segment_ord| { - self.readers[segment_ord] - .read_postings_all_info(term) - .map(|segment_postings| (segment_ord, segment_postings)) - }) - .collect::>(); - - // We can remove the term if all documents which - // contained it have been deleted. - if !segment_postings.is_empty() { - - // We can now serialize this postings, by pushing each document to the - // postings serializer. - - for (segment_ord, mut segment_postings) in segment_postings { - let old_to_new_doc_id = &merged_doc_id_map[segment_ord]; - while segment_postings.advance() { - if let Some(remapped_doc_id) = - old_to_new_doc_id[segment_postings.doc() as usize] { - if !term_written { - let current_field = term.field(); - if last_field != Some(current_field) { - postings_serializer.new_field(current_field); - last_field = Some(current_field); - } - - // we make sure to only write the term iff - // there is at least one document. - postings_serializer.new_term(term.as_slice())?; - - term_written = true; - } - let delta_positions: &[u32] = - delta_position_computer - .compute_delta_positions(segment_postings.positions()); - try!(postings_serializer.write_doc(remapped_doc_id, - segment_postings.term_freq(), - delta_positions)); - } - } - } - - if term_written { - try!(postings_serializer.close_term()); - } + let term_bytes = merged_terms.key(); + let current_field = schema::extract_field_from_term_bytes(term_bytes); + + if last_field != Some(current_field) { + // we reached a new field. + let field_entry = self.schema.get_field_entry(current_field); + // ... set segment postings option the new field. + segment_postings_option = field_entry.field_type().get_segment_postings_option(); + last_field = Some(current_field); + need_to_call_new_field = true; } + // Let's compute the list of non-empty posting lists + let segment_postings: Vec<_> = merged_terms + .current_kvs() + .iter() + .flat_map(|heap_item| { + let segment_ord = heap_item.segment_ord; + let term_info = heap_item.streamer.value(); + let segment_reader = &self.readers[heap_item.segment_ord]; + let mut segment_postings = segment_reader + .read_postings_from_terminfo(&term_info, segment_postings_option); + if segment_postings.advance() { + Some((segment_ord, segment_postings)) + } + else { + None + } + }) + .collect(); + + if segment_postings.is_empty() { + continue; + } + + if need_to_call_new_field { + postings_serializer.new_field(current_field); + last_field = Some(current_field); + need_to_call_new_field = false; + } + + postings_serializer.new_term(term_bytes)?; + + // We can now serialize this postings, by pushing each document to the + // postings serializer. + + for (segment_ord, mut segment_postings) in segment_postings { + let old_to_new_doc_id = &merged_doc_id_map[segment_ord]; + loop { + if let Some(remapped_doc_id) = + old_to_new_doc_id[segment_postings.doc() as usize] { + // we make sure to only write the term iff + // there is at least one document. + let delta_positions: &[u32] = delta_position_computer.compute_delta_positions(segment_postings.positions()); + postings_serializer.write_doc(remapped_doc_id, + segment_postings.term_freq(), + delta_positions)?; + } + if !segment_postings.advance() { + break; + } + } + } + postings_serializer.close_term()?; } Ok(()) } diff --git a/src/indexer/segment_serializer.rs b/src/indexer/segment_serializer.rs index 7d800f708..b75663927 100644 --- a/src/indexer/segment_serializer.rs +++ b/src/indexer/segment_serializer.rs @@ -58,10 +58,10 @@ impl SegmentSerializer { /// Finalize the segment serialization. pub fn close(self) -> Result<()> { - try!(self.fast_field_serializer.close()); - try!(self.postings_serializer.close()); - try!(self.store_writer.close()); - try!(self.fieldnorms_serializer.close()); + self.fast_field_serializer.close()?; + self.postings_serializer.close()?; + self.store_writer.close()?; + self.fieldnorms_serializer.close()?; Ok(()) } } diff --git a/src/lib.rs b/src/lib.rs index 96369739d..ef8a7dd05 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -134,8 +134,6 @@ pub use postings::DocSet; pub use postings::Postings; pub use postings::SegmentPostingsOption; -pub use core::TermIterator; - /// Expose the current version of tantivy, as well /// whether it was compiled with the simd compression. diff --git a/src/postings/serializer.rs b/src/postings/serializer.rs index e60f5c8c7..7b45f6a6e 100644 --- a/src/postings/serializer.rs +++ b/src/postings/serializer.rs @@ -1,5 +1,5 @@ use Result; -use datastruct::FstMapBuilder; +use datastruct::fstmap::FstMapBuilder; use super::TermInfo; use schema::Field; use schema::FieldEntry; diff --git a/src/query/boolean_query/mod.rs b/src/query/boolean_query/mod.rs index 01ef9e824..a9888bad9 100644 --- a/src/query/boolean_query/mod.rs +++ b/src/query/boolean_query/mod.rs @@ -122,7 +122,7 @@ mod tests { fieldnorm_reader_opt: Some(left_fieldnorms), postings: left, }; - + let right_fieldnorms = U64FastFieldReader::from((0u64..9u64).map(|doc| doc * 5).collect::>()); let right = VecPostings::from(vec![1, 3, 8]); diff --git a/src/schema/field_entry.rs b/src/schema/field_entry.rs index 883dc49ff..9feaf64d4 100644 --- a/src/schema/field_entry.rs +++ b/src/schema/field_entry.rs @@ -67,7 +67,7 @@ impl FieldEntry { FieldType::I64(ref options) => options.is_indexed(), } } - + /// Returns true iff the field is a int (signed or unsigned) fast field pub fn is_int_fast(&self) -> bool { match self.field_type { diff --git a/src/schema/field_type.rs b/src/schema/field_type.rs index 67a0d42ff..999f919ba 100644 --- a/src/schema/field_type.rs +++ b/src/schema/field_type.rs @@ -2,7 +2,8 @@ use schema::{TextOptions, IntOptions}; use serde_json::Value as JsonValue; use schema::Value; - +use postings::SegmentPostingsOption; +use schema::TextIndexingOptions; /// Possible error that may occur while parsing a field value /// At this point the JSON is known to be valid. @@ -39,6 +40,26 @@ impl FieldType { } } + pub fn get_segment_postings_option(&self) -> SegmentPostingsOption { + match *self { + FieldType::Str(ref text_options) => { + match text_options.get_indexing_options() { + TextIndexingOptions::TokenizedNoFreq => + SegmentPostingsOption::NoFreq, + TextIndexingOptions::TokenizedWithFreq => + SegmentPostingsOption::Freq, + TextIndexingOptions::TokenizedWithFreqAndPosition => + SegmentPostingsOption::FreqAndPositions, + _ => { + SegmentPostingsOption::NoFreq + } + } + } + FieldType::U64(_) | + FieldType::I64(_) => SegmentPostingsOption::NoFreq, + } + } + /// Parses a field value from json, given the target FieldType. /// /// Tantivy will not try to cast values. From ca76fd5ba08cfddde5a8d75010adb992a342f5a3 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Thu, 18 May 2017 20:41:56 +0900 Subject: [PATCH 02/10] Uncommenting unit test --- src/core/searcher.rs | 14 +++ src/core/segment_reader.rs | 7 +- src/datastruct/fstmap/fstmerger.rs | 140 +++++++++++++---------------- 3 files changed, 78 insertions(+), 83 deletions(-) diff --git a/src/core/searcher.rs b/src/core/searcher.rs index 76c318a2c..9fade4cc5 100644 --- a/src/core/searcher.rs +++ b/src/core/searcher.rs @@ -7,7 +7,9 @@ use query::Query; use DocId; use DocAddress; use schema::Term; +use datastruct::fstmap::FstMerger; use std::fmt; +use postings::TermInfo; /// Holds a list of `SegmentReader`s ready for search. @@ -62,6 +64,18 @@ impl Searcher { pub fn search(&self, query: &Query, collector: &mut C) -> Result { query.search(self, collector) } + + /// Returns a Stream over all of the sorted unique terms of + /// the searcher. + /// + /// This includes all of the fields from all of the segment_readers. + /// See [TermIterator](struct.TermIterator.html). + /// + /// # Warning + /// This API is very likely to change in the future. + pub fn terms(&self) -> FstMerger { + FstMerger::from(self.segment_readers()) + } } diff --git a/src/core/segment_reader.rs b/src/core/segment_reader.rs index 5a0a836e7..8ca96cbf8 100644 --- a/src/core/segment_reader.rs +++ b/src/core/segment_reader.rs @@ -171,7 +171,7 @@ impl SegmentReader { } /// Return the term dictionary datastructure. - pub fn term_infos<'b>(&'b self) -> &'b FstMap { + pub fn term_infos(&self) -> &FstMap { &self.term_infos } @@ -227,11 +227,6 @@ impl SegmentReader { _ => { SegmentPostingsOption::NoFreq }, }; Some(self.read_postings_from_terminfo(&term_info, possible_option)) - - // SegmentPostings::from_data(term_info.doc_freq, - // postings_data, - // &self.delete_bitset, - // freq_handler)) } pub fn read_postings_from_terminfo(&self, diff --git a/src/datastruct/fstmap/fstmerger.rs b/src/datastruct/fstmap/fstmerger.rs index 90f2264d7..687ba5127 100644 --- a/src/datastruct/fstmap/fstmerger.rs +++ b/src/datastruct/fstmap/fstmerger.rs @@ -4,7 +4,7 @@ use super::FstMapStreamer; use common::BinarySerializable; use postings::TermInfo; use std::cmp::Ordering; - +use fst::Streamer; pub struct HeapItem<'a, V> where V: 'a + BinarySerializable { pub streamer: FstMapStreamer<'a, V>, @@ -117,35 +117,6 @@ impl<'a, V> FstMerger<'a, V> where V: 'a + BinarySerializable { - - /* - - - /// Returns the sorted list of segment ordinals - /// that include the current term. - /// - /// This method may be called - /// iff advance() has been called before - /// and "true" was returned. - pub fn segment_ords(&self) -> &[usize] { - &self.current_segment_ords[..] - } - */ - -/* -impl<'a, V> Streamer<'a> for FstMerger<'a, V> { - type Item = &'a Term; - - fn next(&'a mut self) -> Option { - if self.advance() { - Some(&self.current_term) - } else { - None - } - } -} -*/ - impl<'a> From<&'a [SegmentReader]> for FstMerger<'a, TermInfo> where TermInfo: BinarySerializable { fn from(segment_readers: &'a [SegmentReader]) -> FstMerger<'a, TermInfo> { FstMerger::new(segment_readers @@ -155,52 +126,67 @@ impl<'a> From<&'a [SegmentReader]> for FstMerger<'a, TermInfo> where TermInfo: B } } -// #[cfg(test)] -// mod tests { -// use super::*; -// use schema::{SchemaBuilder, Document, TEXT}; -// use core::Index; +impl<'a, V> Streamer<'a> for FstMerger<'a, V> where V: BinarySerializable { + type Item = &'a [u8]; -// #[test] -// fn test_term_iterator() { -// let mut schema_builder = SchemaBuilder::default(); -// let text_field = schema_builder.add_text_field("text", TEXT); -// let index = Index::create_in_ram(schema_builder.build()); -// { -// let mut index_writer = index.writer_with_num_threads(1, 40_000_000).unwrap(); -// { -// { -// let mut doc = Document::default(); -// doc.add_text(text_field, "a b d f"); -// index_writer.add_document(doc); -// } -// index_writer.commit().unwrap(); -// } -// { -// { -// let mut doc = Document::default(); -// doc.add_text(text_field, "a b c d f"); -// index_writer.add_document(doc); -// } -// index_writer.commit().unwrap(); -// } -// { -// { -// let mut doc = Document::default(); -// doc.add_text(text_field, "e f"); -// index_writer.add_document(doc); -// } -// index_writer.commit().unwrap(); -// } -// } -// index.load_searchers().unwrap(); -// let searcher = index.searcher(); -// let mut term_it = searcher.terms(); -// let mut terms = String::new(); -// while let Some(term) = term_it.next() { -// terms.push_str(term.text()); -// } -// assert_eq!(terms, "abcdef"); -// } + fn next(&'a mut self) -> Option { + if self.advance() { + Some(&self.current_streamers[0].streamer.key()) + } + else { + None + } + + } +} -// } +#[cfg(test)] +mod tests { + use super::*; + use schema::{Term, SchemaBuilder, Document, TEXT}; + use core::Index; + + #[test] + fn test_term_iterator() { + let mut schema_builder = SchemaBuilder::default(); + let text_field = schema_builder.add_text_field("text", TEXT); + let index = Index::create_in_ram(schema_builder.build()); + { + let mut index_writer = index.writer_with_num_threads(1, 40_000_000).unwrap(); + { + { + let mut doc = Document::default(); + doc.add_text(text_field, "a b d f"); + index_writer.add_document(doc); + } + index_writer.commit().unwrap(); + } + { + { + let mut doc = Document::default(); + doc.add_text(text_field, "a b c d f"); + index_writer.add_document(doc); + } + index_writer.commit().unwrap(); + } + { + { + let mut doc = Document::default(); + doc.add_text(text_field, "e f"); + index_writer.add_document(doc); + } + index_writer.commit().unwrap(); + } + } + index.load_searchers().unwrap(); + let searcher = index.searcher(); + let mut term_it = searcher.terms(); + let mut term_string = String::new(); + while term_it.advance() { + let term = Term::from_bytes(term_it.key()); + term_string.push_str(term.text()); + } + assert_eq!(&*term_string, "abcdef"); + } + +} From 0272167c2e1055dcb08fd65e6f8ea676ea585e36 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Thu, 18 May 2017 23:06:02 +0900 Subject: [PATCH 03/10] Code cleaning --- src/core/segment_reader.rs | 35 +++++++------------------ src/datastruct/fstmap/fstmap.rs | 9 ++++--- src/datastruct/fstmap/fstmerger.rs | 2 +- src/datastruct/fstmap/streamer.rs | 6 ++--- src/indexer/merger.rs | 5 +++- src/postings/segment_postings_option.rs | 14 +++++++++- src/schema/field_type.rs | 30 +++++++++++++++------ 7 files changed, 58 insertions(+), 43 deletions(-) diff --git a/src/core/segment_reader.rs b/src/core/segment_reader.rs index 8ca96cbf8..810e314d5 100644 --- a/src/core/segment_reader.rs +++ b/src/core/segment_reader.rs @@ -12,6 +12,7 @@ use schema::Document; use directory::ReadOnlySource; use DocId; use std::str; +use std::cmp; use postings::TermInfo; use datastruct::fstmap::FstMap; use std::sync::Arc; @@ -201,34 +202,16 @@ impl SegmentReader { let field = term.field(); let field_entry = self.schema.get_field_entry(field); let term_info = get!(self.get_term_info(term)); - let possible_option = match *field_entry.field_type() { - FieldType::Str(ref options) => { - let indexing_options = options.get_indexing_options(); - match option { - SegmentPostingsOption::NoFreq => SegmentPostingsOption::NoFreq, - SegmentPostingsOption::Freq => { - if indexing_options.is_termfreq_enabled() { - SegmentPostingsOption::Freq - } else { - SegmentPostingsOption::NoFreq - } - } - SegmentPostingsOption::FreqAndPositions => { - if indexing_options == TextIndexingOptions::TokenizedWithFreqAndPosition { - SegmentPostingsOption::FreqAndPositions - } else if indexing_options.is_termfreq_enabled() { - SegmentPostingsOption::Freq - } else { - SegmentPostingsOption::NoFreq - } - } - } - } - _ => { SegmentPostingsOption::NoFreq }, - }; - Some(self.read_postings_from_terminfo(&term_info, possible_option)) + let maximum_option = get!(field_entry.field_type().get_segment_postings_option()); + let best_effort_option = cmp::min(maximum_option, option); + Some(self.read_postings_from_terminfo(&term_info, best_effort_option)) } + + /// Returns a posting object given a `term_info`. + /// This method is for an advanced usage only. + /// + /// Most user should prefer using `read_postings` instead. pub fn read_postings_from_terminfo(&self, term_info: &TermInfo, option: SegmentPostingsOption) diff --git a/src/datastruct/fstmap/fstmap.rs b/src/datastruct/fstmap/fstmap.rs index 54a17ef20..58c02d490 100644 --- a/src/datastruct/fstmap/fstmap.rs +++ b/src/datastruct/fstmap/fstmap.rs @@ -129,7 +129,8 @@ mod tests { use super::*; use directory::{RAMDirectory, Directory}; use std::path::PathBuf; - + use fst::Streamer; + #[test] fn test_fstmap() { let mut directory = RAMDirectory::create(); @@ -146,10 +147,12 @@ mod tests { assert_eq!(fstmap.get("abc"), Some(34u32)); assert_eq!(fstmap.get("abcd"), Some(346u32)); let mut stream = fstmap.stream(); - assert!(stream.advance()); + assert_eq!(stream.next().unwrap(), "abc".as_bytes()); assert_eq!(stream.key(), "abc".as_bytes()); - assert!(stream.advance()); + assert_eq!(stream.value(), 34u32); + assert_eq!(stream.next().unwrap(), "abcd".as_bytes()); assert_eq!(stream.key(), "abcd".as_bytes()); + assert_eq!(stream.value(), 346u32); assert!(!stream.advance()); } diff --git a/src/datastruct/fstmap/fstmerger.rs b/src/datastruct/fstmap/fstmerger.rs index 687ba5127..918065ce6 100644 --- a/src/datastruct/fstmap/fstmerger.rs +++ b/src/datastruct/fstmap/fstmerger.rs @@ -142,7 +142,7 @@ impl<'a, V> Streamer<'a> for FstMerger<'a, V> where V: BinarySerializable { #[cfg(test)] mod tests { - use super::*; + use schema::{Term, SchemaBuilder, Document, TEXT}; use core::Index; diff --git a/src/datastruct/fstmap/streamer.rs b/src/datastruct/fstmap/streamer.rs index b91b13019..c243cecc2 100644 --- a/src/datastruct/fstmap/streamer.rs +++ b/src/datastruct/fstmap/streamer.rs @@ -59,11 +59,11 @@ pub struct FstMapStreamer<'a, V> where V: 'a + BinarySerializable { } -impl<'a, V> fst::Streamer<'a> for FstMapStreamer<'a, V> where V: 'a + BinarySerializable { +impl<'a, 'b, V> fst::Streamer<'b> for FstMapStreamer<'a, V> where V: 'a + BinarySerializable { - type Item = &'a [u8]; + type Item = &'b [u8]; - fn next<'b>(&'b mut self) -> Option<&'b [u8]> { + fn next(&'b mut self) -> Option<&'b [u8]> { if self.advance() { Some(&self.buffer) } diff --git a/src/indexer/merger.rs b/src/indexer/merger.rs index c94b21ea8..aa8e9dc94 100644 --- a/src/indexer/merger.rs +++ b/src/indexer/merger.rs @@ -232,7 +232,10 @@ impl IndexMerger { // we reached a new field. let field_entry = self.schema.get_field_entry(current_field); // ... set segment postings option the new field. - segment_postings_option = field_entry.field_type().get_segment_postings_option(); + segment_postings_option = field_entry + .field_type() + .get_segment_postings_option() + .expect("Encounterred a field that is not supposed to be indexed. Have you modified the index?"); last_field = Some(current_field); need_to_call_new_field = true; } diff --git a/src/postings/segment_postings_option.rs b/src/postings/segment_postings_option.rs index 53aac366a..925722c4a 100644 --- a/src/postings/segment_postings_option.rs +++ b/src/postings/segment_postings_option.rs @@ -6,7 +6,7 @@ /// avoid this extra cost when the information is not required. /// For instance, positions are useful when running phrase queries /// but useless in other queries. -#[derive(Clone, Copy, Debug)] +#[derive(Clone, Copy, Debug, PartialEq, PartialOrd, Ord, Eq)] pub enum SegmentPostingsOption { /// Only the doc ids are decoded NoFreq, @@ -15,3 +15,15 @@ pub enum SegmentPostingsOption { /// DocIds, term frequencies and positions will be decoded. FreqAndPositions, } + +#[cfg(test)] +mod tests { + + use super::SegmentPostingsOption; + + #[test] + fn test_cmp_segment_postings_option() { + assert!(SegmentPostingsOption::FreqAndPositions > SegmentPostingsOption::Freq); + assert!(SegmentPostingsOption::Freq > SegmentPostingsOption::NoFreq); + } +} diff --git a/src/schema/field_type.rs b/src/schema/field_type.rs index 999f919ba..26e5e85e9 100644 --- a/src/schema/field_type.rs +++ b/src/schema/field_type.rs @@ -40,23 +40,37 @@ impl FieldType { } } - pub fn get_segment_postings_option(&self) -> SegmentPostingsOption { + /// Given a field configuration, return the maximal possible + /// `SegmentPostingsOption` available. + /// + /// If the field is not indexed, then returns `None`. + pub fn get_segment_postings_option(&self) -> Option { match *self { FieldType::Str(ref text_options) => { match text_options.get_indexing_options() { + TextIndexingOptions::Untokenized => + Some(SegmentPostingsOption::NoFreq), TextIndexingOptions::TokenizedNoFreq => - SegmentPostingsOption::NoFreq, + Some(SegmentPostingsOption::NoFreq), TextIndexingOptions::TokenizedWithFreq => - SegmentPostingsOption::Freq, + Some(SegmentPostingsOption::Freq), TextIndexingOptions::TokenizedWithFreqAndPosition => - SegmentPostingsOption::FreqAndPositions, - _ => { - SegmentPostingsOption::NoFreq + Some(SegmentPostingsOption::FreqAndPositions), + TextIndexingOptions::Unindexed => { + None } } } - FieldType::U64(_) | - FieldType::I64(_) => SegmentPostingsOption::NoFreq, + FieldType::U64(ref int_options) | + FieldType::I64(ref int_options) => { + if int_options.is_indexed() { + Some(SegmentPostingsOption::NoFreq) + } + else { + None + } + + }, } } From d2926b6ee0d2c8b61f2d2f1b23168a9d06a9eaa4 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Thu, 18 May 2017 23:09:20 +0900 Subject: [PATCH 04/10] Format --- src/core/searcher.rs | 20 +- src/core/segment_reader.rs | 14 +- src/datastruct/fstmap/fstmap.rs | 19 +- src/datastruct/fstmap/fstmerger.rs | 67 +++--- src/datastruct/fstmap/mod.rs | 2 +- src/datastruct/fstmap/streamer.rs | 38 ++-- src/datastruct/stacker/hashmap.rs | 13 +- src/datastruct/stacker/heap.rs | 4 +- src/datastruct/stacker/mod.rs | 2 +- src/indexer/index_writer.rs | 2 +- src/indexer/merger.rs | 40 ++-- src/indexer/segment_writer.rs | 267 ++++++++++++------------ src/postings/postings_writer.rs | 114 +++++----- src/postings/recorder.rs | 4 +- src/postings/segment_postings_option.rs | 2 +- src/postings/serializer.rs | 2 +- src/query/boolean_query/mod.rs | 2 +- src/schema/field_entry.rs | 2 +- src/schema/field_type.rs | 22 +- src/schema/term.rs | 2 +- 20 files changed, 328 insertions(+), 310 deletions(-) diff --git a/src/core/searcher.rs b/src/core/searcher.rs index 9fade4cc5..45bdfccb7 100644 --- a/src/core/searcher.rs +++ b/src/core/searcher.rs @@ -65,16 +65,16 @@ impl Searcher { query.search(self, collector) } - /// Returns a Stream over all of the sorted unique terms of - /// the searcher. - /// - /// This includes all of the fields from all of the segment_readers. - /// See [TermIterator](struct.TermIterator.html). - /// - /// # Warning - /// This API is very likely to change in the future. - pub fn terms(&self) -> FstMerger { - FstMerger::from(self.segment_readers()) + /// Returns a Stream over all of the sorted unique terms of + /// the searcher. + /// + /// This includes all of the fields from all of the segment_readers. + /// See [TermIterator](struct.TermIterator.html). + /// + /// # Warning + /// This API is very likely to change in the future. + pub fn terms(&self) -> FstMerger { + FstMerger::from(self.segment_readers()) } } diff --git a/src/core/segment_reader.rs b/src/core/segment_reader.rs index 810e314d5..1be0451f6 100644 --- a/src/core/segment_reader.rs +++ b/src/core/segment_reader.rs @@ -213,18 +213,14 @@ impl SegmentReader { /// /// Most user should prefer using `read_postings` instead. pub fn read_postings_from_terminfo(&self, - term_info: &TermInfo, - option: SegmentPostingsOption) - -> SegmentPostings { + term_info: &TermInfo, + option: SegmentPostingsOption) + -> SegmentPostings { let offset = term_info.postings_offset as usize; let postings_data = &self.postings_data[offset..]; let freq_handler = match option { - SegmentPostingsOption::NoFreq => { - FreqHandler::new_without_freq() - }, - SegmentPostingsOption::Freq => { - FreqHandler::new_with_freq() - } + SegmentPostingsOption::NoFreq => FreqHandler::new_without_freq(), + SegmentPostingsOption::Freq => FreqHandler::new_with_freq(), SegmentPostingsOption::FreqAndPositions => { let offset = term_info.positions_offset as usize; let offseted_position_data = &self.positions_data[offset..]; diff --git a/src/datastruct/fstmap/fstmap.rs b/src/datastruct/fstmap/fstmap.rs index 58c02d490..f6894e1bd 100644 --- a/src/datastruct/fstmap/fstmap.rs +++ b/src/datastruct/fstmap/fstmap.rs @@ -36,8 +36,8 @@ impl FstMapBuilder { /// in a nice way. pub fn insert_key(&mut self, key: &[u8]) -> io::Result<()> { self.fst_builder - .insert(key, self.data.len() as u64) - .map_err(convert_fst_error)?; + .insert(key, self.data.len() as u64) + .map_err(convert_fst_error)?; Ok(()) } @@ -50,8 +50,8 @@ impl FstMapBuilder { #[cfg(test)] pub fn insert(&mut self, key: &[u8], value: &V) -> io::Result<()> { self.fst_builder - .insert(key, self.data.len() as u64) - .map_err(convert_fst_error)?; + .insert(key, self.data.len() as u64) + .map_err(convert_fst_error)?; value.serialize(&mut self.data)?; Ok(()) } @@ -77,7 +77,7 @@ fn open_fst_index(source: ReadOnlySource) -> io::Result { Ok(fst::Map::from(match source { ReadOnlySource::Anonymous(data) => { Fst::from_shared_bytes(data.data, data.start, data.len) - .map_err(convert_fst_error)? + .map_err(convert_fst_error)? } ReadOnlySource::Mmap(mmap_readonly) => { Fst::from_mmap(mmap_readonly).map_err(convert_fst_error)? @@ -85,8 +85,9 @@ fn open_fst_index(source: ReadOnlySource) -> io::Result { })) } -impl FstMap where V: BinarySerializable { - +impl FstMap + where V: BinarySerializable +{ pub fn from_source(source: ReadOnlySource) -> io::Result> { let total_len = source.len(); let length_offset = total_len - 4; @@ -100,7 +101,7 @@ impl FstMap where V: BinarySerializable { fst_index: fst_index, values_mmap: values_source, _phantom_: PhantomData, - }) + }) } pub(crate) fn read_value(&self, offset: u64) -> V { @@ -130,7 +131,7 @@ mod tests { use directory::{RAMDirectory, Directory}; use std::path::PathBuf; use fst::Streamer; - + #[test] fn test_fstmap() { let mut directory = RAMDirectory::create(); diff --git a/src/datastruct/fstmap/fstmerger.rs b/src/datastruct/fstmap/fstmerger.rs index 918065ce6..b0e1c30b6 100644 --- a/src/datastruct/fstmap/fstmerger.rs +++ b/src/datastruct/fstmap/fstmerger.rs @@ -6,27 +6,34 @@ use postings::TermInfo; use std::cmp::Ordering; use fst::Streamer; -pub struct HeapItem<'a, V> where V: 'a + BinarySerializable { +pub struct HeapItem<'a, V> + where V: 'a + BinarySerializable +{ pub streamer: FstMapStreamer<'a, V>, pub segment_ord: usize, } -impl<'a, V> PartialEq for HeapItem<'a, V> where V: 'a + BinarySerializable { +impl<'a, V> PartialEq for HeapItem<'a, V> + where V: 'a + BinarySerializable +{ fn eq(&self, other: &Self) -> bool { self.segment_ord == other.segment_ord } } -impl<'a, V> Eq for HeapItem<'a, V> where V: 'a + BinarySerializable { -} +impl<'a, V> Eq for HeapItem<'a, V> where V: 'a + BinarySerializable {} -impl<'a, V> PartialOrd for HeapItem<'a, V> where V: 'a + BinarySerializable { +impl<'a, V> PartialOrd for HeapItem<'a, V> + where V: 'a + BinarySerializable +{ fn partial_cmp(&self, other: &HeapItem<'a, V>) -> Option { Some(self.cmp(other)) } } -impl<'a, V> Ord for HeapItem<'a, V> where V: 'a + BinarySerializable { +impl<'a, V> Ord for HeapItem<'a, V> + where V: 'a + BinarySerializable +{ fn cmp(&self, other: &HeapItem<'a, V>) -> Ordering { (&other.streamer.key(), &other.segment_ord).cmp(&(&self.streamer.key(), &self.segment_ord)) } @@ -39,12 +46,16 @@ impl<'a, V> Ord for HeapItem<'a, V> where V: 'a + BinarySerializable { /// - the term /// - a slice with the ordinal of the segments containing /// the terms. -pub struct FstMerger<'a, V> where V: 'a + BinarySerializable { +pub struct FstMerger<'a, V> + where V: 'a + BinarySerializable +{ heap: BinaryHeap>, current_streamers: Vec>, } -impl<'a, V> FstMerger<'a, V> where V: 'a + BinarySerializable { +impl<'a, V> FstMerger<'a, V> + where V: 'a + BinarySerializable +{ fn new(streams: Vec>) -> FstMerger<'a, V> { FstMerger { heap: BinaryHeap::new(), @@ -52,11 +63,11 @@ impl<'a, V> FstMerger<'a, V> where V: 'a + BinarySerializable { .into_iter() .enumerate() .map(|(ord, streamer)| { - HeapItem { - streamer: streamer, - segment_ord: ord, - } - }) + HeapItem { + streamer: streamer, + segment_ord: ord, + } + }) .collect(), } } @@ -83,9 +94,10 @@ impl<'a, V> FstMerger<'a, V> where V: 'a + BinarySerializable { if let Some(next_streamer) = self.heap.peek() { if self.current_streamers[0].streamer.key() != next_streamer.streamer.key() { break; - } - } - else { break; } // no more streamer. + } + } else { + break; + } // no more streamer. let next_heap_it = self.heap.pop().unwrap(); // safe : we peeked beforehand self.current_streamers.push(next_heap_it); } @@ -94,7 +106,7 @@ impl<'a, V> FstMerger<'a, V> where V: 'a + BinarySerializable { false } } - + /// Returns the current term. /// /// This method may be called @@ -117,32 +129,35 @@ impl<'a, V> FstMerger<'a, V> where V: 'a + BinarySerializable { -impl<'a> From<&'a [SegmentReader]> for FstMerger<'a, TermInfo> where TermInfo: BinarySerializable { +impl<'a> From<&'a [SegmentReader]> for FstMerger<'a, TermInfo> + where TermInfo: BinarySerializable +{ fn from(segment_readers: &'a [SegmentReader]) -> FstMerger<'a, TermInfo> { FstMerger::new(segment_readers - .iter() - .map(|reader| reader.term_infos().stream()) - .collect() ) + .iter() + .map(|reader| reader.term_infos().stream()) + .collect()) } } -impl<'a, V> Streamer<'a> for FstMerger<'a, V> where V: BinarySerializable { +impl<'a, V> Streamer<'a> for FstMerger<'a, V> + where V: BinarySerializable +{ type Item = &'a [u8]; fn next(&'a mut self) -> Option { if self.advance() { Some(&self.current_streamers[0].streamer.key()) - } - else { + } else { None } - + } } #[cfg(test)] mod tests { - + use schema::{Term, SchemaBuilder, Document, TEXT}; use core::Index; diff --git a/src/datastruct/fstmap/mod.rs b/src/datastruct/fstmap/mod.rs index 701b5c47b..62d1a786d 100644 --- a/src/datastruct/fstmap/mod.rs +++ b/src/datastruct/fstmap/mod.rs @@ -6,4 +6,4 @@ pub use self::fstmap::FstMap; pub use self::fstmap::FstMapBuilder; pub use self::streamer::FstMapStreamer; pub use self::streamer::FstMapStreamerBuilder; -pub use self::fstmerger::FstMerger; \ No newline at end of file +pub use self::fstmerger::FstMerger; diff --git a/src/datastruct/fstmap/streamer.rs b/src/datastruct/fstmap/streamer.rs index c243cecc2..ba8edcccb 100644 --- a/src/datastruct/fstmap/streamer.rs +++ b/src/datastruct/fstmap/streamer.rs @@ -3,18 +3,21 @@ use fst::map::{StreamBuilder, Stream}; use common::BinarySerializable; use super::FstMap; -pub struct FstMapStreamerBuilder<'a, V> where V: 'a + BinarySerializable { +pub struct FstMapStreamerBuilder<'a, V> + where V: 'a + BinarySerializable +{ fst_map: &'a FstMap, stream_builder: StreamBuilder<'a>, } -impl<'a, V> FstMapStreamerBuilder<'a, V> where V: 'a + BinarySerializable { - +impl<'a, V> FstMapStreamerBuilder<'a, V> + where V: 'a + BinarySerializable +{ pub fn ge>(mut self, bound: T) -> Self { self.stream_builder = self.stream_builder.ge(bound); self } - + pub fn gt>(mut self, bound: T) -> Self { self.stream_builder = self.stream_builder.gt(bound); self @@ -39,7 +42,9 @@ impl<'a, V> FstMapStreamerBuilder<'a, V> where V: 'a + BinarySerializable { } } - pub fn new(fst_map: &'a FstMap, stream_builder: StreamBuilder<'a>) -> FstMapStreamerBuilder<'a, V> { + pub fn new(fst_map: &'a FstMap, + stream_builder: StreamBuilder<'a>) + -> FstMapStreamerBuilder<'a, V> { FstMapStreamerBuilder { fst_map: fst_map, stream_builder: stream_builder, @@ -51,7 +56,9 @@ impl<'a, V> FstMapStreamerBuilder<'a, V> where V: 'a + BinarySerializable { -pub struct FstMapStreamer<'a, V> where V: 'a + BinarySerializable { +pub struct FstMapStreamer<'a, V> + where V: 'a + BinarySerializable +{ fst_map: &'a FstMap, stream: Stream<'a>, offset: u64, @@ -59,30 +66,30 @@ pub struct FstMapStreamer<'a, V> where V: 'a + BinarySerializable { } -impl<'a, 'b, V> fst::Streamer<'b> for FstMapStreamer<'a, V> where V: 'a + BinarySerializable { - +impl<'a, 'b, V> fst::Streamer<'b> for FstMapStreamer<'a, V> + where V: 'a + BinarySerializable +{ type Item = &'b [u8]; - + fn next(&'b mut self) -> Option<&'b [u8]> { if self.advance() { Some(&self.buffer) - } - else { + } else { None } } } -impl<'a, V> FstMapStreamer<'a, V> where V: 'a + BinarySerializable { - +impl<'a, V> FstMapStreamer<'a, V> + where V: 'a + BinarySerializable +{ pub fn advance(&mut self) -> bool { if let Some((term, offset)) = self.stream.next() { self.buffer.clear(); self.buffer.extend_from_slice(term); self.offset = offset; true - } - else { + } else { false } } @@ -95,4 +102,3 @@ impl<'a, V> FstMapStreamer<'a, V> where V: 'a + BinarySerializable { self.fst_map.read_value(self.offset) } } - diff --git a/src/datastruct/stacker/hashmap.rs b/src/datastruct/stacker/hashmap.rs index 72f584919..135d14d41 100644 --- a/src/datastruct/stacker/hashmap.rs +++ b/src/datastruct/stacker/hashmap.rs @@ -86,7 +86,6 @@ impl QuadraticProbing { impl<'a> HashMap<'a> { - pub fn new(num_bucket_power_of_2: usize, heap: &'a Heap) -> HashMap<'a> { let table_size = 1 << num_bucket_power_of_2; let table: Vec = iter::repeat(KeyValue::default()).take(table_size).collect(); @@ -118,18 +117,18 @@ impl<'a> HashMap<'a> { }; addr } - - pub fn iter<'b: 'a>(&'b self,) -> impl Iterator + 'b { + + pub fn iter<'b: 'a>(&'b self) -> impl Iterator + 'b { let heap: &'a Heap = self.heap; let table: &'b [KeyValue] = &self.table; self.occupied .iter() .cloned() .map(move |bucket: usize| { - let kv = table[bucket]; - let addr = kv.value_addr; - (heap.get_slice(kv.key), addr) - }) + let kv = table[bucket]; + let addr = kv.value_addr; + (heap.get_slice(kv.key), addr) + }) } pub fn get_or_create, V: HeapAllocable>(&mut self, key: S) -> &mut V { diff --git a/src/datastruct/stacker/heap.rs b/src/datastruct/stacker/heap.rs index 8cee34710..b4511fb35 100644 --- a/src/datastruct/stacker/heap.rs +++ b/src/datastruct/stacker/heap.rs @@ -115,10 +115,10 @@ impl InnerHeap { pub fn capacity(&self) -> u32 { self.buffer.len() as u32 } - + // Returns the number of free bytes. If the buffer // has reached it's capacity and overflowed to another buffer, return 0. - pub fn num_free_bytes(&self,) -> u32 { + pub fn num_free_bytes(&self) -> u32 { if self.has_been_resized { 0u32 } else { diff --git a/src/datastruct/stacker/mod.rs b/src/datastruct/stacker/mod.rs index ebaecf5ed..93e29ae52 100644 --- a/src/datastruct/stacker/mod.rs +++ b/src/datastruct/stacker/mod.rs @@ -17,7 +17,7 @@ fn test_unrolled_linked_list() { let mut ks: Vec = (1..5).map(|k| k * 100).collect(); ks.push(2); ks.push(3); - for k in (1..5).map(|k| k * 100) { + for k in (1..5).map(|k| k * 100) { let mut hashmap: HashMap = HashMap::new(10, &heap); for j in 0..k { for i in 0..500 { diff --git a/src/indexer/index_writer.rs b/src/indexer/index_writer.rs index 2247148f2..6a038812c 100644 --- a/src/indexer/index_writer.rs +++ b/src/indexer/index_writer.rs @@ -261,7 +261,7 @@ fn index_documents(heap: &mut Heap, for doc in document_iterator { try!(segment_writer.add_document(&doc, schema)); // There is two possible conditions to close the segment. - // One is the memory arena dedicated to the segment is + // One is the memory arena dedicated to the segment is // getting full. if segment_writer.is_buffer_full() { info!("Buffer limit reached, flushing segment with maxdoc={}.", diff --git a/src/indexer/merger.rs b/src/indexer/merger.rs index aa8e9dc94..8036c4977 100644 --- a/src/indexer/merger.rs +++ b/src/indexer/merger.rs @@ -131,7 +131,8 @@ impl IndexMerger { // used both to merge field norms and regular u64 fast fields. fn generic_write_fast_field(&self, fields: Vec, - field_reader_extractor: &Fn(&SegmentReader, Field) -> Option, + field_reader_extractor: &Fn(&SegmentReader, Field) + -> Option, fast_field_serializer: &mut FastFieldSerializer) -> Result<()> { @@ -163,7 +164,7 @@ impl IndexMerger { } } } - + if u64_readers.is_empty() { // we have actually zero documents. min_val = 0; @@ -172,7 +173,8 @@ impl IndexMerger { assert!(min_val <= max_val); - fast_field_serializer.new_u64_fast_field(field, min_val, max_val)?; + fast_field_serializer + .new_u64_fast_field(field, min_val, max_val)?; for (max_doc, u64_reader, delete_bitset) in u64_readers { for doc_id in 0..max_doc { if !delete_bitset.is_deleted(doc_id) { @@ -188,7 +190,7 @@ impl IndexMerger { } fn write_postings(&self, postings_serializer: &mut PostingsSerializer) -> Result<()> { - + let mut merged_terms = FstMerger::from(&self.readers[..]); let mut delta_position_computer = DeltaPositionComputer::new(); @@ -227,7 +229,7 @@ impl IndexMerger { // ... let term_bytes = merged_terms.key(); let current_field = schema::extract_field_from_term_bytes(term_bytes); - + if last_field != Some(current_field) { // we reached a new field. let field_entry = self.schema.get_field_entry(current_field); @@ -235,7 +237,8 @@ impl IndexMerger { segment_postings_option = field_entry .field_type() .get_segment_postings_option() - .expect("Encounterred a field that is not supposed to be indexed. Have you modified the index?"); + .expect("Encounterred a field that is not supposed to be + indexed. Have you modified the index?"); last_field = Some(current_field); need_to_call_new_field = true; } @@ -248,17 +251,17 @@ impl IndexMerger { let segment_ord = heap_item.segment_ord; let term_info = heap_item.streamer.value(); let segment_reader = &self.readers[heap_item.segment_ord]; - let mut segment_postings = segment_reader - .read_postings_from_terminfo(&term_info, segment_postings_option); + let mut segment_postings = + segment_reader + .read_postings_from_terminfo(&term_info, segment_postings_option); if segment_postings.advance() { Some((segment_ord, segment_postings)) - } - else { + } else { None } }) .collect(); - + if segment_postings.is_empty() { continue; } @@ -270,9 +273,9 @@ impl IndexMerger { } postings_serializer.new_term(term_bytes)?; - + // We can now serialize this postings, by pushing each document to the - // postings serializer. + // postings serializer. for (segment_ord, mut segment_postings) in segment_postings { let old_to_new_doc_id = &merged_doc_id_map[segment_ord]; @@ -281,10 +284,13 @@ impl IndexMerger { old_to_new_doc_id[segment_postings.doc() as usize] { // we make sure to only write the term iff // there is at least one document. - let delta_positions: &[u32] = delta_position_computer.compute_delta_positions(segment_postings.positions()); - postings_serializer.write_doc(remapped_doc_id, - segment_postings.term_freq(), - delta_positions)?; + let delta_positions: &[u32] = + delta_position_computer + .compute_delta_positions(segment_postings.positions()); + postings_serializer + .write_doc(remapped_doc_id, + segment_postings.term_freq(), + delta_positions)?; } if !segment_postings.advance() { break; diff --git a/src/indexer/segment_writer.rs b/src/indexer/segment_writer.rs index c1e4df0c5..ffaae2cca 100644 --- a/src/indexer/segment_writer.rs +++ b/src/indexer/segment_writer.rs @@ -24,11 +24,11 @@ use postings::MultiFieldPostingsWriter; pub struct SegmentWriter<'a> { heap: &'a Heap, max_doc: DocId, - multifield_postings: MultiFieldPostingsWriter<'a>, - segment_serializer: SegmentSerializer, - fast_field_writers: FastFieldsWriter, - fieldnorms_writer: FastFieldsWriter, - doc_opstamps: Vec, + multifield_postings: MultiFieldPostingsWriter<'a>, + segment_serializer: SegmentSerializer, + fast_field_writers: FastFieldsWriter, + fieldnorms_writer: FastFieldsWriter, + doc_opstamps: Vec, } @@ -45,120 +45,121 @@ fn create_fieldnorms_writer(schema: &Schema) -> FastFieldsWriter { impl<'a> SegmentWriter<'a> { - - /// Creates a new `SegmentWriter` - /// - /// The arguments are defined as follows - /// - /// - heap: most of the segment writer data (terms, and postings lists recorders) - /// is stored in a user-defined heap object. This makes it possible for the user to define - /// the flushing behavior as a buffer limit - /// - segment: The segment being written - /// - schema - pub fn for_segment(heap: &'a Heap, - mut segment: Segment, - schema: &Schema) -> Result> { - let segment_serializer = try!(SegmentSerializer::for_segment(&mut segment)); - let multifield_postings = MultiFieldPostingsWriter::new(schema, heap); - Ok(SegmentWriter { - heap: heap, - max_doc: 0, - multifield_postings: multifield_postings, - fieldnorms_writer: create_fieldnorms_writer(schema), - segment_serializer: segment_serializer, - fast_field_writers: FastFieldsWriter::from_schema(schema), - doc_opstamps: Vec::with_capacity(1_000), - }) - } - - /// Lay on disk the current content of the `SegmentWriter` - /// - /// Finalize consumes the `SegmentWriter`, so that it cannot - /// be used afterwards. - pub fn finalize(self) -> Result> { - write(&self.multifield_postings, - &self.fast_field_writers, - &self.fieldnorms_writer, - self.segment_serializer)?; - Ok(self.doc_opstamps) - } - - /// Returns true iff the segment writer's buffer has reached capacity. - /// - /// The limit is defined as `the user defined heap size - an arbitrary margin of 10MB` - /// The `Segment` is `finalize`d when the buffer gets full. - /// - /// Because, we cannot cut through a document, the margin is there to ensure that we rarely - /// exceeds the heap size. - pub fn is_buffer_full(&self,) -> bool { - self.heap.num_free_bytes() <= MARGIN_IN_BYTES - } - - - /// Return true if the term dictionary hashmap is reaching capacity. - /// It is one of the condition that triggers a `SegmentWriter` to - /// be finalized. - pub(crate) fn is_termdic_saturated(&self,) -> bool { - self.multifield_postings.is_termdic_saturated() - } - + /// Creates a new `SegmentWriter` + /// + /// The arguments are defined as follows + /// + /// - heap: most of the segment writer data (terms, and postings lists recorders) + /// is stored in a user-defined heap object. This makes it possible for the user to define + /// the flushing behavior as a buffer limit + /// - segment: The segment being written + /// - schema + pub fn for_segment(heap: &'a Heap, + mut segment: Segment, + schema: &Schema) + -> Result> { + let segment_serializer = try!(SegmentSerializer::for_segment(&mut segment)); + let multifield_postings = MultiFieldPostingsWriter::new(schema, heap); + Ok(SegmentWriter { + heap: heap, + max_doc: 0, + multifield_postings: multifield_postings, + fieldnorms_writer: create_fieldnorms_writer(schema), + segment_serializer: segment_serializer, + fast_field_writers: FastFieldsWriter::from_schema(schema), + doc_opstamps: Vec::with_capacity(1_000), + }) + } - /// Indexes a new document - /// - /// As a user, you should rather use `IndexWriter`'s add_document. - pub fn add_document(&mut self, add_operation: &AddOperation, schema: &Schema) -> io::Result<()> { + /// Lay on disk the current content of the `SegmentWriter` + /// + /// Finalize consumes the `SegmentWriter`, so that it cannot + /// be used afterwards. + pub fn finalize(self) -> Result> { + write(&self.multifield_postings, + &self.fast_field_writers, + &self.fieldnorms_writer, + self.segment_serializer)?; + Ok(self.doc_opstamps) + } + + /// Returns true iff the segment writer's buffer has reached capacity. + /// + /// The limit is defined as `the user defined heap size - an arbitrary margin of 10MB` + /// The `Segment` is `finalize`d when the buffer gets full. + /// + /// Because, we cannot cut through a document, the margin is there to ensure that we rarely + /// exceeds the heap size. + pub fn is_buffer_full(&self) -> bool { + self.heap.num_free_bytes() <= MARGIN_IN_BYTES + } + + + /// Return true if the term dictionary hashmap is reaching capacity. + /// It is one of the condition that triggers a `SegmentWriter` to + /// be finalized. + pub(crate) fn is_termdic_saturated(&self) -> bool { + self.multifield_postings.is_termdic_saturated() + } + + + /// Indexes a new document + /// + /// As a user, you should rather use `IndexWriter`'s add_document. + pub fn add_document(&mut self, + add_operation: &AddOperation, + schema: &Schema) + -> io::Result<()> { let doc_id = self.max_doc; let doc = &add_operation.document; self.doc_opstamps.push(add_operation.opstamp); for (field, field_values) in doc.get_sorted_field_values() { - let field_options = schema.get_field_entry(field); - match *field_options.field_type() { - FieldType::Str(ref text_options) => { - let num_tokens: u32 = - if text_options.get_indexing_options().is_tokenized() { - self.multifield_postings.index_text(doc_id, field, &field_values) - } - else { - let num_field_values = field_values.len() as u32; - for field_value in field_values { - let term = Term::from_field_text(field, field_value.value().text()); - self.multifield_postings.suscribe(doc_id, &term); - } - num_field_values - }; - self.fieldnorms_writer - .get_field_writer(field) - .map(|field_norms_writer| { - field_norms_writer.add_val(num_tokens as u64) - }); - } - FieldType::U64(ref int_option) => { - if int_option.is_indexed() { - for field_value in field_values { - let term = Term::from_field_u64(field_value.field(), field_value.value().u64_value()); - self.multifield_postings.suscribe(doc_id, &term); - } - } - } - FieldType::I64(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().i64_value()); - self.multifield_postings.suscribe(doc_id, &term); - } - } - } - } - } - self.fieldnorms_writer.fill_val_up_to(doc_id); - self.fast_field_writers.add_document(doc); - let stored_fieldvalues: Vec<&FieldValue> = doc - .field_values() - .iter() - .filter(|field_value| schema.get_field_entry(field_value.field()).is_stored()) - .collect(); - let doc_writer = self.segment_serializer.get_store_writer(); - try!(doc_writer.store(&stored_fieldvalues)); + let field_options = schema.get_field_entry(field); + match *field_options.field_type() { + FieldType::Str(ref text_options) => { + let num_tokens: u32 = if text_options.get_indexing_options().is_tokenized() { + self.multifield_postings + .index_text(doc_id, field, &field_values) + } else { + let num_field_values = field_values.len() as u32; + for field_value in field_values { + let term = Term::from_field_text(field, field_value.value().text()); + self.multifield_postings.suscribe(doc_id, &term); + } + num_field_values + }; + self.fieldnorms_writer + .get_field_writer(field) + .map(|field_norms_writer| field_norms_writer.add_val(num_tokens as u64)); + } + FieldType::U64(ref int_option) => { + if int_option.is_indexed() { + for field_value in field_values { + let term = Term::from_field_u64(field_value.field(), + field_value.value().u64_value()); + self.multifield_postings.suscribe(doc_id, &term); + } + } + } + FieldType::I64(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().i64_value()); + self.multifield_postings.suscribe(doc_id, &term); + } + } + } + } + } + self.fieldnorms_writer.fill_val_up_to(doc_id); + self.fast_field_writers.add_document(doc); + let stored_fieldvalues: Vec<&FieldValue> = doc.field_values() + .iter() + .filter(|field_value| schema.get_field_entry(field_value.field()).is_stored()) + .collect(); + let doc_writer = self.segment_serializer.get_store_writer(); + try!(doc_writer.store(&stored_fieldvalues)); self.max_doc += 1; Ok(()) } @@ -186,27 +187,27 @@ impl<'a> SegmentWriter<'a> { } // This method is used as a trick to workaround the borrow checker -fn write( - multifield_postings: &MultiFieldPostingsWriter, - fast_field_writers: &FastFieldsWriter, - fieldnorms_writer: &FastFieldsWriter, - mut serializer: SegmentSerializer) -> Result<()> { - - try!(multifield_postings.serialize(serializer.get_postings_serializer())); - try!(fast_field_writers.serialize(serializer.get_fast_field_serializer())); - try!(fieldnorms_writer.serialize(serializer.get_fieldnorms_serializer())); - try!(serializer.close()); +fn write(multifield_postings: &MultiFieldPostingsWriter, + fast_field_writers: &FastFieldsWriter, + fieldnorms_writer: &FastFieldsWriter, + mut serializer: SegmentSerializer) + -> Result<()> { - Ok(()) + try!(multifield_postings.serialize(serializer.get_postings_serializer())); + try!(fast_field_writers.serialize(serializer.get_fast_field_serializer())); + try!(fieldnorms_writer.serialize(serializer.get_fieldnorms_serializer())); + try!(serializer.close()); + + Ok(()) } impl<'a> SerializableSegment for SegmentWriter<'a> { - fn write(&self, serializer: SegmentSerializer) -> Result { - let max_doc = self.max_doc; - write(&self.multifield_postings, - &self.fast_field_writers, - &self.fieldnorms_writer, - serializer)?; - Ok(max_doc) - } + fn write(&self, serializer: SegmentSerializer) -> Result { + let max_doc = self.max_doc; + write(&self.multifield_postings, + &self.fast_field_writers, + &self.fieldnorms_writer, + serializer)?; + Ok(max_doc) + } } diff --git a/src/postings/postings_writer.rs b/src/postings/postings_writer.rs index d2acf3fe1..018e22daf 100644 --- a/src/postings/postings_writer.rs +++ b/src/postings/postings_writer.rs @@ -17,25 +17,24 @@ use schema::FieldEntry; use schema::FieldType; use schema::TextIndexingOptions; -fn posting_from_field_entry<'a>(field_entry: &FieldEntry, heap: &'a Heap) -> Box { - match *field_entry.field_type() { - FieldType::Str(ref text_options) => { - match text_options.get_indexing_options() { - TextIndexingOptions::TokenizedWithFreq => { - SpecializedPostingsWriter::::new_boxed(heap) - } - TextIndexingOptions::TokenizedWithFreqAndPosition => { - SpecializedPostingsWriter::::new_boxed(heap) - } - _ => { - SpecializedPostingsWriter::::new_boxed(heap) - } - } - } - FieldType::U64(_) | FieldType::I64(_) => { - SpecializedPostingsWriter::::new_boxed(heap) - } - } +fn posting_from_field_entry<'a>(field_entry: &FieldEntry, + heap: &'a Heap) + -> Box { + match *field_entry.field_type() { + FieldType::Str(ref text_options) => { + match text_options.get_indexing_options() { + TextIndexingOptions::TokenizedWithFreq => { + SpecializedPostingsWriter::::new_boxed(heap) + } + TextIndexingOptions::TokenizedWithFreqAndPosition => { + SpecializedPostingsWriter::::new_boxed(heap) + } + _ => SpecializedPostingsWriter::::new_boxed(heap), + } + } + FieldType::U64(_) | + FieldType::I64(_) => SpecializedPostingsWriter::::new_boxed(heap), + } } @@ -46,15 +45,14 @@ pub struct MultiFieldPostingsWriter<'a> { } impl<'a> MultiFieldPostingsWriter<'a> { - /// Create a new `MultiFieldPostingsWriter` given /// a schema and a heap. pub fn new(schema: &Schema, heap: &'a Heap) -> MultiFieldPostingsWriter<'a> { let capacity = heap.capacity(); let hashmap_size = hashmap_size_in_bits(capacity); let term_index = HashMap::new(hashmap_size, heap); - - let mut per_field_postings_writers: Vec<_> = vec!(); + + let mut per_field_postings_writers: Vec<_> = vec![]; for field_entry in schema.fields() { let field_entry = posting_from_field_entry(field_entry, heap); per_field_postings_writers.push(field_entry); @@ -62,15 +60,11 @@ impl<'a> MultiFieldPostingsWriter<'a> { MultiFieldPostingsWriter { heap: heap, term_index: term_index, - per_field_postings_writers: per_field_postings_writers + per_field_postings_writers: per_field_postings_writers, } } - pub fn index_text(&mut self, - doc: DocId, - field: Field, - field_values: &[&FieldValue]) - -> u32 { + pub fn index_text(&mut self, doc: DocId, field: Field, field_values: &[&FieldValue]) -> u32 { let postings_writer = self.per_field_postings_writers[field.0 as usize].deref_mut(); postings_writer.index_text(&mut self.term_index, doc, field, field_values, self.heap) } @@ -82,23 +76,19 @@ impl<'a> MultiFieldPostingsWriter<'a> { /// Serialize the inverted index. - /// It pushes all term, one field at a time, towards the + /// It pushes all term, one field at a time, towards the /// postings serializer. pub fn serialize(&self, serializer: &mut PostingsSerializer) -> Result<()> { - let mut term_offsets: Vec<(&[u8], u32)> = self.term_index - .iter() - .collect(); + let mut term_offsets: Vec<(&[u8], u32)> = self.term_index.iter().collect(); term_offsets.sort_by_key(|&(k, _v)| k); - - let mut offsets: Vec<(Field, usize)> = vec!(); + + let mut offsets: Vec<(Field, usize)> = vec![]; let term_offsets_it = term_offsets .iter() .cloned() - .map(|(key, _)| { - extract_field_from_term_bytes(key) - }) + .map(|(key, _)| extract_field_from_term_bytes(key)) .enumerate(); - + let mut prev_field = Field(u32::max_value()); for (offset, field) in term_offsets_it { if field != prev_field { @@ -109,13 +99,10 @@ impl<'a> MultiFieldPostingsWriter<'a> { 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 (_, stop) = offsets[i + 1]; let postings_writer = &self.per_field_postings_writers[field.0 as usize]; - postings_writer.serialize( - field, - &term_offsets[start..stop], - serializer, - self.heap)?; + postings_writer + .serialize(field, &term_offsets[start..stop], serializer, self.heap)?; } Ok(()) } @@ -139,15 +126,25 @@ pub trait PostingsWriter { /// * term - the term /// * heap - heap used to store the postings informations as well as the terms /// in the hashmap. - fn suscribe(&mut self, term_index: &mut HashMap, doc: DocId, pos: u32, term: &Term, heap: &Heap); + fn suscribe(&mut self, + term_index: &mut HashMap, + doc: DocId, + pos: u32, + term: &Term, + heap: &Heap); /// Serializes the postings on disk. /// The actual serialization format is handled by the `PostingsSerializer`. - fn serialize(&self, field: Field, term_addrs: &[(&[u8], u32)], serializer: &mut PostingsSerializer, heap: &Heap) -> io::Result<()>; - + fn serialize(&self, + field: Field, + term_addrs: &[(&[u8], u32)], + serializer: &mut PostingsSerializer, + heap: &Heap) + -> io::Result<()>; + /// Tokenize a text and suscribe all of its token. fn index_text<'a>(&mut self, - term_index: &mut HashMap, + term_index: &mut HashMap, doc_id: DocId, field: Field, field_values: &[&'a FieldValue], @@ -212,8 +209,12 @@ impl<'a, Rec: Recorder + 'static> SpecializedPostingsWriter<'a, Rec> { } impl<'a, Rec: Recorder + 'static> PostingsWriter for SpecializedPostingsWriter<'a, Rec> { - - fn suscribe(&mut self, term_index: &mut HashMap, doc: DocId, position: u32, term: &Term, heap: &Heap) { + fn suscribe(&mut self, + term_index: &mut HashMap, + doc: DocId, + position: u32, + term: &Term, + heap: &Heap) { let recorder: &mut Rec = term_index.get_or_create(term); let current_doc = recorder.current_doc(); if current_doc != doc { @@ -224,19 +225,20 @@ impl<'a, Rec: Recorder + 'static> PostingsWriter for SpecializedPostingsWriter<' } recorder.record_position(position, heap); } - + fn serialize(&self, - field: Field, - term_addrs: &[(&[u8], u32)], - serializer: &mut PostingsSerializer, - heap: &Heap) -> io::Result<()> { + field: Field, + term_addrs: &[(&[u8], u32)], + serializer: &mut PostingsSerializer, + heap: &Heap) + -> io::Result<()> { serializer.new_field(field); for &(term_bytes, addr) in term_addrs { let recorder: &mut Rec = self.heap.get_mut_ref(addr); try!(serializer.new_term(term_bytes)); try!(recorder.serialize(addr, serializer, heap)); try!(serializer.close_term()); - } + } Ok(()) } } diff --git a/src/postings/recorder.rs b/src/postings/recorder.rs index 8d9937615..c340d13fd 100644 --- a/src/postings/recorder.rs +++ b/src/postings/recorder.rs @@ -96,7 +96,6 @@ impl HeapAllocable for TermFrequencyRecorder { } impl Recorder for TermFrequencyRecorder { - fn current_doc(&self) -> DocId { self.current_doc } @@ -127,7 +126,7 @@ impl Recorder for TermFrequencyRecorder { let mut doc_iter = self.stack .iter(self_addr, heap) .chain(Some(self.current_tf).into_iter()); - + while let Some(doc) = doc_iter.next() { let term_freq = doc_iter .next() @@ -136,7 +135,6 @@ impl Recorder for TermFrequencyRecorder { } Ok(()) } - } /// Recorder encoding term frequencies as well as positions. diff --git a/src/postings/segment_postings_option.rs b/src/postings/segment_postings_option.rs index 925722c4a..51a07bb0b 100644 --- a/src/postings/segment_postings_option.rs +++ b/src/postings/segment_postings_option.rs @@ -18,7 +18,7 @@ pub enum SegmentPostingsOption { #[cfg(test)] mod tests { - + use super::SegmentPostingsOption; #[test] diff --git a/src/postings/serializer.rs b/src/postings/serializer.rs index 7b45f6a6e..1f84999b9 100644 --- a/src/postings/serializer.rs +++ b/src/postings/serializer.rs @@ -106,7 +106,7 @@ impl PostingsSerializer { segment.schema()) } - /// Must be called before starting pushing terms of + /// Must be called before starting pushing terms of /// a given field. /// /// Loads the indexing options for the given field. diff --git a/src/query/boolean_query/mod.rs b/src/query/boolean_query/mod.rs index a9888bad9..01ef9e824 100644 --- a/src/query/boolean_query/mod.rs +++ b/src/query/boolean_query/mod.rs @@ -122,7 +122,7 @@ mod tests { fieldnorm_reader_opt: Some(left_fieldnorms), postings: left, }; - + let right_fieldnorms = U64FastFieldReader::from((0u64..9u64).map(|doc| doc * 5).collect::>()); let right = VecPostings::from(vec![1, 3, 8]); diff --git a/src/schema/field_entry.rs b/src/schema/field_entry.rs index 9feaf64d4..883dc49ff 100644 --- a/src/schema/field_entry.rs +++ b/src/schema/field_entry.rs @@ -67,7 +67,7 @@ impl FieldEntry { FieldType::I64(ref options) => options.is_indexed(), } } - + /// Returns true iff the field is a int (signed or unsigned) fast field pub fn is_int_fast(&self) -> bool { match self.field_type { diff --git a/src/schema/field_type.rs b/src/schema/field_type.rs index 26e5e85e9..ae5591c0e 100644 --- a/src/schema/field_type.rs +++ b/src/schema/field_type.rs @@ -48,29 +48,23 @@ impl FieldType { match *self { FieldType::Str(ref text_options) => { match text_options.get_indexing_options() { - TextIndexingOptions::Untokenized => - Some(SegmentPostingsOption::NoFreq), - TextIndexingOptions::TokenizedNoFreq => - Some(SegmentPostingsOption::NoFreq), - TextIndexingOptions::TokenizedWithFreq => - Some(SegmentPostingsOption::Freq), - TextIndexingOptions::TokenizedWithFreqAndPosition => - Some(SegmentPostingsOption::FreqAndPositions), - TextIndexingOptions::Unindexed => { - None + TextIndexingOptions::Untokenized => Some(SegmentPostingsOption::NoFreq), + TextIndexingOptions::TokenizedNoFreq => Some(SegmentPostingsOption::NoFreq), + TextIndexingOptions::TokenizedWithFreq => Some(SegmentPostingsOption::Freq), + TextIndexingOptions::TokenizedWithFreqAndPosition => { + Some(SegmentPostingsOption::FreqAndPositions) } + TextIndexingOptions::Unindexed => None, } } FieldType::U64(ref int_options) | FieldType::I64(ref int_options) => { if int_options.is_indexed() { Some(SegmentPostingsOption::NoFreq) - } - else { + } else { None } - - }, + } } } diff --git a/src/schema/term.rs b/src/schema/term.rs index 988520cdc..0051adb48 100644 --- a/src/schema/term.rs +++ b/src/schema/term.rs @@ -22,7 +22,7 @@ pub(crate) fn extract_field_from_term_bytes(term_bytes: &[u8]) -> Field { impl Term { /// Returns the field. - pub fn field(&self,) -> Field { + pub fn field(&self) -> Field { extract_field_from_term_bytes(&self.0) } From 2a08c247afb9214356409b19b6242b018d7907b2 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Thu, 18 May 2017 23:14:58 +0900 Subject: [PATCH 05/10] Clippy --- src/datastruct/fstmap/fstmap.rs | 2 +- src/datastruct/fstmap/fstmerger.rs | 5 +++-- src/lib.rs | 2 ++ src/postings/postings_writer.rs | 1 + src/schema/field_type.rs | 4 ++-- src/schema/term.rs | 1 + 6 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/datastruct/fstmap/fstmap.rs b/src/datastruct/fstmap/fstmap.rs index f6894e1bd..f6fedf577 100644 --- a/src/datastruct/fstmap/fstmap.rs +++ b/src/datastruct/fstmap/fstmap.rs @@ -121,7 +121,7 @@ impl FstMap } pub fn range(&self) -> FstMapStreamerBuilder { - FstMapStreamerBuilder::new(&self, self.fst_index.range()) + FstMapStreamerBuilder::new(self, self.fst_index.range()) } } diff --git a/src/datastruct/fstmap/fstmerger.rs b/src/datastruct/fstmap/fstmerger.rs index b0e1c30b6..c535bc28e 100644 --- a/src/datastruct/fstmap/fstmerger.rs +++ b/src/datastruct/fstmap/fstmerger.rs @@ -86,6 +86,7 @@ impl<'a, V> FstMerger<'a, V> /// Advance the term iterator to the next term. /// Returns true if there is indeed another term /// False if there is none. + #[allow(while_let_loop)] pub fn advance(&mut self) -> bool { self.advance_segments(); if let Some(head) = self.heap.pop() { @@ -113,7 +114,7 @@ impl<'a, V> FstMerger<'a, V> /// iff advance() has been called before /// and "true" was returned. pub fn key(&self) -> &[u8] { - &self.current_streamers[0].streamer.key() + self.current_streamers[0].streamer.key() } /// Returns the sorted list of segment ordinals @@ -147,7 +148,7 @@ impl<'a, V> Streamer<'a> for FstMerger<'a, V> fn next(&'a mut self) -> Option { if self.advance() { - Some(&self.current_streamers[0].streamer.key()) + Some(self.current_streamers[0].streamer.key()) } else { None } diff --git a/src/lib.rs b/src/lib.rs index ef8a7dd05..b0a7ddbb6 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -12,6 +12,8 @@ #![doc(test(attr(allow(unused_variables), deny(warnings))))] +#![allow(unknown_lints)] + #![warn(missing_docs)] //! # `tantivy` diff --git a/src/postings/postings_writer.rs b/src/postings/postings_writer.rs index 018e22daf..33d297e62 100644 --- a/src/postings/postings_writer.rs +++ b/src/postings/postings_writer.rs @@ -78,6 +78,7 @@ impl<'a> MultiFieldPostingsWriter<'a> { /// Serialize the inverted index. /// It pushes all term, one field at a time, towards the /// postings serializer. + #[allow(needless_range_loop)] pub fn serialize(&self, serializer: &mut PostingsSerializer) -> Result<()> { let mut term_offsets: Vec<(&[u8], u32)> = self.term_index.iter().collect(); term_offsets.sort_by_key(|&(k, _v)| k); diff --git a/src/schema/field_type.rs b/src/schema/field_type.rs index ae5591c0e..06b62cdc8 100644 --- a/src/schema/field_type.rs +++ b/src/schema/field_type.rs @@ -48,8 +48,8 @@ impl FieldType { match *self { FieldType::Str(ref text_options) => { match text_options.get_indexing_options() { - TextIndexingOptions::Untokenized => Some(SegmentPostingsOption::NoFreq), - TextIndexingOptions::TokenizedNoFreq => Some(SegmentPostingsOption::NoFreq), + TextIndexingOptions::Untokenized | + TextIndexingOptions::TokenizedNoFreq => Some(SegmentPostingsOption::NoFreq), TextIndexingOptions::TokenizedWithFreq => Some(SegmentPostingsOption::Freq), TextIndexingOptions::TokenizedWithFreqAndPosition => { Some(SegmentPostingsOption::FreqAndPositions) diff --git a/src/schema/term.rs b/src/schema/term.rs index 0051adb48..06d8ffbd9 100644 --- a/src/schema/term.rs +++ b/src/schema/term.rs @@ -106,6 +106,7 @@ impl Term { /// /// If you want to build a field for a given `str`, /// you want to use `from_field_text`. + #[cfg(test)] pub(crate) fn from_bytes(data: &[u8]) -> Term { Term(Vec::from(data)) } From b3f62b8accf6b1fbcfeecff2ff302619b74a38c9 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Thu, 18 May 2017 23:33:15 +0900 Subject: [PATCH 06/10] Better API --- src/core/segment_reader.rs | 12 +++++------ src/datastruct/fstmap/fstmap.rs | 32 +++++++++++++++++++++++++++--- src/datastruct/fstmap/fstmerger.rs | 2 +- src/datastruct/fstmap/streamer.rs | 4 +++- src/schema/field_type.rs | 2 +- 5 files changed, 40 insertions(+), 12 deletions(-) diff --git a/src/core/segment_reader.rs b/src/core/segment_reader.rs index 1be0451f6..25f2a023f 100644 --- a/src/core/segment_reader.rs +++ b/src/core/segment_reader.rs @@ -43,7 +43,7 @@ use schema::TextIndexingOptions; pub struct SegmentReader { segment_id: SegmentId, segment_meta: SegmentMeta, - term_infos: Arc>, + terms: Arc>, postings_data: ReadOnlySource, store_reader: StoreReader, fast_fields_reader: Arc, @@ -135,7 +135,7 @@ impl SegmentReader { pub fn open(segment: Segment) -> Result { let source = try!(segment.open_read(SegmentComponent::TERMS)); - let term_infos = try!(FstMap::from_source(source)); + let terms = try!(FstMap::from_source(source)); let store_reader = StoreReader::from(try!(segment.open_read(SegmentComponent::STORE))); let postings_shared_mmap = try!(segment.open_read(SegmentComponent::POSTINGS)); @@ -160,7 +160,7 @@ impl SegmentReader { Ok(SegmentReader { segment_meta: segment.meta().clone(), postings_data: postings_shared_mmap, - term_infos: Arc::new(term_infos), + terms: Arc::new(terms), segment_id: segment.id(), store_reader: store_reader, fast_fields_reader: Arc::new(fast_fields_reader), @@ -172,8 +172,8 @@ impl SegmentReader { } /// Return the term dictionary datastructure. - pub fn term_infos(&self) -> &FstMap { - &self.term_infos + pub fn terms(&self) -> &FstMap { + &self.terms } /// Returns the document (or to be accurate, its stored field) @@ -259,7 +259,7 @@ impl SegmentReader { /// Returns the term info associated with the term. pub fn get_term_info(&self, term: &Term) -> Option { - self.term_infos.get(term.as_slice()) + self.terms.get(term.as_slice()) } /// Returns the segment id diff --git a/src/datastruct/fstmap/fstmap.rs b/src/datastruct/fstmap/fstmap.rs index f6fedf577..8338cee37 100644 --- a/src/datastruct/fstmap/fstmap.rs +++ b/src/datastruct/fstmap/fstmap.rs @@ -5,6 +5,7 @@ use super::{FstMapStreamerBuilder, FstMapStreamer}; use directory::ReadOnlySource; use common::BinarySerializable; use std::marker::PhantomData; +use schema::{Field, Term}; fn convert_fst_error(e: fst::Error) -> io::Error { @@ -104,22 +105,47 @@ impl FstMap }) } - pub(crate) fn read_value(&self, offset: u64) -> V { + + /// In the `FstMap`, the dictionary itself associated + /// each key `&[u8]` to a `u64` that is in fact the address + /// of the value object in a data array. + /// + /// This method deserialize this object, and returns it. + pub(crate) fn read_value(&self, offset: u64) -> io::Result { let buffer = self.values_mmap.as_slice(); let mut cursor = &buffer[(offset as usize)..]; - V::deserialize(&mut cursor).expect("Data in FST is corrupted") + V::deserialize(&mut cursor) } + /// Returns, if present the value associated to a given key. pub fn get>(&self, key: K) -> Option { self.fst_index .get(key) - .map(|offset| self.read_value(offset)) + .map(|offset| { + self.read_value(offset) + .expect("The fst is corrupted. Failed to deserialize a value.") + }) } + + /// Returns a stream of all the sorted terms. pub fn stream(&self) -> FstMapStreamer { self.range().into_stream() } + + /// Returns a stream of all the sorted terms in the given field. + pub fn stream_field(&self, field: Field) -> FstMapStreamer { + let start_term = Term::from_field_text(field, ""); + let stop_term = Term::from_field_text(Field(field.0 + 1), ""); + self.range() + .ge(start_term.as_slice()) + .lt(stop_term.as_slice()) + .into_stream() + } + + /// Returns a range builder, to stream all of the terms + /// within an interval. pub fn range(&self) -> FstMapStreamerBuilder { FstMapStreamerBuilder::new(self, self.fst_index.range()) } diff --git a/src/datastruct/fstmap/fstmerger.rs b/src/datastruct/fstmap/fstmerger.rs index c535bc28e..a4342855a 100644 --- a/src/datastruct/fstmap/fstmerger.rs +++ b/src/datastruct/fstmap/fstmerger.rs @@ -136,7 +136,7 @@ impl<'a> From<&'a [SegmentReader]> for FstMerger<'a, TermInfo> fn from(segment_readers: &'a [SegmentReader]) -> FstMerger<'a, TermInfo> { FstMerger::new(segment_readers .iter() - .map(|reader| reader.term_infos().stream()) + .map(|reader| reader.terms().stream()) .collect()) } } diff --git a/src/datastruct/fstmap/streamer.rs b/src/datastruct/fstmap/streamer.rs index ba8edcccb..3bef03389 100644 --- a/src/datastruct/fstmap/streamer.rs +++ b/src/datastruct/fstmap/streamer.rs @@ -99,6 +99,8 @@ impl<'a, V> FstMapStreamer<'a, V> } pub fn value(&self) -> V { - self.fst_map.read_value(self.offset) + self.fst_map + .read_value(self.offset) + .expect("Fst data is corrupted. Failed to deserialize a value.") } } diff --git a/src/schema/field_type.rs b/src/schema/field_type.rs index 06b62cdc8..fcaaf8013 100644 --- a/src/schema/field_type.rs +++ b/src/schema/field_type.rs @@ -49,7 +49,7 @@ impl FieldType { FieldType::Str(ref text_options) => { match text_options.get_indexing_options() { TextIndexingOptions::Untokenized | - TextIndexingOptions::TokenizedNoFreq => Some(SegmentPostingsOption::NoFreq), + TextIndexingOptions::TokenizedNoFreq => Some(SegmentPostingsOption::NoFreq), TextIndexingOptions::TokenizedWithFreq => Some(SegmentPostingsOption::Freq), TextIndexingOptions::TokenizedWithFreqAndPosition => { Some(SegmentPostingsOption::FreqAndPositions) From 02bfa9be527efc05991fa20bfba9d5496236e959 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Fri, 19 May 2017 08:43:52 +0900 Subject: [PATCH 07/10] Moving to termdict --- src/core/searcher.rs | 2 +- src/core/segment_reader.rs | 2 +- src/datastruct/fstmap/mod.rs | 9 - src/datastruct/mod.rs | 1 - src/directory/mod.rs | 5 + src/fastfield/mod.rs | 46 ++--- src/indexer/merger.rs | 2 +- src/lib.rs | 6 +- src/postings/serializer.rs | 2 +- src/{datastruct/fstmap => termdict}/fstmap.rs | 26 +-- src/termdict/fstmap.rs.bk | 183 ++++++++++++++++++ .../fstmap => termdict}/fstmerger.rs | 0 src/termdict/mod.rs | 26 +++ .../fstmap => termdict}/streamer.rs | 0 14 files changed, 258 insertions(+), 52 deletions(-) delete mode 100644 src/datastruct/fstmap/mod.rs rename src/{datastruct/fstmap => termdict}/fstmap.rs (89%) create mode 100644 src/termdict/fstmap.rs.bk rename src/{datastruct/fstmap => termdict}/fstmerger.rs (100%) create mode 100644 src/termdict/mod.rs rename src/{datastruct/fstmap => termdict}/streamer.rs (100%) diff --git a/src/core/searcher.rs b/src/core/searcher.rs index 45bdfccb7..daa547f00 100644 --- a/src/core/searcher.rs +++ b/src/core/searcher.rs @@ -7,7 +7,7 @@ use query::Query; use DocId; use DocAddress; use schema::Term; -use datastruct::fstmap::FstMerger; +use termdict::FstMerger; use std::fmt; use postings::TermInfo; diff --git a/src/core/segment_reader.rs b/src/core/segment_reader.rs index 25f2a023f..9a7471881 100644 --- a/src/core/segment_reader.rs +++ b/src/core/segment_reader.rs @@ -14,7 +14,7 @@ use DocId; use std::str; use std::cmp; use postings::TermInfo; -use datastruct::fstmap::FstMap; +use termdict::FstMap; use std::sync::Arc; use std::fmt; use schema::Field; diff --git a/src/datastruct/fstmap/mod.rs b/src/datastruct/fstmap/mod.rs deleted file mode 100644 index 62d1a786d..000000000 --- a/src/datastruct/fstmap/mod.rs +++ /dev/null @@ -1,9 +0,0 @@ -mod fstmap; -mod streamer; -mod fstmerger; - -pub use self::fstmap::FstMap; -pub use self::fstmap::FstMapBuilder; -pub use self::streamer::FstMapStreamer; -pub use self::streamer::FstMapStreamerBuilder; -pub use self::fstmerger::FstMerger; diff --git a/src/datastruct/mod.rs b/src/datastruct/mod.rs index 96cc8242f..5ff10f7c7 100644 --- a/src/datastruct/mod.rs +++ b/src/datastruct/mod.rs @@ -1,4 +1,3 @@ -pub mod fstmap; mod skip; pub mod stacker; diff --git a/src/directory/mod.rs b/src/directory/mod.rs index f47cfdcbb..b107d78c5 100644 --- a/src/directory/mod.rs +++ b/src/directory/mod.rs @@ -1,3 +1,8 @@ +/*! + +WORM directory abstraction. + +*/ mod mmap_directory; mod ram_directory; mod directory; diff --git a/src/fastfield/mod.rs b/src/fastfield/mod.rs index 061ed3910..84fe5307a 100644 --- a/src/fastfield/mod.rs +++ b/src/fastfield/mod.rs @@ -1,25 +1,27 @@ -//! # Fast fields -//! -//! Fast fields are the equivalent of `DocValues` in `Lucene`. -//! Fast fields is a non-compressed column-oriented fashion storage -//! of `tantivy`. -//! -//! It is designed for the fast random access of some document -//! fields given a document id. -//! -//! `FastField` are useful when a field is required for all or most of -//! the `DocSet` : for instance for scoring, grouping, filtering, or facetting. -//! -//! -//! Fields have to be declared as `FAST` in the schema. -//! Currently only 64-bits integers (signed or unsigned) are -//! supported. -//! -//! They are stored in a bitpacked fashion so that their -//! memory usage is directly linear with the amplitude of the -//! values stored. -//! -//! Read access performance is comparable to that of an array lookup. +/*! +Fast fields is a column oriented storage storage. + +It is the equivalent of `Lucene`'s `DocValues`. + +Fast fields is a column-oriented fashion storage of `tantivy`. + +It is designed for the fast random access of some document +fields given a document id. + +`FastField` are useful when a field is required for all or most of +the `DocSet` : for instance for scoring, grouping, filtering, or facetting. + + +Fields have to be declared as `FAST` in the schema. +Currently only 64-bits integers (signed or unsigned) are +supported. + +They are stored in a bitpacked fashion so that their +memory usage is directly linear with the amplitude of the +values stored. + +Read access performance is comparable to that of an array lookup. +*/ mod reader; mod writer; diff --git a/src/indexer/merger.rs b/src/indexer/merger.rs index 8036c4977..f26177ea3 100644 --- a/src/indexer/merger.rs +++ b/src/indexer/merger.rs @@ -12,7 +12,7 @@ use postings::Postings; use postings::DocSet; use fastfield::DeleteBitSet; use schema::{Schema, Field}; -use datastruct::fstmap::FstMerger; +use termdict::FstMerger; use fastfield::FastFieldSerializer; use fastfield::FastFieldReader; use store::StoreWriter; diff --git a/src/lib.rs b/src/lib.rs index b0a7ddbb6..bfd098a96 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -111,9 +111,10 @@ mod datastruct; +pub mod termdict; + /// Query module pub mod query; -/// Directory module pub mod directory; /// Collector module pub mod collector; @@ -147,8 +148,7 @@ pub fn version() -> &'static str { } } -/// Tantivy's makes it possible to personalize when -/// the indexer should merge its segments +/// Defines tantivy's merging strategy pub mod merge_policy { pub use indexer::MergePolicy; pub use indexer::LogMergePolicy; diff --git a/src/postings/serializer.rs b/src/postings/serializer.rs index 1f84999b9..048f23dca 100644 --- a/src/postings/serializer.rs +++ b/src/postings/serializer.rs @@ -1,5 +1,5 @@ use Result; -use datastruct::fstmap::FstMapBuilder; +use termdict::FstMapBuilder; use super::TermInfo; use schema::Field; use schema::FieldEntry; diff --git a/src/datastruct/fstmap/fstmap.rs b/src/termdict/fstmap.rs similarity index 89% rename from src/datastruct/fstmap/fstmap.rs rename to src/termdict/fstmap.rs index 8338cee37..bcf8c3a50 100644 --- a/src/datastruct/fstmap/fstmap.rs +++ b/src/termdict/fstmap.rs @@ -12,6 +12,7 @@ fn convert_fst_error(e: fst::Error) -> io::Error { io::Error::new(io::ErrorKind::Other, e) } + pub struct FstMapBuilder { fst_builder: fst::MapBuilder, data: Vec, @@ -75,15 +76,17 @@ pub struct FstMap { fn open_fst_index(source: ReadOnlySource) -> io::Result { - Ok(fst::Map::from(match source { - ReadOnlySource::Anonymous(data) => { - Fst::from_shared_bytes(data.data, data.start, data.len) - .map_err(convert_fst_error)? - } - ReadOnlySource::Mmap(mmap_readonly) => { - Fst::from_mmap(mmap_readonly).map_err(convert_fst_error)? - } - })) + let fst = match source { + ReadOnlySource::Anonymous(data) => { + Fst::from_shared_bytes(data.data, data.start, data.len) + .map_err(convert_fst_error)? + } + ReadOnlySource::Mmap(mmap_readonly) => { + Fst::from_mmap(mmap_readonly) + .map_err(convert_fst_error)? + } + }; + Ok(fst::Map::from(fst)) } impl FstMap @@ -106,9 +109,6 @@ impl FstMap } - /// In the `FstMap`, the dictionary itself associated - /// each key `&[u8]` to a `u64` that is in fact the address - /// of the value object in a data array. /// /// This method deserialize this object, and returns it. pub(crate) fn read_value(&self, offset: u64) -> io::Result { @@ -125,7 +125,7 @@ impl FstMap self.read_value(offset) .expect("The fst is corrupted. Failed to deserialize a value.") }) - } + } /// Returns a stream of all the sorted terms. diff --git a/src/termdict/fstmap.rs.bk b/src/termdict/fstmap.rs.bk new file mode 100644 index 000000000..732d14084 --- /dev/null +++ b/src/termdict/fstmap.rs.bk @@ -0,0 +1,183 @@ +use std::io::{self, Write}; +use fst; +use fst::raw::Fst; +use super::{FstMapStreamerBuilder, FstMapStreamer}; +use directory::ReadOnlySource; +use common::BinarySerializable; +use std::marker::PhantomData; +use schema::{Field, Term}; + + +fn convert_fst_error(e: fst::Error) -> io::Error { + io::Error::new(io::ErrorKind::Other, e) +} + +pub struct FstMapBuilder { + fst_builder: fst::MapBuilder, + data: Vec, + _phantom_: PhantomData, +} + +impl FstMapBuilder { + pub fn new(w: W) -> io::Result> { + let fst_builder = fst::MapBuilder::new(w).map_err(convert_fst_error)?; + Ok(FstMapBuilder { + fst_builder: fst_builder, + data: Vec::new(), + _phantom_: PhantomData, + }) + } + + /// Horribly unsafe, nobody should ever do that... except me :) + /// + /// If used, it must be used by systematically alternating calls + /// to insert_key and insert_value. + /// + /// TODO see if I can bend Rust typesystem to enforce that + /// in a nice way. + pub fn insert_key(&mut self, key: &[u8]) -> io::Result<()> { + self.fst_builder + .insert(key, self.data.len() as u64) + .map_err(convert_fst_error)?; + Ok(()) + } + + /// Horribly unsafe, nobody should ever do that... except me :) + pub fn insert_value(&mut self, value: &V) -> io::Result<()> { + value.serialize(&mut self.data)?; + Ok(()) + } + + #[cfg(test)] + pub fn insert(&mut self, key: &[u8], value: &V) -> io::Result<()> { + self.fst_builder + .insert(key, self.data.len() as u64) + .map_err(convert_fst_error)?; + value.serialize(&mut self.data)?; + Ok(()) + } + + pub fn finish(self) -> io::Result { + let mut file = self.fst_builder.into_inner().map_err(convert_fst_error)?; + let footer_size = self.data.len() as u32; + file.write_all(&self.data)?; + (footer_size as u32).serialize(&mut file)?; + file.flush()?; + Ok(file) + } +} + +pub struct FstMap { + fst_index: fst::Map, + values_mmap: ReadOnlySource, + _phantom_: PhantomData, +} + + +fn open_fst_index(source: ReadOnlySource) -> io::Result { + Ok(fst::Map::from(match source { + ReadOnlySource::Anonymous(data) => { + Fst::from_shared_bytes(data.data, data.start, data.len) + .map_err(convert_fst_error)? + } + ReadOnlySource::Mmap(mmap_readonly) => { + Fst::from_mmap(mmap_readonly).map_err(convert_fst_error)? + } + })) +} + +impl FstMap + where V: BinarySerializable +{ + pub fn from_source(source: ReadOnlySource) -> io::Result> { + let total_len = source.len(); + let length_offset = total_len - 4; + let mut split_len_buffer: &[u8] = &source.as_slice()[length_offset..]; + let footer_size = u32::deserialize(&mut split_len_buffer)? as usize; + let split_len = length_offset - footer_size; + let fst_source = source.slice(0, split_len); + let values_source = source.slice(split_len, length_offset); + let fst_index = open_fst_index(fst_source)?; + Ok(FstMap { + fst_index: fst_index, + values_mmap: values_source, + _phantom_: PhantomData, + }) + } + + + /// In the `FstMap`, the dictionary itself associated + /// each key `&[u8]` to a `u64` that is in fact the address + /// of the value object in a data array. + /// + /// This method deserialize this object, and returns it. + pub(crate) fn read_value(&self, offset: u64) -> io::Result { + let buffer = self.values_mmap.as_slice(); + let mut cursor = &buffer[(offset as usize)..]; + V::deserialize(&mut cursor) + } + + /// Returns, if present the value associated to a given key. + pub fn get>(&self, key: K) -> Option { + self.fst_index + .get(key) + .map(|offset| self.read_value(offset).expect("The fst is corrupted. Failed to deserialize a value.")) + } + + + /// Returns a stream of all the sorted terms. + pub fn stream(&self) -> FstMapStreamer { + self.range().into_stream() + } + + + /// Returns a stream of all the sorted terms in the given field. + pub fn stream_field(&self, field: Field) -> FstMapStreamer { + let start_term = Term::from_field_text(field, ""); + let stop_term = Term::from_field_text(Field(field.0 + 1), ""); + self.range() + .ge(start_term.as_slice()) + .lt(stop_term.as_slice()) + .into_stream() + } + + /// Returns a range builder, to stream all of the terms + /// within an interval. + pub fn range(&self) -> FstMapStreamerBuilder { + FstMapStreamerBuilder::new(self, self.fst_index.range()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use directory::{RAMDirectory, Directory}; + use std::path::PathBuf; + use fst::Streamer; + + #[test] + fn test_fstmap() { + let mut directory = RAMDirectory::create(); + let path = PathBuf::from("fstmap"); + { + let write = directory.open_write(&path).unwrap(); + let mut fstmap_builder = FstMapBuilder::new(write).unwrap(); + fstmap_builder.insert("abc".as_bytes(), &34u32).unwrap(); + fstmap_builder.insert("abcd".as_bytes(), &346u32).unwrap(); + fstmap_builder.finish().unwrap(); + } + let source = directory.open_read(&path).unwrap(); + let fstmap: FstMap = FstMap::from_source(source).unwrap(); + assert_eq!(fstmap.get("abc"), Some(34u32)); + assert_eq!(fstmap.get("abcd"), Some(346u32)); + let mut stream = fstmap.stream(); + assert_eq!(stream.next().unwrap(), "abc".as_bytes()); + assert_eq!(stream.key(), "abc".as_bytes()); + assert_eq!(stream.value(), 34u32); + assert_eq!(stream.next().unwrap(), "abcd".as_bytes()); + assert_eq!(stream.key(), "abcd".as_bytes()); + assert_eq!(stream.value(), 346u32); + assert!(!stream.advance()); + } + +} diff --git a/src/datastruct/fstmap/fstmerger.rs b/src/termdict/fstmerger.rs similarity index 100% rename from src/datastruct/fstmap/fstmerger.rs rename to src/termdict/fstmerger.rs diff --git a/src/termdict/mod.rs b/src/termdict/mod.rs new file mode 100644 index 000000000..3ef5c4c63 --- /dev/null +++ b/src/termdict/mod.rs @@ -0,0 +1,26 @@ +/*! +The term dictionary contains all of the terms in +`tantivy index` in a sorted manner. + +It is implemented as a wrapper of the `Fst` crate in order +to add a value type. + +A finite state transducer itself associates +each term `&[u8]` to a `u64` that is in fact an address +in a buffer. The value is then accessible via +deserializing the value at this address. + +Keys (`&[u8]`) in this datastructure are +sorted. + +*/ + +mod fstmap; +mod streamer; +mod fstmerger; + +pub use self::fstmap::FstMap; +pub(crate) use self::fstmap::FstMapBuilder; +pub use self::streamer::FstMapStreamer; +pub use self::streamer::FstMapStreamerBuilder; +pub use self::fstmerger::FstMerger; \ No newline at end of file diff --git a/src/datastruct/fstmap/streamer.rs b/src/termdict/streamer.rs similarity index 100% rename from src/datastruct/fstmap/streamer.rs rename to src/termdict/streamer.rs From c57ab6a3354ea8e6df51b2b41152124eaaa0e322 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Fri, 19 May 2017 09:26:18 +0900 Subject: [PATCH 08/10] Renamed fstmap to termdict --- src/core/segment_reader.rs | 8 +- src/indexer/index_writer.rs | 2 +- src/indexer/segment_writer.rs | 4 +- src/postings/postings_writer.rs | 2 +- src/postings/serializer.rs | 6 +- src/termdict/fstmap.rs.bk | 183 ----------------------- src/termdict/{fstmerger.rs => merger.rs} | 6 +- src/termdict/mod.rs | 14 +- src/termdict/streamer.rs | 26 ++-- src/termdict/{fstmap.rs => termdict.rs} | 61 ++++---- 10 files changed, 62 insertions(+), 250 deletions(-) delete mode 100644 src/termdict/fstmap.rs.bk rename src/termdict/{fstmerger.rs => merger.rs} (97%) rename src/termdict/{fstmap.rs => termdict.rs} (73%) diff --git a/src/core/segment_reader.rs b/src/core/segment_reader.rs index 9a7471881..d8df2fa8f 100644 --- a/src/core/segment_reader.rs +++ b/src/core/segment_reader.rs @@ -14,7 +14,7 @@ use DocId; use std::str; use std::cmp; use postings::TermInfo; -use termdict::FstMap; +use termdict::TermDictionary; use std::sync::Arc; use std::fmt; use schema::Field; @@ -43,7 +43,7 @@ use schema::TextIndexingOptions; pub struct SegmentReader { segment_id: SegmentId, segment_meta: SegmentMeta, - terms: Arc>, + terms: Arc, postings_data: ReadOnlySource, store_reader: StoreReader, fast_fields_reader: Arc, @@ -135,7 +135,7 @@ impl SegmentReader { pub fn open(segment: Segment) -> Result { let source = try!(segment.open_read(SegmentComponent::TERMS)); - let terms = try!(FstMap::from_source(source)); + let terms = try!(TermDictionary::from_source(source)); let store_reader = StoreReader::from(try!(segment.open_read(SegmentComponent::STORE))); let postings_shared_mmap = try!(segment.open_read(SegmentComponent::POSTINGS)); @@ -172,7 +172,7 @@ impl SegmentReader { } /// Return the term dictionary datastructure. - pub fn terms(&self) -> &FstMap { + pub fn terms(&self) -> &TermDictionary { &self.terms } diff --git a/src/indexer/index_writer.rs b/src/indexer/index_writer.rs index 6a038812c..0a774a160 100644 --- a/src/indexer/index_writer.rs +++ b/src/indexer/index_writer.rs @@ -273,7 +273,7 @@ fn index_documents(heap: &mut Heap, // // Tantivy does not resize its hashtable. When it reaches // capacity, we just stop indexing new document. - if segment_writer.is_termdic_saturated() { + if segment_writer.is_term_saturated() { info!("Term dic saturated, flushing segment with maxdoc={}.", segment_writer.max_doc()); break; diff --git a/src/indexer/segment_writer.rs b/src/indexer/segment_writer.rs index ffaae2cca..c5d4d6662 100644 --- a/src/indexer/segment_writer.rs +++ b/src/indexer/segment_writer.rs @@ -98,8 +98,8 @@ impl<'a> SegmentWriter<'a> { /// Return true if the term dictionary hashmap is reaching capacity. /// It is one of the condition that triggers a `SegmentWriter` to /// be finalized. - pub(crate) fn is_termdic_saturated(&self) -> bool { - self.multifield_postings.is_termdic_saturated() + pub(crate) fn is_term_saturated(&self) -> bool { + self.multifield_postings.is_term_saturated() } diff --git a/src/postings/postings_writer.rs b/src/postings/postings_writer.rs index 33d297e62..65fba2f20 100644 --- a/src/postings/postings_writer.rs +++ b/src/postings/postings_writer.rs @@ -109,7 +109,7 @@ impl<'a> MultiFieldPostingsWriter<'a> { } /// Return true iff the term dictionary is saturated. - pub fn is_termdic_saturated(&self) -> bool { + pub fn is_term_saturated(&self) -> bool { self.term_index.is_saturated() } } diff --git a/src/postings/serializer.rs b/src/postings/serializer.rs index 048f23dca..1313ad445 100644 --- a/src/postings/serializer.rs +++ b/src/postings/serializer.rs @@ -1,5 +1,5 @@ use Result; -use termdict::FstMapBuilder; +use termdict::TermDictionaryBuilder; use super::TermInfo; use schema::Field; use schema::FieldEntry; @@ -50,7 +50,7 @@ use common::BinarySerializable; /// A description of the serialization format is /// [available here](https://fulmicoton.gitbooks.io/tantivy-doc/content/inverted-index.html). pub struct PostingsSerializer { - terms_fst_builder: FstMapBuilder, + terms_fst_builder: TermDictionaryBuilder, postings_write: WritePtr, positions_write: WritePtr, written_bytes_postings: usize, @@ -74,7 +74,7 @@ impl PostingsSerializer { positions_write: WritePtr, schema: Schema) -> Result { - let terms_fst_builder = try!(FstMapBuilder::new(terms_write)); + let terms_fst_builder = try!(TermDictionaryBuilder::new(terms_write)); Ok(PostingsSerializer { terms_fst_builder: terms_fst_builder, postings_write: postings_write, diff --git a/src/termdict/fstmap.rs.bk b/src/termdict/fstmap.rs.bk deleted file mode 100644 index 732d14084..000000000 --- a/src/termdict/fstmap.rs.bk +++ /dev/null @@ -1,183 +0,0 @@ -use std::io::{self, Write}; -use fst; -use fst::raw::Fst; -use super::{FstMapStreamerBuilder, FstMapStreamer}; -use directory::ReadOnlySource; -use common::BinarySerializable; -use std::marker::PhantomData; -use schema::{Field, Term}; - - -fn convert_fst_error(e: fst::Error) -> io::Error { - io::Error::new(io::ErrorKind::Other, e) -} - -pub struct FstMapBuilder { - fst_builder: fst::MapBuilder, - data: Vec, - _phantom_: PhantomData, -} - -impl FstMapBuilder { - pub fn new(w: W) -> io::Result> { - let fst_builder = fst::MapBuilder::new(w).map_err(convert_fst_error)?; - Ok(FstMapBuilder { - fst_builder: fst_builder, - data: Vec::new(), - _phantom_: PhantomData, - }) - } - - /// Horribly unsafe, nobody should ever do that... except me :) - /// - /// If used, it must be used by systematically alternating calls - /// to insert_key and insert_value. - /// - /// TODO see if I can bend Rust typesystem to enforce that - /// in a nice way. - pub fn insert_key(&mut self, key: &[u8]) -> io::Result<()> { - self.fst_builder - .insert(key, self.data.len() as u64) - .map_err(convert_fst_error)?; - Ok(()) - } - - /// Horribly unsafe, nobody should ever do that... except me :) - pub fn insert_value(&mut self, value: &V) -> io::Result<()> { - value.serialize(&mut self.data)?; - Ok(()) - } - - #[cfg(test)] - pub fn insert(&mut self, key: &[u8], value: &V) -> io::Result<()> { - self.fst_builder - .insert(key, self.data.len() as u64) - .map_err(convert_fst_error)?; - value.serialize(&mut self.data)?; - Ok(()) - } - - pub fn finish(self) -> io::Result { - let mut file = self.fst_builder.into_inner().map_err(convert_fst_error)?; - let footer_size = self.data.len() as u32; - file.write_all(&self.data)?; - (footer_size as u32).serialize(&mut file)?; - file.flush()?; - Ok(file) - } -} - -pub struct FstMap { - fst_index: fst::Map, - values_mmap: ReadOnlySource, - _phantom_: PhantomData, -} - - -fn open_fst_index(source: ReadOnlySource) -> io::Result { - Ok(fst::Map::from(match source { - ReadOnlySource::Anonymous(data) => { - Fst::from_shared_bytes(data.data, data.start, data.len) - .map_err(convert_fst_error)? - } - ReadOnlySource::Mmap(mmap_readonly) => { - Fst::from_mmap(mmap_readonly).map_err(convert_fst_error)? - } - })) -} - -impl FstMap - where V: BinarySerializable -{ - pub fn from_source(source: ReadOnlySource) -> io::Result> { - let total_len = source.len(); - let length_offset = total_len - 4; - let mut split_len_buffer: &[u8] = &source.as_slice()[length_offset..]; - let footer_size = u32::deserialize(&mut split_len_buffer)? as usize; - let split_len = length_offset - footer_size; - let fst_source = source.slice(0, split_len); - let values_source = source.slice(split_len, length_offset); - let fst_index = open_fst_index(fst_source)?; - Ok(FstMap { - fst_index: fst_index, - values_mmap: values_source, - _phantom_: PhantomData, - }) - } - - - /// In the `FstMap`, the dictionary itself associated - /// each key `&[u8]` to a `u64` that is in fact the address - /// of the value object in a data array. - /// - /// This method deserialize this object, and returns it. - pub(crate) fn read_value(&self, offset: u64) -> io::Result { - let buffer = self.values_mmap.as_slice(); - let mut cursor = &buffer[(offset as usize)..]; - V::deserialize(&mut cursor) - } - - /// Returns, if present the value associated to a given key. - pub fn get>(&self, key: K) -> Option { - self.fst_index - .get(key) - .map(|offset| self.read_value(offset).expect("The fst is corrupted. Failed to deserialize a value.")) - } - - - /// Returns a stream of all the sorted terms. - pub fn stream(&self) -> FstMapStreamer { - self.range().into_stream() - } - - - /// Returns a stream of all the sorted terms in the given field. - pub fn stream_field(&self, field: Field) -> FstMapStreamer { - let start_term = Term::from_field_text(field, ""); - let stop_term = Term::from_field_text(Field(field.0 + 1), ""); - self.range() - .ge(start_term.as_slice()) - .lt(stop_term.as_slice()) - .into_stream() - } - - /// Returns a range builder, to stream all of the terms - /// within an interval. - pub fn range(&self) -> FstMapStreamerBuilder { - FstMapStreamerBuilder::new(self, self.fst_index.range()) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use directory::{RAMDirectory, Directory}; - use std::path::PathBuf; - use fst::Streamer; - - #[test] - fn test_fstmap() { - let mut directory = RAMDirectory::create(); - let path = PathBuf::from("fstmap"); - { - let write = directory.open_write(&path).unwrap(); - let mut fstmap_builder = FstMapBuilder::new(write).unwrap(); - fstmap_builder.insert("abc".as_bytes(), &34u32).unwrap(); - fstmap_builder.insert("abcd".as_bytes(), &346u32).unwrap(); - fstmap_builder.finish().unwrap(); - } - let source = directory.open_read(&path).unwrap(); - let fstmap: FstMap = FstMap::from_source(source).unwrap(); - assert_eq!(fstmap.get("abc"), Some(34u32)); - assert_eq!(fstmap.get("abcd"), Some(346u32)); - let mut stream = fstmap.stream(); - assert_eq!(stream.next().unwrap(), "abc".as_bytes()); - assert_eq!(stream.key(), "abc".as_bytes()); - assert_eq!(stream.value(), 34u32); - assert_eq!(stream.next().unwrap(), "abcd".as_bytes()); - assert_eq!(stream.key(), "abcd".as_bytes()); - assert_eq!(stream.value(), 346u32); - assert!(!stream.advance()); - } - -} diff --git a/src/termdict/fstmerger.rs b/src/termdict/merger.rs similarity index 97% rename from src/termdict/fstmerger.rs rename to src/termdict/merger.rs index a4342855a..e1d3fd155 100644 --- a/src/termdict/fstmerger.rs +++ b/src/termdict/merger.rs @@ -1,6 +1,6 @@ use std::collections::BinaryHeap; use core::SegmentReader; -use super::FstMapStreamer; +use super::TermStreamer; use common::BinarySerializable; use postings::TermInfo; use std::cmp::Ordering; @@ -9,7 +9,7 @@ use fst::Streamer; pub struct HeapItem<'a, V> where V: 'a + BinarySerializable { - pub streamer: FstMapStreamer<'a, V>, + pub streamer: TermStreamer<'a, V>, pub segment_ord: usize, } @@ -56,7 +56,7 @@ pub struct FstMerger<'a, V> impl<'a, V> FstMerger<'a, V> where V: 'a + BinarySerializable { - fn new(streams: Vec>) -> FstMerger<'a, V> { + fn new(streams: Vec>) -> FstMerger<'a, V> { FstMerger { heap: BinaryHeap::new(), current_streamers: streams diff --git a/src/termdict/mod.rs b/src/termdict/mod.rs index 3ef5c4c63..708538744 100644 --- a/src/termdict/mod.rs +++ b/src/termdict/mod.rs @@ -15,12 +15,12 @@ sorted. */ -mod fstmap; +mod termdict; mod streamer; -mod fstmerger; +mod merger; -pub use self::fstmap::FstMap; -pub(crate) use self::fstmap::FstMapBuilder; -pub use self::streamer::FstMapStreamer; -pub use self::streamer::FstMapStreamerBuilder; -pub use self::fstmerger::FstMerger; \ No newline at end of file +pub use self::termdict::TermDictionary; +pub(crate) use self::termdict::TermDictionaryBuilder; +pub use self::streamer::TermStreamer; +pub use self::streamer::TermStreamerBuilder; +pub use self::merger::FstMerger; \ No newline at end of file diff --git a/src/termdict/streamer.rs b/src/termdict/streamer.rs index 3bef03389..c5784e6bd 100644 --- a/src/termdict/streamer.rs +++ b/src/termdict/streamer.rs @@ -1,16 +1,16 @@ use fst::{self, IntoStreamer, Streamer}; use fst::map::{StreamBuilder, Stream}; use common::BinarySerializable; -use super::FstMap; +use super::TermDictionary; -pub struct FstMapStreamerBuilder<'a, V> +pub struct TermStreamerBuilder<'a, V> where V: 'a + BinarySerializable { - fst_map: &'a FstMap, + fst_map: &'a TermDictionary, stream_builder: StreamBuilder<'a>, } -impl<'a, V> FstMapStreamerBuilder<'a, V> +impl<'a, V> TermStreamerBuilder<'a, V> where V: 'a + BinarySerializable { pub fn ge>(mut self, bound: T) -> Self { @@ -33,8 +33,8 @@ impl<'a, V> FstMapStreamerBuilder<'a, V> self } - pub fn into_stream(self) -> FstMapStreamer<'a, V> { - FstMapStreamer { + pub fn into_stream(self) -> TermStreamer<'a, V> { + TermStreamer { fst_map: self.fst_map, stream: self.stream_builder.into_stream(), buffer: Vec::with_capacity(100), @@ -42,10 +42,10 @@ impl<'a, V> FstMapStreamerBuilder<'a, V> } } - pub fn new(fst_map: &'a FstMap, + pub fn new(fst_map: &'a TermDictionary, stream_builder: StreamBuilder<'a>) - -> FstMapStreamerBuilder<'a, V> { - FstMapStreamerBuilder { + -> TermStreamerBuilder<'a, V> { + TermStreamerBuilder { fst_map: fst_map, stream_builder: stream_builder, } @@ -56,17 +56,17 @@ impl<'a, V> FstMapStreamerBuilder<'a, V> -pub struct FstMapStreamer<'a, V> +pub struct TermStreamer<'a, V> where V: 'a + BinarySerializable { - fst_map: &'a FstMap, + fst_map: &'a TermDictionary, stream: Stream<'a>, offset: u64, buffer: Vec, } -impl<'a, 'b, V> fst::Streamer<'b> for FstMapStreamer<'a, V> +impl<'a, 'b, V> fst::Streamer<'b> for TermStreamer<'a, V> where V: 'a + BinarySerializable { type Item = &'b [u8]; @@ -80,7 +80,7 @@ impl<'a, 'b, V> fst::Streamer<'b> for FstMapStreamer<'a, V> } } -impl<'a, V> FstMapStreamer<'a, V> +impl<'a, V> TermStreamer<'a, V> where V: 'a + BinarySerializable { pub fn advance(&mut self) -> bool { diff --git a/src/termdict/fstmap.rs b/src/termdict/termdict.rs similarity index 73% rename from src/termdict/fstmap.rs rename to src/termdict/termdict.rs index bcf8c3a50..232826234 100644 --- a/src/termdict/fstmap.rs +++ b/src/termdict/termdict.rs @@ -1,11 +1,12 @@ use std::io::{self, Write}; use fst; use fst::raw::Fst; -use super::{FstMapStreamerBuilder, FstMapStreamer}; +use super::{TermStreamerBuilder, TermStreamer}; use directory::ReadOnlySource; use common::BinarySerializable; use std::marker::PhantomData; use schema::{Field, Term}; +use postings::TermInfo; fn convert_fst_error(e: fst::Error) -> io::Error { @@ -13,43 +14,39 @@ fn convert_fst_error(e: fst::Error) -> io::Error { } -pub struct FstMapBuilder { +pub struct TermDictionaryBuilder where V: BinarySerializable { fst_builder: fst::MapBuilder, data: Vec, _phantom_: PhantomData, } -impl FstMapBuilder { - pub fn new(w: W) -> io::Result> { +impl TermDictionaryBuilder { + pub fn new(w: W) -> io::Result> { let fst_builder = fst::MapBuilder::new(w).map_err(convert_fst_error)?; - Ok(FstMapBuilder { + Ok(TermDictionaryBuilder { fst_builder: fst_builder, data: Vec::new(), _phantom_: PhantomData, }) } - /// Horribly unsafe, nobody should ever do that... except me :) + /// Horribly unsafe internal API /// /// If used, it must be used by systematically alternating calls /// to insert_key and insert_value. - /// - /// TODO see if I can bend Rust typesystem to enforce that - /// in a nice way. - pub fn insert_key(&mut self, key: &[u8]) -> io::Result<()> { + pub(crate) fn insert_key(&mut self, key: &[u8]) -> io::Result<()> { self.fst_builder .insert(key, self.data.len() as u64) .map_err(convert_fst_error)?; Ok(()) } - /// Horribly unsafe, nobody should ever do that... except me :) - pub fn insert_value(&mut self, value: &V) -> io::Result<()> { + /// Horribly unsafe internal API + pub(crate) fn insert_value(&mut self, value: &V) -> io::Result<()> { value.serialize(&mut self.data)?; Ok(()) } - #[cfg(test)] pub fn insert(&mut self, key: &[u8], value: &V) -> io::Result<()> { self.fst_builder .insert(key, self.data.len() as u64) @@ -68,7 +65,7 @@ impl FstMapBuilder { } } -pub struct FstMap { +pub struct TermDictionary where V: BinarySerializable { fst_index: fst::Map, values_mmap: ReadOnlySource, _phantom_: PhantomData, @@ -89,10 +86,9 @@ fn open_fst_index(source: ReadOnlySource) -> io::Result { Ok(fst::Map::from(fst)) } -impl FstMap - where V: BinarySerializable +impl TermDictionary where V: BinarySerializable { - pub fn from_source(source: ReadOnlySource) -> io::Result> { + pub fn from_source(source: ReadOnlySource) -> io::Result> { let total_len = source.len(); let length_offset = total_len - 4; let mut split_len_buffer: &[u8] = &source.as_slice()[length_offset..]; @@ -101,7 +97,7 @@ impl FstMap let fst_source = source.slice(0, split_len); let values_source = source.slice(split_len, length_offset); let fst_index = open_fst_index(fst_source)?; - Ok(FstMap { + Ok(TermDictionary { fst_index: fst_index, values_mmap: values_source, _phantom_: PhantomData, @@ -109,7 +105,6 @@ impl FstMap } - /// /// This method deserialize this object, and returns it. pub(crate) fn read_value(&self, offset: u64) -> io::Result { let buffer = self.values_mmap.as_slice(); @@ -129,13 +124,13 @@ impl FstMap /// Returns a stream of all the sorted terms. - pub fn stream(&self) -> FstMapStreamer { + pub fn stream(&self) -> TermStreamer { self.range().into_stream() } /// Returns a stream of all the sorted terms in the given field. - pub fn stream_field(&self, field: Field) -> FstMapStreamer { + pub fn stream_field(&self, field: Field) -> TermStreamer { let start_term = Term::from_field_text(field, ""); let stop_term = Term::from_field_text(Field(field.0 + 1), ""); self.range() @@ -146,8 +141,8 @@ impl FstMap /// Returns a range builder, to stream all of the terms /// within an interval. - pub fn range(&self) -> FstMapStreamerBuilder { - FstMapStreamerBuilder::new(self, self.fst_index.range()) + pub fn range(&self) -> TermStreamerBuilder { + TermStreamerBuilder::new(self, self.fst_index.range()) } } @@ -159,21 +154,21 @@ mod tests { use fst::Streamer; #[test] - fn test_fstmap() { + fn test_term_dictionary() { let mut directory = RAMDirectory::create(); - let path = PathBuf::from("fstmap"); + let path = PathBuf::from("TermDictionary"); { let write = directory.open_write(&path).unwrap(); - let mut fstmap_builder = FstMapBuilder::new(write).unwrap(); - fstmap_builder.insert("abc".as_bytes(), &34u32).unwrap(); - fstmap_builder.insert("abcd".as_bytes(), &346u32).unwrap(); - fstmap_builder.finish().unwrap(); + let mut term_dictionary_builder = TermDictionaryBuilder::new(write).unwrap(); + term_dictionary_builder.insert("abc".as_bytes(), &34u32).unwrap(); + term_dictionary_builder.insert("abcd".as_bytes(), &346u32).unwrap(); + term_dictionary_builder.finish().unwrap(); } let source = directory.open_read(&path).unwrap(); - let fstmap: FstMap = FstMap::from_source(source).unwrap(); - assert_eq!(fstmap.get("abc"), Some(34u32)); - assert_eq!(fstmap.get("abcd"), Some(346u32)); - let mut stream = fstmap.stream(); + let term_dict: TermDictionary = TermDictionary::from_source(source).unwrap(); + assert_eq!(term_dict.get("abc"), Some(34u32)); + assert_eq!(term_dict.get("abcd"), Some(346u32)); + let mut stream = term_dict.stream(); assert_eq!(stream.next().unwrap(), "abc".as_bytes()); assert_eq!(stream.key(), "abc".as_bytes()); assert_eq!(stream.value(), 34u32); From 57a5547ae814cdc5fd33e923626ef859826f5bd3 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Fri, 19 May 2017 11:12:59 +0900 Subject: [PATCH 09/10] Comments and cleaning up API --- src/core/searcher.rs | 6 ++-- src/core/segment_reader.rs | 7 +++-- src/fastfield/mod.rs | 18 ++++++------ src/fastfield/reader.rs | 6 ++-- src/indexer/merger.rs | 4 +-- src/store/mod.rs | 4 +-- src/store/reader.rs | 25 ++++++++--------- src/termdict/merger.rs | 16 +++++------ src/termdict/mod.rs | 8 ++---- src/termdict/streamer.rs | 43 +++++++++++++++++++++++++--- src/termdict/termdict.rs | 57 ++++++++++++++++++++++++++------------ 11 files changed, 123 insertions(+), 71 deletions(-) diff --git a/src/core/searcher.rs b/src/core/searcher.rs index daa547f00..8f6d36b82 100644 --- a/src/core/searcher.rs +++ b/src/core/searcher.rs @@ -7,7 +7,7 @@ use query::Query; use DocId; use DocAddress; use schema::Term; -use termdict::FstMerger; +use termdict::TermMerger; use std::fmt; use postings::TermInfo; @@ -73,8 +73,8 @@ impl Searcher { /// /// # Warning /// This API is very likely to change in the future. - pub fn terms(&self) -> FstMerger { - FstMerger::from(self.segment_readers()) + pub fn terms(&self) -> TermMerger { + TermMerger::from(self.segment_readers()) } } diff --git a/src/core/segment_reader.rs b/src/core/segment_reader.rs index d8df2fa8f..57e8c2a40 100644 --- a/src/core/segment_reader.rs +++ b/src/core/segment_reader.rs @@ -136,14 +136,15 @@ impl SegmentReader { let source = try!(segment.open_read(SegmentComponent::TERMS)); let terms = try!(TermDictionary::from_source(source)); - let store_reader = StoreReader::from(try!(segment.open_read(SegmentComponent::STORE))); + let store_reader = + StoreReader::from_source(try!(segment.open_read(SegmentComponent::STORE))); let postings_shared_mmap = try!(segment.open_read(SegmentComponent::POSTINGS)); let fast_field_data = try!(segment.open_read(SegmentComponent::FASTFIELDS)); - let fast_fields_reader = try!(FastFieldsReader::open(fast_field_data)); + let fast_fields_reader = try!(FastFieldsReader::from_source(fast_field_data)); let fieldnorms_data = try!(segment.open_read(SegmentComponent::FIELDNORMS)); - let fieldnorms_reader = try!(FastFieldsReader::open(fieldnorms_data)); + let fieldnorms_reader = try!(FastFieldsReader::from_source(fieldnorms_data)); let positions_data = segment .open_read(SegmentComponent::POSITIONS) diff --git a/src/fastfield/mod.rs b/src/fastfield/mod.rs index 84fe5307a..ad2988ce4 100644 --- a/src/fastfield/mod.rs +++ b/src/fastfield/mod.rs @@ -97,7 +97,7 @@ mod tests { assert_eq!(source.len(), 38 as usize); } { - let fast_field_readers = FastFieldsReader::open(source).unwrap(); + let fast_field_readers = FastFieldsReader::from_source(source).unwrap(); let fast_field_reader: U64FastFieldReader = fast_field_readers.open_reader(*FIELD).unwrap(); assert_eq!(fast_field_reader.get(0), 13u64); @@ -131,7 +131,7 @@ mod tests { assert_eq!(source.len(), 63 as usize); } { - let fast_field_readers = FastFieldsReader::open(source).unwrap(); + let fast_field_readers = FastFieldsReader::from_source(source).unwrap(); let fast_field_reader: U64FastFieldReader = fast_field_readers.open_reader(*FIELD).unwrap(); assert_eq!(fast_field_reader.get(0), 4u64); @@ -167,7 +167,7 @@ mod tests { assert_eq!(source.len(), 36 as usize); } { - let fast_field_readers = FastFieldsReader::open(source).unwrap(); + let fast_field_readers = FastFieldsReader::from_source(source).unwrap(); let fast_field_reader: U64FastFieldReader = fast_field_readers.open_reader(*FIELD).unwrap(); for doc in 0..10_000 { @@ -200,7 +200,7 @@ mod tests { assert_eq!(source.len(), 80044 as usize); } { - let fast_field_readers = FastFieldsReader::open(source).unwrap(); + let fast_field_readers = FastFieldsReader::from_source(source).unwrap(); let fast_field_reader: U64FastFieldReader = fast_field_readers.open_reader(*FIELD).unwrap(); assert_eq!(fast_field_reader.get(0), 0u64); @@ -237,7 +237,7 @@ mod tests { assert_eq!(source.len(), 17711 as usize); } { - let fast_field_readers = FastFieldsReader::open(source).unwrap(); + let fast_field_readers = FastFieldsReader::from_source(source).unwrap(); let fast_field_reader: I64FastFieldReader = fast_field_readers.open_reader(i64_field).unwrap(); assert_eq!(fast_field_reader.min_value(), -100i64); @@ -268,7 +268,7 @@ mod tests { let source = directory.open_read(&path).unwrap(); { - let fast_field_readers = FastFieldsReader::open(source).unwrap(); + let fast_field_readers = FastFieldsReader::from_source(source).unwrap(); let fast_field_reader: I64FastFieldReader = fast_field_readers.open_reader(i64_field).unwrap(); assert_eq!(fast_field_reader.get(0u32), 0i64); @@ -301,7 +301,7 @@ mod tests { } let source = directory.open_read(&path).unwrap(); { - let fast_field_readers = FastFieldsReader::open(source).unwrap(); + let fast_field_readers = FastFieldsReader::from_source(source).unwrap(); let fast_field_reader: U64FastFieldReader = fast_field_readers.open_reader(*FIELD).unwrap(); let mut a = 0u64; @@ -359,7 +359,7 @@ mod tests { } let source = directory.open_read(&path).unwrap(); { - let fast_field_readers = FastFieldsReader::open(source).unwrap(); + let fast_field_readers = FastFieldsReader::from_source(source).unwrap(); let fast_field_reader: U64FastFieldReader = fast_field_readers.open_reader(*FIELD).unwrap(); b.iter(|| { @@ -390,7 +390,7 @@ mod tests { } let source = directory.open_read(&path).unwrap(); { - let fast_field_readers = FastFieldsReader::open(source).unwrap(); + let fast_field_readers = FastFieldsReader::from_source(source).unwrap(); let fast_field_reader: U64FastFieldReader = fast_field_readers.open_reader(*FIELD).unwrap(); b.iter(|| { diff --git a/src/fastfield/reader.rs b/src/fastfield/reader.rs index 7f4684fda..aa3050632 100644 --- a/src/fastfield/reader.rs +++ b/src/fastfield/reader.rs @@ -128,7 +128,7 @@ impl From> for U64FastFieldReader { serializer.close().unwrap(); } let source = directory.open_read(path).unwrap(); - let fast_field_readers = FastFieldsReader::open(source).unwrap(); + let fast_field_readers = FastFieldsReader::from_source(source).unwrap(); fast_field_readers.open_reader(field).unwrap() } } @@ -194,12 +194,12 @@ pub struct FastFieldsReader { } impl FastFieldsReader { - /// Opens the `FastFieldsReader` file + /// Opens a `FastFieldsReader` /// /// When opening the fast field reader, the /// the list of the offset is read (as a footer of the /// data file). - pub fn open(source: ReadOnlySource) -> io::Result { + pub fn from_source(source: ReadOnlySource) -> io::Result { let header_offset; let field_offsets: Vec<(Field, u32)>; { diff --git a/src/indexer/merger.rs b/src/indexer/merger.rs index f26177ea3..d08b4371f 100644 --- a/src/indexer/merger.rs +++ b/src/indexer/merger.rs @@ -12,7 +12,7 @@ use postings::Postings; use postings::DocSet; use fastfield::DeleteBitSet; use schema::{Schema, Field}; -use termdict::FstMerger; +use termdict::TermMerger; use fastfield::FastFieldSerializer; use fastfield::FastFieldReader; use store::StoreWriter; @@ -191,7 +191,7 @@ impl IndexMerger { fn write_postings(&self, postings_serializer: &mut PostingsSerializer) -> Result<()> { - let mut merged_terms = FstMerger::from(&self.readers[..]); + let mut merged_terms = TermMerger::from(&self.readers[..]); let mut delta_position_computer = DeltaPositionComputer::new(); let mut max_doc = 0; diff --git a/src/store/mod.rs b/src/store/mod.rs index dd7fa9390..8e2431d03 100644 --- a/src/store/mod.rs +++ b/src/store/mod.rs @@ -58,7 +58,7 @@ mod tests { let schema = write_lorem_ipsum_store(store_file, 1_000); let field_title = schema.get_field("title").unwrap(); let store_source = directory.open_read(path).unwrap(); - let store = StoreReader::from(store_source); + let store = StoreReader::from_source(store_source); for i in 0..1_000 { assert_eq!(*store.get(i).unwrap().get_first(field_title).unwrap().text(), format!("Doc {}", i)); @@ -82,7 +82,7 @@ mod tests { let path = Path::new("store"); write_lorem_ipsum_store(directory.open_write(path).unwrap(), 1_000); let store_source = directory.open_read(path).unwrap(); - let store = StoreReader::from(store_source); + let store = StoreReader::from_source(store_source); b.iter(|| { store.get(12).unwrap(); }); } diff --git a/src/store/reader.rs b/src/store/reader.rs index 5569a11a5..329ce5ae7 100644 --- a/src/store/reader.rs +++ b/src/store/reader.rs @@ -21,6 +21,17 @@ pub struct StoreReader { } impl StoreReader { + pub fn from_source(data: ReadOnlySource) -> StoreReader { + let (data_source, offset_index_source, max_doc) = split_source(data); + StoreReader { + data: data_source, + offset_index_source: offset_index_source, + current_block_offset: RefCell::new(usize::max_value()), + current_block: RefCell::new(Vec::new()), + max_doc: max_doc, + } + } + fn block_offset(&self, doc_id: DocId) -> (DocId, u64) { SkipList::from(self.offset_index_source.as_slice()) .seek(doc_id + 1) @@ -76,17 +87,3 @@ fn split_source(data: ReadOnlySource) -> (ReadOnlySource, ReadOnlySource, DocId) drop(data); res } - - -impl From for StoreReader { - fn from(data: ReadOnlySource) -> StoreReader { - let (data_source, offset_index_source, max_doc) = split_source(data); - StoreReader { - data: data_source, - offset_index_source: offset_index_source, - current_block_offset: RefCell::new(usize::max_value()), - current_block: RefCell::new(Vec::new()), - max_doc: max_doc, - } - } -} diff --git a/src/termdict/merger.rs b/src/termdict/merger.rs index e1d3fd155..b87b90165 100644 --- a/src/termdict/merger.rs +++ b/src/termdict/merger.rs @@ -46,18 +46,18 @@ impl<'a, V> Ord for HeapItem<'a, V> /// - the term /// - a slice with the ordinal of the segments containing /// the terms. -pub struct FstMerger<'a, V> +pub struct TermMerger<'a, V> where V: 'a + BinarySerializable { heap: BinaryHeap>, current_streamers: Vec>, } -impl<'a, V> FstMerger<'a, V> +impl<'a, V> TermMerger<'a, V> where V: 'a + BinarySerializable { - fn new(streams: Vec>) -> FstMerger<'a, V> { - FstMerger { + fn new(streams: Vec>) -> TermMerger<'a, V> { + TermMerger { heap: BinaryHeap::new(), current_streamers: streams .into_iter() @@ -130,18 +130,18 @@ impl<'a, V> FstMerger<'a, V> -impl<'a> From<&'a [SegmentReader]> for FstMerger<'a, TermInfo> +impl<'a> From<&'a [SegmentReader]> for TermMerger<'a, TermInfo> where TermInfo: BinarySerializable { - fn from(segment_readers: &'a [SegmentReader]) -> FstMerger<'a, TermInfo> { - FstMerger::new(segment_readers + fn from(segment_readers: &'a [SegmentReader]) -> TermMerger<'a, TermInfo> { + TermMerger::new(segment_readers .iter() .map(|reader| reader.terms().stream()) .collect()) } } -impl<'a, V> Streamer<'a> for FstMerger<'a, V> +impl<'a, V> Streamer<'a> for TermMerger<'a, V> where V: BinarySerializable { type Item = &'a [u8]; diff --git a/src/termdict/mod.rs b/src/termdict/mod.rs index 708538744..7e9db1d76 100644 --- a/src/termdict/mod.rs +++ b/src/termdict/mod.rs @@ -10,9 +10,7 @@ each term `&[u8]` to a `u64` that is in fact an address in a buffer. The value is then accessible via deserializing the value at this address. -Keys (`&[u8]`) in this datastructure are -sorted. - +Keys (`&[u8]`) in this datastructure are sorted. */ mod termdict; @@ -20,7 +18,7 @@ mod streamer; mod merger; pub use self::termdict::TermDictionary; -pub(crate) use self::termdict::TermDictionaryBuilder; +pub use self::termdict::TermDictionaryBuilder; pub use self::streamer::TermStreamer; pub use self::streamer::TermStreamerBuilder; -pub use self::merger::FstMerger; \ No newline at end of file +pub use self::merger::TermMerger; diff --git a/src/termdict/streamer.rs b/src/termdict/streamer.rs index c5784e6bd..9913e45d0 100644 --- a/src/termdict/streamer.rs +++ b/src/termdict/streamer.rs @@ -3,6 +3,8 @@ use fst::map::{StreamBuilder, Stream}; use common::BinarySerializable; use super::TermDictionary; +/// `TermStreamerBuilder` is an helper object used to define +/// a range of terms that should be streamed. pub struct TermStreamerBuilder<'a, V> where V: 'a + BinarySerializable { @@ -13,26 +15,32 @@ pub struct TermStreamerBuilder<'a, V> impl<'a, V> TermStreamerBuilder<'a, V> where V: 'a + BinarySerializable { + /// Limit the range to terms greater or equal to the bound pub fn ge>(mut self, bound: T) -> Self { self.stream_builder = self.stream_builder.ge(bound); self } + /// Limit the range to terms strictly greater than the bound pub fn gt>(mut self, bound: T) -> Self { self.stream_builder = self.stream_builder.gt(bound); self } + /// Limit the range to terms lesser or equal to the bound pub fn le>(mut self, bound: T) -> Self { self.stream_builder = self.stream_builder.le(bound); self } + /// Limit the range to terms lesser or equal to the bound pub fn lt>(mut self, bound: T) -> Self { self.stream_builder = self.stream_builder.lt(bound); self } + /// Creates the stream corresponding to the range + /// of terms defined using the `TermStreamerBuilder`. pub fn into_stream(self) -> TermStreamer<'a, V> { TermStreamer { fst_map: self.fst_map, @@ -42,9 +50,10 @@ impl<'a, V> TermStreamerBuilder<'a, V> } } - pub fn new(fst_map: &'a TermDictionary, - stream_builder: StreamBuilder<'a>) - -> TermStreamerBuilder<'a, V> { + /// Crates a new `TermStreamBuilder` + pub(crate) fn new(fst_map: &'a TermDictionary, + stream_builder: StreamBuilder<'a>) + -> TermStreamerBuilder<'a, V> { TermStreamerBuilder { fst_map: fst_map, stream_builder: stream_builder, @@ -55,7 +64,8 @@ impl<'a, V> TermStreamerBuilder<'a, V> - +/// `TermStreamer` acts as a cursor over a range of terms of a segment. +/// Terms are guaranteed to be sorted. pub struct TermStreamer<'a, V> where V: 'a + BinarySerializable { @@ -83,6 +93,9 @@ impl<'a, 'b, V> fst::Streamer<'b> for TermStreamer<'a, V> impl<'a, V> TermStreamer<'a, V> where V: 'a + BinarySerializable { + /// Advance position the stream on the next item. + /// Before the first call to `.advance()`, the stream + /// is an unitialized state. pub fn advance(&mut self) -> bool { if let Some((term, offset)) = self.stream.next() { self.buffer.clear(); @@ -94,10 +107,32 @@ impl<'a, V> TermStreamer<'a, V> } } + /// Accesses the current key. + /// + /// `.key()` should return the key that was returned + /// by the `.next()` method. + /// + /// If the end of the stream as been reached, and `.next()` + /// has been called and returned `None`, `.key()` remains + /// the value of the last key encounterred. + /// + /// Before any call to `.next()`, `.key()` returns an empty array. pub fn key(&self) -> &[u8] { &self.buffer } + /// Accesses the current value. + /// + /// Values are accessed in a lazy manner, their data is fetched + /// and deserialized only at the moment of the call to `.value()`. + /// + /// Calling `.value()` after the end of the stream will return the + /// last `.value()` encounterred. + /// + /// # Panics + /// + /// Calling `.value()` before the first call to `.advance()` or `.next()` + /// is undefined behavior. pub fn value(&self) -> V { self.fst_map .read_value(self.offset) diff --git a/src/termdict/termdict.rs b/src/termdict/termdict.rs index 232826234..d5c990e7f 100644 --- a/src/termdict/termdict.rs +++ b/src/termdict/termdict.rs @@ -14,13 +14,20 @@ fn convert_fst_error(e: fst::Error) -> io::Error { } -pub struct TermDictionaryBuilder where V: BinarySerializable { +/// Builder for the new term dictionary. +/// +/// Just like for the fst crate, all terms must be inserted in order. +pub struct TermDictionaryBuilder + where V: BinarySerializable +{ fst_builder: fst::MapBuilder, data: Vec, _phantom_: PhantomData, } impl TermDictionaryBuilder { + + /// Creates a new `TermDictionaryBuilder` pub fn new(w: W) -> io::Result> { let fst_builder = fst::MapBuilder::new(w).map_err(convert_fst_error)?; Ok(TermDictionaryBuilder { @@ -30,10 +37,13 @@ impl TermDictionaryBuilder { }) } - /// Horribly unsafe internal API + /// # Warning + /// Horribly dangerous internal API /// /// If used, it must be used by systematically alternating calls /// to insert_key and insert_value. + /// + /// Prefer using `.insert(key, value)` pub(crate) fn insert_key(&mut self, key: &[u8]) -> io::Result<()> { self.fst_builder .insert(key, self.data.len() as u64) @@ -41,12 +51,17 @@ impl TermDictionaryBuilder { Ok(()) } - /// Horribly unsafe internal API + /// # Warning + /// + /// Horribly dangerous internal API. See `.insert_key(...)`. pub(crate) fn insert_value(&mut self, value: &V) -> io::Result<()> { value.serialize(&mut self.data)?; Ok(()) } + /// Inserts a `(key, value)` pair in the term dictionary. + /// + /// *Keys have to be inserted in order.* pub fn insert(&mut self, key: &[u8], value: &V) -> io::Result<()> { self.fst_builder .insert(key, self.data.len() as u64) @@ -55,6 +70,8 @@ impl TermDictionaryBuilder { Ok(()) } + /// Finalize writing the builder, and returns the underlying + /// `Write` object. pub fn finish(self) -> io::Result { let mut file = self.fst_builder.into_inner().map_err(convert_fst_error)?; let footer_size = self.data.len() as u32; @@ -65,13 +82,15 @@ impl TermDictionaryBuilder { } } -pub struct TermDictionary where V: BinarySerializable { +/// Datastructure to access the `terms` of a segment. +pub struct TermDictionary + where V: BinarySerializable +{ fst_index: fst::Map, values_mmap: ReadOnlySource, _phantom_: PhantomData, } - fn open_fst_index(source: ReadOnlySource) -> io::Result { let fst = match source { ReadOnlySource::Anonymous(data) => { @@ -79,15 +98,16 @@ fn open_fst_index(source: ReadOnlySource) -> io::Result { .map_err(convert_fst_error)? } ReadOnlySource::Mmap(mmap_readonly) => { - Fst::from_mmap(mmap_readonly) - .map_err(convert_fst_error)? + Fst::from_mmap(mmap_readonly).map_err(convert_fst_error)? } }; Ok(fst::Map::from(fst)) } -impl TermDictionary where V: BinarySerializable +impl TermDictionary + where V: BinarySerializable { + /// Opens a `TermDictionary` given a data source. pub fn from_source(source: ReadOnlySource) -> io::Result> { let total_len = source.len(); let length_offset = total_len - 4; @@ -104,15 +124,14 @@ impl TermDictionary where V: BinarySerializable }) } - - /// This method deserialize this object, and returns it. + /// Deserialize and returns the value at address `offset` pub(crate) fn read_value(&self, offset: u64) -> io::Result { let buffer = self.values_mmap.as_slice(); let mut cursor = &buffer[(offset as usize)..]; V::deserialize(&mut cursor) } - /// Returns, if present the value associated to a given key. + /// Lookups the value corresponding to the key. pub fn get>(&self, key: K) -> Option { self.fst_index .get(key) @@ -120,16 +139,14 @@ impl TermDictionary where V: BinarySerializable self.read_value(offset) .expect("The fst is corrupted. Failed to deserialize a value.") }) - } + } - - /// Returns a stream of all the sorted terms. + /// A stream of all the sorted terms. [See also `.stream_field()`](#method.stream_field) pub fn stream(&self) -> TermStreamer { self.range().into_stream() } - - /// Returns a stream of all the sorted terms in the given field. + /// A stream of all the sorted terms in the given field. pub fn stream_field(&self, field: Field) -> TermStreamer { let start_term = Term::from_field_text(field, ""); let stop_term = Term::from_field_text(Field(field.0 + 1), ""); @@ -160,8 +177,12 @@ mod tests { { let write = directory.open_write(&path).unwrap(); let mut term_dictionary_builder = TermDictionaryBuilder::new(write).unwrap(); - term_dictionary_builder.insert("abc".as_bytes(), &34u32).unwrap(); - term_dictionary_builder.insert("abcd".as_bytes(), &346u32).unwrap(); + term_dictionary_builder + .insert("abc".as_bytes(), &34u32) + .unwrap(); + term_dictionary_builder + .insert("abcd".as_bytes(), &346u32) + .unwrap(); term_dictionary_builder.finish().unwrap(); } let source = directory.open_read(&path).unwrap(); From 7ee93fbed549e479699668896de9d0457bce4b5e Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Fri, 19 May 2017 20:08:04 +0900 Subject: [PATCH 10/10] Cleaning --- src/core/segment_reader.rs | 20 +++++++++++--------- src/indexer/merger.rs | 37 ++++++++++++++++++++++--------------- src/termdict/merger.rs | 6 +++--- src/termdict/streamer.rs | 9 +++++---- src/termdict/termdict.rs | 5 ++--- 5 files changed, 43 insertions(+), 34 deletions(-) diff --git a/src/core/segment_reader.rs b/src/core/segment_reader.rs index 57e8c2a40..ba05fb632 100644 --- a/src/core/segment_reader.rs +++ b/src/core/segment_reader.rs @@ -134,17 +134,19 @@ impl SegmentReader { /// Open a new segment for reading. pub fn open(segment: Segment) -> Result { - let source = try!(segment.open_read(SegmentComponent::TERMS)); - let terms = try!(TermDictionary::from_source(source)); - let store_reader = - StoreReader::from_source(try!(segment.open_read(SegmentComponent::STORE))); - let postings_shared_mmap = try!(segment.open_read(SegmentComponent::POSTINGS)); + let source = segment.open_read(SegmentComponent::TERMS)?; + let terms = TermDictionary::from_source(source)?; - let fast_field_data = try!(segment.open_read(SegmentComponent::FASTFIELDS)); - let fast_fields_reader = try!(FastFieldsReader::from_source(fast_field_data)); + let store_source = segment.open_read(SegmentComponent::STORE)?; + let store_reader = StoreReader::from_source(store_source); - let fieldnorms_data = try!(segment.open_read(SegmentComponent::FIELDNORMS)); - let fieldnorms_reader = try!(FastFieldsReader::from_source(fieldnorms_data)); + let postings_shared_mmap = segment.open_read(SegmentComponent::POSTINGS)?; + + let fast_field_data = segment.open_read(SegmentComponent::FASTFIELDS)?; + let fast_fields_reader = FastFieldsReader::from_source(fast_field_data)?; + + let fieldnorms_data = segment.open_read(SegmentComponent::FIELDNORMS)?; + let fieldnorms_reader = FastFieldsReader::from_source(fieldnorms_data)?; let positions_data = segment .open_read(SegmentComponent::POSITIONS) diff --git a/src/indexer/merger.rs b/src/indexer/merger.rs index d08b4371f..80fd140ab 100644 --- a/src/indexer/merger.rs +++ b/src/indexer/merger.rs @@ -189,7 +189,7 @@ impl IndexMerger { Ok(()) } - fn write_postings(&self, postings_serializer: &mut PostingsSerializer) -> Result<()> { + fn write_postings(&self, serializer: &mut PostingsSerializer) -> Result<()> { let mut merged_terms = TermMerger::from(&self.readers[..]); let mut delta_position_computer = DeltaPositionComputer::new(); @@ -214,7 +214,6 @@ impl IndexMerger { let mut last_field: Option = None; let mut segment_postings_option = SegmentPostingsOption::FreqAndPositions; - let mut need_to_call_new_field = false; while merged_terms.advance() { // Create the total list of doc ids @@ -240,7 +239,10 @@ impl IndexMerger { .expect("Encounterred a field that is not supposed to be indexed. Have you modified the index?"); last_field = Some(current_field); - need_to_call_new_field = true; + + // it is perfectly safe to call `.new_field` + // even if there is no postings associated. + serializer.new_field(current_field); } // Let's compute the list of non-empty posting lists @@ -262,17 +264,19 @@ impl IndexMerger { }) .collect(); + // At this point, `segment_postings` contains the posting list + // of all of the segments containing the given term. + // + // These segments are non-empty and advance has already been called. + if segment_postings.is_empty() { + // by continuing here, the `term` will be entirely removed. continue; } - if need_to_call_new_field { - postings_serializer.new_field(current_field); - last_field = Some(current_field); - need_to_call_new_field = false; - } - - postings_serializer.new_term(term_bytes)?; + // We know that there is at least one document containing + // the term, so we add it. + serializer.new_term(term_bytes)?; // We can now serialize this postings, by pushing each document to the // postings serializer. @@ -280,6 +284,8 @@ impl IndexMerger { for (segment_ord, mut segment_postings) in segment_postings { let old_to_new_doc_id = &merged_doc_id_map[segment_ord]; loop { + // `.advance()` has been called once before the loop. + // Hence we cannot use a `while segment_postings.advance()` loop. if let Some(remapped_doc_id) = old_to_new_doc_id[segment_postings.doc() as usize] { // we make sure to only write the term iff @@ -287,17 +293,18 @@ impl IndexMerger { let delta_positions: &[u32] = delta_position_computer .compute_delta_positions(segment_postings.positions()); - postings_serializer - .write_doc(remapped_doc_id, - segment_postings.term_freq(), - delta_positions)?; + let term_freq = segment_postings.term_freq(); + serializer + .write_doc(remapped_doc_id, term_freq, delta_positions)?; } if !segment_postings.advance() { break; } } } - postings_serializer.close_term()?; + + // closing the term. + serializer.close_term()?; } Ok(()) } diff --git a/src/termdict/merger.rs b/src/termdict/merger.rs index b87b90165..dbbc5f6e2 100644 --- a/src/termdict/merger.rs +++ b/src/termdict/merger.rs @@ -135,9 +135,9 @@ impl<'a> From<&'a [SegmentReader]> for TermMerger<'a, TermInfo> { fn from(segment_readers: &'a [SegmentReader]) -> TermMerger<'a, TermInfo> { TermMerger::new(segment_readers - .iter() - .map(|reader| reader.terms().stream()) - .collect()) + .iter() + .map(|reader| reader.terms().stream()) + .collect()) } } diff --git a/src/termdict/streamer.rs b/src/termdict/streamer.rs index 9913e45d0..13327172b 100644 --- a/src/termdict/streamer.rs +++ b/src/termdict/streamer.rs @@ -77,13 +77,14 @@ pub struct TermStreamer<'a, V> impl<'a, 'b, V> fst::Streamer<'b> for TermStreamer<'a, V> - where V: 'a + BinarySerializable + where V: 'b + BinarySerializable { - type Item = &'b [u8]; + type Item = (&'b [u8], V); - fn next(&'b mut self) -> Option<&'b [u8]> { + fn next(&'b mut self) -> Option<(&'b [u8], V)> { if self.advance() { - Some(&self.buffer) + let v = self.value(); + Some((&self.buffer, v)) } else { None } diff --git a/src/termdict/termdict.rs b/src/termdict/termdict.rs index d5c990e7f..e713a24d7 100644 --- a/src/termdict/termdict.rs +++ b/src/termdict/termdict.rs @@ -26,7 +26,6 @@ pub struct TermDictionaryBuilder } impl TermDictionaryBuilder { - /// Creates a new `TermDictionaryBuilder` pub fn new(w: W) -> io::Result> { let fst_builder = fst::MapBuilder::new(w).map_err(convert_fst_error)?; @@ -190,10 +189,10 @@ mod tests { assert_eq!(term_dict.get("abc"), Some(34u32)); assert_eq!(term_dict.get("abcd"), Some(346u32)); let mut stream = term_dict.stream(); - assert_eq!(stream.next().unwrap(), "abc".as_bytes()); + assert_eq!(stream.next().unwrap(), ("abc".as_bytes(), 34u32)); assert_eq!(stream.key(), "abc".as_bytes()); assert_eq!(stream.value(), 34u32); - assert_eq!(stream.next().unwrap(), "abcd".as_bytes()); + assert_eq!(stream.next().unwrap(), ("abcd".as_bytes(), 346u32)); assert_eq!(stream.key(), "abcd".as_bytes()); assert_eq!(stream.value(), 346u32); assert!(!stream.advance());