From 54fc557a6d85228a222a08c4ac194837cfeea41c Mon Sep 17 00:00:00 2001 From: Pascal Seitz Date: Tue, 3 May 2022 19:18:40 +0800 Subject: [PATCH] use a single memory arena to store all terms --- src/indexer/json_term_writer.rs | 21 ++++--- src/indexer/segment_writer.rs | 39 ++---------- src/postings/indexing_context.rs | 6 +- src/postings/json_postings_writer.rs | 3 +- src/postings/mod.rs | 1 - src/postings/postings_writer.rs | 19 +++--- src/postings/stacker/memory_arena.rs | 2 +- src/postings/stacker/mod.rs | 2 +- src/postings/stacker/term_hashmap.rs | 84 ++++++++++++++------------ src/query/query_parser/query_parser.rs | 2 +- src/schema/document.rs | 4 +- src/schema/schema.rs | 12 +--- src/schema/term.rs | 42 +++---------- 13 files changed, 95 insertions(+), 142 deletions(-) diff --git a/src/indexer/json_term_writer.rs b/src/indexer/json_term_writer.rs index 53de24907..56f1d827c 100644 --- a/src/indexer/json_term_writer.rs +++ b/src/indexer/json_term_writer.rs @@ -57,7 +57,7 @@ struct IndexingPositionsPerPath { impl IndexingPositionsPerPath { fn get_position(&mut self, term: &Term) -> &mut IndexingPosition { self.positions_per_path - .entry(murmurhash2(term.as_slice())) + .entry(murmurhash2(term.value_bytes())) .or_insert_with(Default::default) } } @@ -208,7 +208,7 @@ impl<'a> JsonTermWriter<'a> { pub fn wrap(term_buffer: &'a mut Term) -> Self { term_buffer.clear_with_type(Type::Json); let mut path_stack = Vec::with_capacity(10); - path_stack.push(5); + path_stack.push(5); // magic number? Self { term_buffer, path_stack, @@ -251,7 +251,7 @@ impl<'a> JsonTermWriter<'a> { #[cfg(test)] pub(crate) fn path(&self) -> &[u8] { let end_of_path = self.path_stack.last().cloned().unwrap_or(6); // TODO remove magic number - &self.term().as_slice()[..end_of_path - 1] + &self.term().value_bytes()[..end_of_path - 1] } pub fn set_fast_value(&mut self, val: T) { @@ -321,7 +321,7 @@ mod tests { let mut json_writer = JsonTermWriter::wrap(&mut term); json_writer.push_path_segment("color"); json_writer.set_str("red"); - assert_eq!(json_writer.term().as_slice(), b"color\x00sred") + assert_eq!(json_writer.term().value_bytes(), b"color\x00sred") } #[test] @@ -333,7 +333,7 @@ mod tests { json_writer.push_path_segment("color"); json_writer.set_fast_value(-4i64); assert_eq!( - json_writer.term().as_slice(), + json_writer.term().value_bytes(), b"color\x00i\x7f\xff\xff\xff\xff\xff\xff\xfc" ) } @@ -347,7 +347,7 @@ mod tests { json_writer.push_path_segment("color"); json_writer.set_fast_value(4u64); assert_eq!( - json_writer.term().as_slice(), + json_writer.term().value_bytes(), b"color\x00u\x00\x00\x00\x00\x00\x00\x00\x04" ) } @@ -361,7 +361,7 @@ mod tests { json_writer.push_path_segment("color"); json_writer.set_fast_value(4.0f64); assert_eq!( - json_writer.term().as_slice(), + json_writer.term().value_bytes(), b"color\x00f\xc0\x10\x00\x00\x00\x00\x00\x00" ) } @@ -376,7 +376,10 @@ mod tests { json_writer.set_str("something"); json_writer.push_path_segment("color"); json_writer.set_str("red"); - assert_eq!(json_writer.term().as_slice(), b"attribute\x01color\x00sred") + assert_eq!( + json_writer.term().value_bytes(), + b"attribute\x01color\x00sred" + ) } #[test] @@ -389,7 +392,7 @@ mod tests { json_writer.push_path_segment("hue"); json_writer.pop_path_segment(); json_writer.set_str("red"); - assert_eq!(json_writer.term().as_slice(), b"color\x00sred") + assert_eq!(json_writer.term().value_bytes(), b"color\x00sred") } #[test] diff --git a/src/indexer/segment_writer.rs b/src/indexer/segment_writer.rs index 73a4ab8e7..8c31ab2cd 100644 --- a/src/indexer/segment_writer.rs +++ b/src/indexer/segment_writer.rs @@ -6,8 +6,7 @@ use crate::fieldnorm::{FieldNormReaders, FieldNormsWriter}; use crate::indexer::json_term_writer::index_json_values; use crate::indexer::segment_serializer::SegmentSerializer; use crate::postings::{ - compute_table_size, serialize_postings, IndexingContext, IndexingPosition, - PerFieldPostingsWriter, PostingsWriter, + serialize_postings, IndexingContext, IndexingPosition, PerFieldPostingsWriter, PostingsWriter, }; use crate::schema::{FieldEntry, FieldType, FieldValue, Schema, Term, Value}; use crate::store::{StoreReader, StoreWriter}; @@ -16,25 +15,6 @@ use crate::tokenizer::{ }; use crate::{DocId, Document, Opstamp, SegmentComponent}; -/// Computes the initial size of the hash table. -/// -/// Returns the recommended initial table size as a power of 2. -/// -/// Note this is a very dumb way to compute log2, but it is easier to proofread that way. -fn compute_initial_table_size(per_thread_memory_budget: usize) -> crate::Result { - let table_memory_upper_bound = per_thread_memory_budget / 3; - (10..20) // We cap it at 2^19 = 512K capacity. - .map(|power| 1 << power) - .take_while(|capacity| compute_table_size(*capacity) < table_memory_upper_bound) - .last() - .ok_or_else(|| { - crate::TantivyError::InvalidArgument(format!( - "per thread memory budget (={per_thread_memory_budget}) is too small. Raise the \ - memory budget or lower the number of threads." - )) - }) -} - fn remap_doc_opstamps( opstamps: Vec, doc_id_mapping_opt: Option<&DocIdMapping>, @@ -78,12 +58,11 @@ impl SegmentWriter { /// - segment: The segment being written /// - schema pub fn for_segment( - memory_budget_in_bytes: usize, + _memory_budget_in_bytes: usize, segment: Segment, schema: Schema, ) -> crate::Result { let tokenizer_manager = segment.index().tokenizers().clone(); - let table_size = compute_initial_table_size(memory_budget_in_bytes)?; let segment_serializer = SegmentSerializer::for_segment(segment, false)?; let per_field_postings_writers = PerFieldPostingsWriter::for_schema(&schema); let per_field_text_analyzers = schema @@ -106,7 +85,7 @@ impl SegmentWriter { .collect(); Ok(SegmentWriter { max_doc: 0, - ctx: IndexingContext::new(table_size), + ctx: IndexingContext::new(), per_field_postings_writers, fieldnorms_writer: FieldNormsWriter::for_schema(&schema), segment_serializer, @@ -224,7 +203,7 @@ impl SegmentWriter { let mut indexing_position = IndexingPosition::default(); for mut token_stream in token_streams { - //assert_eq!(term_buffer.as_slice().len(), 5); + // assert_eq!(term_buffer.as_slice().len(), 5); postings_writer.index_text( doc_id, &mut *token_stream, @@ -419,7 +398,6 @@ pub fn prepare_doc_for_store(doc: Document, schema: &Schema) -> Document { #[cfg(test)] mod tests { - use super::compute_initial_table_size; use crate::collector::Count; use crate::indexer::json_term_writer::JsonTermWriter; use crate::postings::TermInfo; @@ -430,15 +408,6 @@ mod tests { use crate::tokenizer::{PreTokenizedString, Token}; use crate::{DateTime, DocAddress, DocSet, Document, Index, Postings, Term, TERMINATED}; - #[test] - fn test_hashmap_size() { - assert_eq!(compute_initial_table_size(100_000).unwrap(), 1 << 11); - assert_eq!(compute_initial_table_size(1_000_000).unwrap(), 1 << 14); - assert_eq!(compute_initial_table_size(10_000_000).unwrap(), 1 << 17); - assert_eq!(compute_initial_table_size(1_000_000_000).unwrap(), 1 << 19); - assert_eq!(compute_initial_table_size(4_000_000_000).unwrap(), 1 << 19); - } - #[test] fn test_prepare_for_store() { let mut schema_builder = Schema::builder(); diff --git a/src/postings/indexing_context.rs b/src/postings/indexing_context.rs index 3f80f78d6..cffffa9a7 100644 --- a/src/postings/indexing_context.rs +++ b/src/postings/indexing_context.rs @@ -5,18 +5,20 @@ use crate::postings::stacker::MemoryArena; pub(crate) struct IndexingContext { /// Arena is a memory arena that stores posting lists / term frequencies / positions. pub arena: MemoryArena, + pub arena_terms: MemoryArena, } impl IndexingContext { /// Create a new IndexingContext given the size of the term hash map. - pub(crate) fn new(table_size: usize) -> IndexingContext { + pub(crate) fn new() -> IndexingContext { IndexingContext { arena: MemoryArena::new(), + arena_terms: MemoryArena::new(), } } /// Returns the memory usage for the inverted index memory arenas, in bytes. pub(crate) fn mem_usage(&self) -> usize { - self.arena.mem_usage() + self.arena.mem_usage() + self.arena_terms.mem_usage() } } diff --git a/src/postings/json_postings_writer.rs b/src/postings/json_postings_writer.rs index 204fc586d..2a57162dc 100644 --- a/src/postings/json_postings_writer.rs +++ b/src/postings/json_postings_writer.rs @@ -1,5 +1,6 @@ use std::io; +use super::stacker::TermHashMap; use crate::fastfield::MultiValuedFastFieldWriter; use crate::indexer::doc_id_mapping::DocIdMapping; use crate::postings::postings_writer::SpecializedPostingsWriter; @@ -13,8 +14,6 @@ use crate::schema::Type; use crate::tokenizer::TokenStream; use crate::{DocId, Term}; -use super::stacker::TermHashMap; - #[derive(Default)] pub(crate) struct JsonPostingsWriter { str_posting_writer: SpecializedPostingsWriter, diff --git a/src/postings/mod.rs b/src/postings/mod.rs index 8f4e3078a..ad4ec5f44 100644 --- a/src/postings/mod.rs +++ b/src/postings/mod.rs @@ -26,7 +26,6 @@ pub(crate) use self::postings_writer::{serialize_postings, IndexingPosition, Pos pub use self::segment_postings::SegmentPostings; pub use self::serializer::{FieldSerializer, InvertedIndexSerializer}; pub(crate) use self::skip::{BlockInfo, SkipReader}; -pub(crate) use self::stacker::compute_table_size; pub use self::term_info::TermInfo; pub(crate) type UnorderedTermId = u64; diff --git a/src/postings/postings_writer.rs b/src/postings/postings_writer.rs index cb7f0fbe7..f3aef8e33 100644 --- a/src/postings/postings_writer.rs +++ b/src/postings/postings_writer.rs @@ -1,7 +1,6 @@ use std::collections::HashMap; use std::io; use std::marker::PhantomData; -use std::ops::Range; use fnv::FnvHashMap; @@ -40,7 +39,7 @@ pub(crate) fn serialize_postings( let mut term_offsets: Vec<(Term<&[u8]>, Addr, UnorderedTermId)> = Vec::with_capacity(postings_writer.term_map().len()); - term_offsets.extend(postings_writer.term_map().iter()); + term_offsets.extend(postings_writer.term_map().iter(&ctx.arena_terms)); term_offsets.sort_unstable_by_key(|(k, _, _)| k.clone()); let field_entry = schema.get_field_entry(field); @@ -121,7 +120,7 @@ pub(crate) trait PostingsWriter { indexing_position: &mut IndexingPosition, mut term_id_fast_field_writer_opt: Option<&mut MultiValuedFastFieldWriter>, ) { - let end_of_path_idx = term_buffer.as_slice().len(); + let end_of_path_idx = term_buffer.value_bytes().len(); let mut num_tokens = 0; let mut end_position = 0; token_stream.process(&mut |token: &Token| { @@ -183,7 +182,7 @@ impl SpecializedPostingsWriter { term_index: &TermHashMap, serializer: &mut FieldSerializer, ) -> io::Result<()> { - let recorder: Rec = term_index.read(addr); + let recorder: Rec = term_index.read(addr, &ctx.arena_terms); let term_doc_freq = recorder.term_doc_freq().unwrap_or(0u32); serializer.new_term(term.value_bytes(), term_doc_freq)?; recorder.serialize(&ctx.arena, doc_id_map, serializer, buffer_lender); @@ -208,11 +207,14 @@ impl PostingsWriter for SpecializedPostingsWriter { term: &Term, ctx: &mut IndexingContext, ) -> UnorderedTermId { - //debug_assert!(term.as_slice().len() >= 4); + //debug_assert!(term.value_bytes().len() >= 1); self.total_num_tokens += 1; let arena = &mut ctx.arena; - self.term_map - .mutate_or_create(term.as_slice(), |opt_recorder: Option| { + let arena_terms = &mut ctx.arena_terms; + self.term_map.mutate_or_create( + term.value_bytes(), + arena_terms, + |opt_recorder: Option| { if let Some(mut recorder) = opt_recorder { let current_doc = recorder.current_doc(); if current_doc != doc { @@ -227,7 +229,8 @@ impl PostingsWriter for SpecializedPostingsWriter { recorder.record_position(position, arena); recorder } - }) as UnorderedTermId + }, + ) as UnorderedTermId } fn serialize( diff --git a/src/postings/stacker/memory_arena.rs b/src/postings/stacker/memory_arena.rs index ffc6a49fe..3b704442a 100644 --- a/src/postings/stacker/memory_arena.rs +++ b/src/postings/stacker/memory_arena.rs @@ -24,7 +24,7 @@ //! stores your object using `ptr::write_unaligned` and `ptr::read_unaligned`. use std::{mem, ptr}; -const NUM_BITS_PAGE_ADDR: usize = 17; +const NUM_BITS_PAGE_ADDR: usize = 20; const PAGE_SIZE: usize = 1 << NUM_BITS_PAGE_ADDR; // pages are 1 MB large /// Represents a pointer into the `MemoryArena` diff --git a/src/postings/stacker/mod.rs b/src/postings/stacker/mod.rs index 19fff9216..661815f23 100644 --- a/src/postings/stacker/mod.rs +++ b/src/postings/stacker/mod.rs @@ -4,4 +4,4 @@ mod term_hashmap; pub(crate) use self::expull::ExpUnrolledLinkedList; pub(crate) use self::memory_arena::{Addr, MemoryArena}; -pub(crate) use self::term_hashmap::{compute_table_size, TermHashMap}; +pub(crate) use self::term_hashmap::TermHashMap; diff --git a/src/postings/stacker/term_hashmap.rs b/src/postings/stacker/term_hashmap.rs index c2c985caa..01b6e4cfe 100644 --- a/src/postings/stacker/term_hashmap.rs +++ b/src/postings/stacker/term_hashmap.rs @@ -1,6 +1,6 @@ +use std::convert::TryInto; use std::{iter, mem, slice}; -use byteorder::{ByteOrder, NativeEndian}; use murmurhash32::murmurhash2; use super::{Addr, MemoryArena}; @@ -8,13 +8,6 @@ use crate::postings::stacker::memory_arena::store; use crate::postings::UnorderedTermId; use crate::Term; -/// Returns the actual memory size in bytes -/// required to create a table with a given capacity. -/// required to create a table of size -pub(crate) fn compute_table_size(capacity: usize) -> usize { - capacity * mem::size_of::() -} - /// `KeyValue` is the item stored in the hash table. /// The key is actually a `BytesRef` object stored in an external memory arena. /// The `value_addr` also points to an address in the memory arena. @@ -52,7 +45,6 @@ impl KeyValue { /// or copying the key as long as there is no insert. pub struct TermHashMap { table: Box<[KeyValue]>, - memory_arena: MemoryArena, mask: usize, occupied: Vec, len: usize, @@ -82,18 +74,21 @@ impl QuadraticProbing { } } -pub struct Iter<'a> { +pub struct Iter<'a, 'm> { hashmap: &'a TermHashMap, + memory_arena: &'m MemoryArena, inner: slice::Iter<'a, usize>, } -impl<'a> Iterator for Iter<'a> { - type Item = (Term<&'a [u8]>, Addr, UnorderedTermId); +impl<'a, 'm> Iterator for Iter<'a, 'm> { + type Item = (Term<&'m [u8]>, Addr, UnorderedTermId); fn next(&mut self) -> Option { self.inner.next().cloned().map(move |bucket: usize| { let kv = self.hashmap.table[bucket]; - let (key, offset): (&'a [u8], Addr) = self.hashmap.get_key_value(kv.key_value_addr); + let (key, offset): (&'m [u8], Addr) = self + .hashmap + .get_key_value(kv.key_value_addr, self.memory_arena); (Term::wrap(key), offset, kv.unordered_term_id) }) } @@ -113,21 +108,19 @@ impl TermHashMap { pub(crate) fn new(table_size: usize) -> TermHashMap { assert!(table_size > 0); let table_size_power_of_2 = compute_previous_power_of_two(table_size); - let memory_arena = MemoryArena::new(); let table: Vec = iter::repeat(KeyValue::default()) .take(table_size_power_of_2) .collect(); TermHashMap { table: table.into_boxed_slice(), - memory_arena, mask: table_size_power_of_2 - 1, occupied: Vec::with_capacity(table_size_power_of_2 / 2), len: 0, } } - pub fn read(&self, addr: Addr) -> Item { - self.memory_arena.read(addr) + pub fn read(&self, addr: Addr, memory_arena: &MemoryArena) -> Item { + memory_arena.read(addr) } fn probe(&self, hash: u32) -> QuadraticProbing { @@ -136,6 +129,8 @@ impl TermHashMap { pub fn mem_usage(&self) -> usize { self.table.len() * mem::size_of::() + + self.occupied.len() + * std::mem::size_of_val(&self.occupied.get(0).cloned().unwrap_or_default()) } fn is_saturated(&self) -> bool { @@ -143,16 +138,22 @@ impl TermHashMap { } #[inline] - fn get_key_value(&self, addr: Addr) -> (&[u8], Addr) { - let data = self.memory_arena.slice_from(addr); - let key_bytes_len = NativeEndian::read_u16(data) as usize; - let key_bytes: &[u8] = &data[2..][..key_bytes_len]; + fn get_key_value<'m>(&self, addr: Addr, memory_arena: &'m MemoryArena) -> (&'m [u8], Addr) { + let data = memory_arena.slice_from(addr); + let (key_bytes_len_enc, data) = data.split_at(2); + let key_bytes_len: u16 = u16::from_ne_bytes(key_bytes_len_enc.try_into().unwrap()); + let key_bytes: &[u8] = &data[..key_bytes_len as usize]; (key_bytes, addr.offset(2u32 + key_bytes_len as u32)) } #[inline] - fn get_value_addr_if_key_match(&self, target_key: &[u8], addr: Addr) -> Option { - let (stored_key, value_addr) = self.get_key_value(addr); + fn get_value_addr_if_key_match( + &self, + target_key: &[u8], + addr: Addr, + memory_arena: &mut MemoryArena, + ) -> Option { + let (stored_key, value_addr) = self.get_key_value(addr, memory_arena); if stored_key == target_key { Some(value_addr) } else { @@ -176,10 +177,11 @@ impl TermHashMap { self.len } - pub fn iter(&self) -> Iter<'_> { + pub fn iter<'a, 'm>(&'a self, memory_arena: &'m MemoryArena) -> Iter<'a, 'm> { Iter { inner: self.occupied.iter(), hashmap: self, + memory_arena, } } @@ -216,6 +218,7 @@ impl TermHashMap { pub fn mutate_or_create( &mut self, key: &[u8], + memory_arena: &mut MemoryArena, mut updater: TMutator, ) -> UnorderedTermId where @@ -226,28 +229,33 @@ impl TermHashMap { self.resize(); } let hash = murmurhash2(key); + let mut probe = self.probe(hash); loop { let bucket = probe.next_probe(); let kv: KeyValue = self.table[bucket]; + if kv.is_empty() { // The key does not exists yet. let val = updater(None); let num_bytes = std::mem::size_of::() + key.len() + std::mem::size_of::(); - let key_addr = self.memory_arena.allocate_space(num_bytes); + let key_addr = memory_arena.allocate_space(num_bytes); { - let data = self.memory_arena.slice_mut(key_addr, num_bytes); - NativeEndian::write_u16(data, key.len() as u16); - let stop = 2 + key.len(); - data[2..stop].copy_from_slice(key); + let data = memory_arena.slice_mut(key_addr, num_bytes); + let (key_len, data) = data.split_at_mut(2); + key_len.copy_from_slice(&(key.len() as u16).to_le_bytes()); + let stop = key.len(); + data[..key.len()].copy_from_slice(key); store(&mut data[stop..], val); } return self.set_bucket(hash, key_addr, bucket); } else if kv.hash == hash { - if let Some(val_addr) = self.get_value_addr_if_key_match(key, kv.key_value_addr) { - let v = self.memory_arena.read(val_addr); + if let Some(val_addr) = + self.get_value_addr_if_key_match(key, kv.key_value_addr, memory_arena) + { + let v = memory_arena.read(val_addr); let new_v = updater(Some(v)); - self.memory_arena.write_at(val_addr, new_v); + memory_arena.write_at(val_addr, new_v); return kv.unordered_term_id; } } @@ -261,26 +269,28 @@ mod tests { use std::collections::HashMap; use super::{compute_previous_power_of_two, TermHashMap}; + use crate::postings::stacker::MemoryArena; #[test] fn test_hash_map() { + let mut arena = MemoryArena::new(); let mut hash_map: TermHashMap = TermHashMap::new(1 << 18); - hash_map.mutate_or_create(b"abc", |opt_val: Option| { + hash_map.mutate_or_create(b"abc", &mut arena, |opt_val: Option| { assert_eq!(opt_val, None); 3u32 }); - hash_map.mutate_or_create(b"abcd", |opt_val: Option| { + hash_map.mutate_or_create(b"abcd", &mut arena, |opt_val: Option| { assert_eq!(opt_val, None); 4u32 }); - hash_map.mutate_or_create(b"abc", |opt_val: Option| { + hash_map.mutate_or_create(b"abc", &mut arena, |opt_val: Option| { assert_eq!(opt_val, Some(3u32)); 5u32 }); let mut vanilla_hash_map = HashMap::new(); - let iter_values = hash_map.iter(); + let iter_values = hash_map.iter(&arena); for (key, addr, _) in iter_values { - let val: u32 = hash_map.memory_arena.read(addr); + let val: u32 = arena.read(addr); vanilla_hash_map.insert(key.to_owned(), val); } assert_eq!(vanilla_hash_map.len(), 2); diff --git a/src/query/query_parser/query_parser.rs b/src/query/query_parser/query_parser.rs index 354638111..4274403a9 100644 --- a/src/query/query_parser/query_parser.rs +++ b/src/query/query_parser/query_parser.rs @@ -718,7 +718,7 @@ fn generate_literals_for_json_object( } json_term_writer.close_path_and_set_type(Type::Str); drop(json_term_writer); - let term_num_bytes = term.as_slice().len(); + let term_num_bytes = term.value_bytes().len(); let mut token_stream = text_analyzer.token_stream(phrase); let mut terms: Vec<(usize, Term)> = Vec::new(); token_stream.process(&mut |token| { diff --git a/src/schema/document.rs b/src/schema/document.rs index f0df92b67..0c1e2fe22 100644 --- a/src/schema/document.rs +++ b/src/schema/document.rs @@ -78,9 +78,7 @@ impl Document { /// Adding a facet to the document. pub fn add_facet(&mut self, field: Field, path: F) - where - Facet: From, - { + where Facet: From { let facet = Facet::from(path); let value = Value::Facet(facet); self.add_field_value(field, value); diff --git a/src/schema/schema.rs b/src/schema/schema.rs index f1d10a96b..235d0412d 100644 --- a/src/schema/schema.rs +++ b/src/schema/schema.rs @@ -347,9 +347,7 @@ impl Schema { impl Serialize for Schema { fn serialize(&self, serializer: S) -> Result - where - S: Serializer, - { + where S: Serializer { let mut seq = serializer.serialize_seq(Some(self.0.fields.len()))?; for e in &self.0.fields { seq.serialize_element(e)?; @@ -360,9 +358,7 @@ impl Serialize for Schema { impl<'de> Deserialize<'de> for Schema { fn deserialize(deserializer: D) -> Result - where - D: Deserializer<'de>, - { + where D: Deserializer<'de> { struct SchemaVisitor; impl<'de> Visitor<'de> for SchemaVisitor { @@ -373,9 +369,7 @@ impl<'de> Deserialize<'de> for Schema { } fn visit_seq(self, mut seq: A) -> Result - where - A: SeqAccess<'de>, - { + where A: SeqAccess<'de> { let mut schema = SchemaBuilder { fields: Vec::with_capacity(seq.size_hint().unwrap_or(0)), fields_map: HashMap::with_capacity(seq.size_hint().unwrap_or(0)), diff --git a/src/schema/term.rs b/src/schema/term.rs index ddaefc9a6..006af1168 100644 --- a/src/schema/term.rs +++ b/src/schema/term.rs @@ -172,17 +172,15 @@ impl Term { } impl Ord for Term -where - B: AsRef<[u8]>, +where B: AsRef<[u8]> { fn cmp(&self, other: &Self) -> std::cmp::Ordering { - self.as_slice().cmp(other.as_slice()) + self.value_bytes().cmp(other.value_bytes()) } } impl PartialOrd for Term -where - B: AsRef<[u8]>, +where B: AsRef<[u8]> { fn partial_cmp(&self, other: &Self) -> Option { Some(self.cmp(other)) @@ -190,19 +188,17 @@ where } impl PartialEq for Term -where - B: AsRef<[u8]>, +where B: AsRef<[u8]> { fn eq(&self, other: &Self) -> bool { - self.as_slice() == other.as_slice() + self.value_bytes() == other.value_bytes() } } impl Eq for Term where B: AsRef<[u8]> {} impl Hash for Term -where - B: AsRef<[u8]>, +where B: AsRef<[u8]> { fn hash(&self, state: &mut H) { self.data.as_ref().hash(state) @@ -210,8 +206,7 @@ where } impl Term -where - B: AsRef<[u8]>, +where B: AsRef<[u8]> { /// Wraps a object holding bytes pub fn wrap(data: B) -> Term { @@ -224,10 +219,6 @@ where fn typ_code(&self) -> u8 { self.field_type as u8 - //*self - //.as_slice() - //.get(4) - //.expect("the byte representation is too short") } /// Return the type of the term. @@ -238,9 +229,6 @@ where /// Returns the field. pub fn field(&self) -> Field { self.field - //let mut field_id_bytes = [0u8; 4]; - //field_id_bytes.copy_from_slice(&self.data.as_ref()[..4]); - //Field::from_field_id(u32::from_be_bytes(field_id_bytes)) } /// Returns the text associated with the term. @@ -248,9 +236,6 @@ where /// Returns None if the field is not of string type /// or if the bytes are not valid utf-8. pub fn as_str(&self) -> Option<&str> { - //if self.as_slice().len() < 5 { - //return None; - //} if self.typ() != Type::Str { return None; } @@ -266,14 +251,6 @@ where pub fn value_bytes(&self) -> &[u8] { &self.data.as_ref() } - - /// Returns the underlying `&[u8]`. - /// - /// Do NOT rely on this byte representation in the index. - /// This value is likely to change in the future. - pub(crate) fn as_slice(&self) -> &[u8] { - self.data.as_ref() - } } fn write_opt(f: &mut fmt::Formatter, val_opt: Option) -> fmt::Result { @@ -344,8 +321,7 @@ fn debug_value_bytes(typ: Type, bytes: &[u8], f: &mut fmt::Formatter) -> fmt::Re } impl fmt::Debug for Term -where - B: AsRef<[u8]>, +where B: AsRef<[u8]> { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { let field_id = self.field().field_id(); @@ -380,6 +356,6 @@ mod tests { let term = Term::from_field_u64(count_field, 983u64); assert_eq!(term.field(), count_field); assert_eq!(term.typ(), Type::U64); - assert_eq!(term.as_slice().len(), super::FAST_VALUE_TERM_LEN); + assert_eq!(term.value_bytes().len(), super::FAST_VALUE_TERM_LEN); } }