Field type with codes (#1255)

* Term are now typed.

This change is backward compatible:
While the Term has a byte representation that is modified, a Term itself
is a transient object that is not serialized as is in the index.

Its .field() and .value_bytes() on the other hand are unchanged.
This change offers better Debug information for terms.

While not necessary it also will help in the support for JSON types.

* Renamed Hierarchical Facet -> Facet
This commit is contained in:
Paul Masurel
2022-01-07 20:49:00 +09:00
committed by GitHub
parent 1c6d9bdc6a
commit 732f6847c0
19 changed files with 405 additions and 289 deletions

View File

@@ -100,7 +100,7 @@ impl SegmentReader {
let field_entry = self.schema.get_field_entry(field);
match field_entry.field_type() {
FieldType::HierarchicalFacet(_) => {
FieldType::Facet(_) => {
let term_ords_reader = self.fast_fields().u64s(field)?;
let termdict = self
.termdict_composite

View File

@@ -110,7 +110,7 @@ impl FastValue for u64 {
fn fast_field_cardinality(field_type: &FieldType) -> Option<Cardinality> {
match *field_type {
FieldType::U64(ref integer_options) => integer_options.get_fastfield_cardinality(),
FieldType::HierarchicalFacet(_) => Some(Cardinality::MultiValues),
FieldType::Facet(_) => Some(Cardinality::MultiValues),
_ => None,
}
}

View File

@@ -40,7 +40,7 @@ fn type_and_cardinality(field_type: &FieldType) -> Option<(FastType, Cardinality
FieldType::Date(options) => options
.get_fastfield_cardinality()
.map(|cardinality| (FastType::Date, cardinality)),
FieldType::HierarchicalFacet(_) => Some((FastType::U64, Cardinality::MultiValues)),
FieldType::Facet(_) => Some((FastType::U64, Cardinality::MultiValues)),
_ => None,
}
}

View File

@@ -54,7 +54,7 @@ impl FastFieldsWriter {
None => {}
}
}
FieldType::HierarchicalFacet(_) => {
FieldType::Facet(_) => {
let fast_field_writer = MultiValuedFastFieldWriter::new(field, true);
multi_values_writers.push(fast_field_writer);
}

View File

@@ -294,10 +294,10 @@ impl IndexMerger {
for (field, field_entry) in self.schema.fields() {
let field_type = field_entry.field_type();
match field_type {
FieldType::HierarchicalFacet(_) => {
FieldType::Facet(_) => {
let term_ordinal_mapping = term_ord_mappings
.remove(&field)
.expect("Logic Error in Tantivy (Please report). HierarchicalFact field should have required a\
.expect("Logic Error in Tantivy (Please report). Facet field should have required a\
`term_ordinal_mapping`.");
self.write_hierarchical_facet_field(
field,
@@ -844,7 +844,7 @@ impl IndexMerger {
}
let mut term_ord_mapping_opt = match field_type {
FieldType::HierarchicalFacet(_) => Some(TermOrdinalMapping::new(max_term_ords)),
FieldType::Facet(_) => Some(TermOrdinalMapping::new(max_term_ords)),
_ => None,
};

View File

@@ -2,7 +2,6 @@ use super::{
doc_id_mapping::{get_doc_id_mapping_from_field, DocIdMapping},
operation::AddOperation,
};
use crate::fastfield::FastFieldsWriter;
use crate::fieldnorm::{FieldNormReaders, FieldNormsWriter};
use crate::indexer::segment_serializer::SegmentSerializer;
use crate::postings::compute_table_size;
@@ -18,6 +17,7 @@ use crate::tokenizer::{FacetTokenizer, TextAnalyzer};
use crate::tokenizer::{TokenStreamChain, Tokenizer};
use crate::Opstamp;
use crate::{core::Segment, store::StoreWriter};
use crate::{fastfield::FastFieldsWriter, schema::Type};
use crate::{DocId, SegmentComponent};
/// Computes the initial size of the hash table.
@@ -173,8 +173,8 @@ impl SegmentWriter {
let (term_buffer, multifield_postings) =
(&mut self.term_buffer, &mut self.multifield_postings);
match *field_entry.field_type() {
FieldType::HierarchicalFacet(_) => {
term_buffer.set_field(field);
FieldType::Facet(_) => {
term_buffer.set_field(Type::Facet, field);
for field_value in field_values {
let facet = field_value.value().facet().ok_or_else(make_schema_error)?;
let facet_str = facet.encoded_str();
@@ -238,7 +238,7 @@ impl SegmentWriter {
}
FieldType::U64(_) => {
for field_value in field_values {
term_buffer.set_field(field_value.field());
term_buffer.set_field(Type::U64, field_value.field());
let u64_val = field_value
.value()
.u64_value()
@@ -249,7 +249,7 @@ impl SegmentWriter {
}
FieldType::Date(_) => {
for field_value in field_values {
term_buffer.set_field(field_value.field());
term_buffer.set_field(Type::Date, field_value.field());
let date_val = field_value
.value()
.date_value()
@@ -260,7 +260,7 @@ impl SegmentWriter {
}
FieldType::I64(_) => {
for field_value in field_values {
term_buffer.set_field(field_value.field());
term_buffer.set_field(Type::I64, field_value.field());
let i64_val = field_value
.value()
.i64_value()
@@ -271,7 +271,7 @@ impl SegmentWriter {
}
FieldType::F64(_) => {
for field_value in field_values {
term_buffer.set_field(field_value.field());
term_buffer.set_field(Type::F64, field_value.field());
let f64_val = field_value
.value()
.f64_value()
@@ -282,7 +282,7 @@ impl SegmentWriter {
}
FieldType::Bytes(_) => {
for field_value in field_values {
term_buffer.set_field(field_value.field());
term_buffer.set_field(Type::Bytes, field_value.field());
let bytes = field_value
.value()
.bytes_value()

View File

@@ -5,8 +5,8 @@ use crate::postings::recorder::{
};
use crate::postings::UnorderedTermId;
use crate::postings::{FieldSerializer, InvertedIndexSerializer};
use crate::schema::IndexRecordOption;
use crate::schema::{Field, FieldEntry, FieldType, Schema, Term};
use crate::schema::{IndexRecordOption, Type};
use crate::termdict::TermOrdinal;
use crate::tokenizer::TokenStream;
use crate::tokenizer::{Token, MAX_TOKEN_LEN};
@@ -39,9 +39,7 @@ fn posting_from_field_entry(field_entry: &FieldEntry) -> Box<dyn PostingsWriter>
| FieldType::F64(_)
| FieldType::Date(_)
| FieldType::Bytes(_)
| FieldType::HierarchicalFacet(_) => {
SpecializedPostingsWriter::<NothingRecorder>::new_boxed()
}
| FieldType::Facet(_) => SpecializedPostingsWriter::<NothingRecorder>::new_boxed(),
}
}
@@ -53,11 +51,11 @@ pub struct MultiFieldPostingsWriter {
}
fn make_field_partition(
term_offsets: &[(&[u8], Addr, UnorderedTermId)],
term_offsets: &[(Term<&[u8]>, Addr, UnorderedTermId)],
) -> Vec<(Field, Range<usize>)> {
let term_offsets_it = term_offsets
.iter()
.map(|(key, _, _)| Term::wrap(key).field())
.map(|(term, _, _)| term.field())
.enumerate();
let mut prev_field_opt = None;
let mut fields = vec![];
@@ -132,10 +130,10 @@ impl MultiFieldPostingsWriter {
fieldnorm_readers: FieldNormReaders,
doc_id_map: Option<&DocIdMapping>,
) -> crate::Result<HashMap<Field, FnvHashMap<UnorderedTermId, TermOrdinal>>> {
let mut term_offsets: Vec<(&[u8], Addr, UnorderedTermId)> =
let mut term_offsets: Vec<(Term<&[u8]>, Addr, UnorderedTermId)> =
Vec::with_capacity(self.term_index.len());
term_offsets.extend(self.term_index.iter());
term_offsets.sort_unstable_by_key(|&(k, _, _)| k);
term_offsets.sort_unstable_by_key(|(k, _, _)| k.clone());
let mut unordered_term_mappings: HashMap<Field, FnvHashMap<UnorderedTermId, TermOrdinal>> =
HashMap::new();
@@ -146,7 +144,7 @@ impl MultiFieldPostingsWriter {
let field_entry = self.schema.get_field_entry(field);
match *field_entry.field_type() {
FieldType::Str(_) | FieldType::HierarchicalFacet(_) => {
FieldType::Str(_) | FieldType::Facet(_) => {
// populating the (unordered term ord) -> (ordered term ord) mapping
// for the field.
let unordered_term_ids = term_offsets[byte_offsets.clone()]
@@ -210,7 +208,7 @@ pub trait PostingsWriter {
/// The actual serialization format is handled by the `PostingsSerializer`.
fn serialize(
&self,
term_addrs: &[(&[u8], Addr, UnorderedTermId)],
term_addrs: &[(Term<&[u8]>, Addr, UnorderedTermId)],
serializer: &mut FieldSerializer<'_>,
term_heap: &MemoryArena,
heap: &MemoryArena,
@@ -227,7 +225,7 @@ pub trait PostingsWriter {
heap: &mut MemoryArena,
term_buffer: &mut Term,
) -> u32 {
term_buffer.set_field(field);
term_buffer.set_field(Type::Str, field);
let mut sink = |token: &Token| {
// We skip all tokens with a len greater than u16.
if token.text.len() <= MAX_TOKEN_LEN {
@@ -281,7 +279,7 @@ impl<Rec: Recorder + 'static> PostingsWriter for SpecializedPostingsWriter<Rec>
) -> UnorderedTermId {
debug_assert!(term.as_slice().len() >= 4);
self.total_num_tokens += 1;
term_index.mutate_or_create(term, |opt_recorder: Option<Rec>| {
term_index.mutate_or_create(term.as_slice(), |opt_recorder: Option<Rec>| {
if let Some(mut recorder) = opt_recorder {
let current_doc = recorder.current_doc();
if current_doc != doc {
@@ -301,17 +299,17 @@ impl<Rec: Recorder + 'static> PostingsWriter for SpecializedPostingsWriter<Rec>
fn serialize(
&self,
term_addrs: &[(&[u8], Addr, UnorderedTermId)],
term_addrs: &[(Term<&[u8]>, Addr, UnorderedTermId)],
serializer: &mut FieldSerializer<'_>,
termdict_heap: &MemoryArena,
heap: &MemoryArena,
doc_id_map: Option<&DocIdMapping>,
) -> io::Result<()> {
let mut buffer_lender = BufferLender::default();
for &(term_bytes, addr, _) in term_addrs {
let recorder: Rec = termdict_heap.read(addr);
for (term, addr, _) in term_addrs {
let recorder: Rec = termdict_heap.read(*addr);
let term_doc_freq = recorder.term_doc_freq().unwrap_or(0u32);
serializer.new_term(&term_bytes[4..], term_doc_freq)?;
serializer.new_term(term.value_bytes(), term_doc_freq)?;
recorder.serialize(&mut buffer_lender, serializer, heap, doc_id_map);
serializer.close_term()?;
}

View File

@@ -3,6 +3,7 @@ use murmurhash32::murmurhash2;
use super::{Addr, MemoryArena};
use crate::postings::stacker::memory_arena::store;
use crate::postings::UnorderedTermId;
use crate::Term;
use byteorder::{ByteOrder, NativeEndian};
use std::iter;
use std::mem;
@@ -81,13 +82,13 @@ pub struct Iter<'a> {
}
impl<'a> Iterator for Iter<'a> {
type Item = (&'a [u8], Addr, UnorderedTermId);
type Item = (Term<&'a [u8]>, Addr, UnorderedTermId);
fn next(&mut self) -> Option<Self::Item> {
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);
(key, offset, kv.unordered_term_id)
(Term::wrap(key), offset, kv.unordered_term_id)
})
}
}
@@ -189,21 +190,19 @@ impl TermHashMap {
/// will be in charge of returning a default value.
/// If the key already as an associated value, then it will be passed
/// `Some(previous_value)`.
pub fn mutate_or_create<S, V, TMutator>(
pub fn mutate_or_create<V, TMutator>(
&mut self,
key: S,
key: &[u8],
mut updater: TMutator,
) -> UnorderedTermId
where
S: AsRef<[u8]>,
V: Copy + 'static,
TMutator: FnMut(Option<V>) -> V,
{
if self.is_saturated() {
self.resize();
}
let key_bytes: &[u8] = key.as_ref();
let hash = murmurhash2(key.as_ref());
let hash = murmurhash2(key);
let mut probe = self.probe(hash);
loop {
let bucket = probe.next_probe();
@@ -211,21 +210,18 @@ impl TermHashMap {
if kv.is_empty() {
// The key does not exists yet.
let val = updater(None);
let num_bytes =
std::mem::size_of::<u16>() + key_bytes.len() + std::mem::size_of::<V>();
let num_bytes = std::mem::size_of::<u16>() + key.len() + std::mem::size_of::<V>();
let key_addr = self.heap.allocate_space(num_bytes);
{
let data = self.heap.slice_mut(key_addr, num_bytes);
NativeEndian::write_u16(data, key_bytes.len() as u16);
let stop = 2 + key_bytes.len();
data[2..stop].copy_from_slice(key_bytes);
NativeEndian::write_u16(data, key.len() as u16);
let stop = 2 + key.len();
data[2..stop].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_bytes, kv.key_value_addr)
{
if let Some(val_addr) = self.get_value_addr_if_key_match(key, kv.key_value_addr) {
let v = self.heap.read(val_addr);
let new_v = updater(Some(v));
self.heap.write_at(val_addr, new_v);
@@ -245,25 +241,18 @@ mod tests {
#[test]
fn test_hash_map() {
let mut hash_map: TermHashMap = TermHashMap::new(18);
{
hash_map.mutate_or_create("abc", |opt_val: Option<u32>| {
assert_eq!(opt_val, None);
3u32
});
}
{
hash_map.mutate_or_create("abcd", |opt_val: Option<u32>| {
assert_eq!(opt_val, None);
4u32
});
}
{
hash_map.mutate_or_create("abc", |opt_val: Option<u32>| {
assert_eq!(opt_val, Some(3u32));
5u32
});
}
hash_map.mutate_or_create(b"abc", |opt_val: Option<u32>| {
assert_eq!(opt_val, None);
3u32
});
hash_map.mutate_or_create(b"abcd", |opt_val: Option<u32>| {
assert_eq!(opt_val, None);
4u32
});
hash_map.mutate_or_create(b"abc", |opt_val: Option<u32>| {
assert_eq!(opt_val, Some(3u32));
5u32
});
let mut vanilla_hash_map = HashMap::new();
let iter_values = hash_map.iter();
for (key, addr, _) in iter_values {

View File

@@ -132,10 +132,15 @@ impl FuzzyTermQuery {
match LEV_BUILDER.get(&(self.distance, self.transposition_cost_one)) {
// Unwrap the option and build the Ok(AutomatonWeight)
Some(automaton_builder) => {
let term_text = self.term.as_str().ok_or_else(|| {
crate::TantivyError::InvalidArgument(
"The fuzzy term query requires a string term.".to_string(),
)
})?;
let automaton = if self.prefix {
automaton_builder.build_prefix_dfa(self.term.text())
automaton_builder.build_prefix_dfa(term_text)
} else {
automaton_builder.build_dfa(self.term.text())
automaton_builder.build_dfa(term_text)
};
Ok(AutomatonWeight::new(
self.term.field(),

View File

@@ -180,7 +180,7 @@ impl MoreLikeThis {
// extract the raw value, possibly tokenizing & filtering to update the term frequency map
match field_entry.field_type() {
FieldType::HierarchicalFacet(_) => {
FieldType::Facet(_) => {
let facets: Vec<&str> = field_values
.iter()
.map(|field_value| match *field_value.value() {

View File

@@ -367,8 +367,8 @@ impl QueryParser {
))
}
}
FieldType::HierarchicalFacet(_) => match Facet::from_text(phrase) {
Ok(facet) => Ok(vec![(0, Term::from_field_text(field, facet.encoded_str()))]),
FieldType::Facet(_) => match Facet::from_text(phrase) {
Ok(facet) => Ok(vec![(0, Term::from_facet(field, &facet))]),
Err(e) => Err(QueryParserError::from(e)),
},
FieldType::Bytes(_) => {
@@ -665,7 +665,7 @@ mod test {
let query = query_parser.parse_query("facet:/root/branch/leaf").unwrap();
assert_eq!(
format!("{:?}", query),
"TermQuery(Term(field=11,bytes=[114, 111, 111, 116, 0, 98, 114, 97, 110, 99, 104, 0, 108, 101, 97, 102]))"
r#"TermQuery(Term(type=Facet, field=11, val="/root/branch/leaf"))"#
);
}
@@ -678,7 +678,7 @@ mod test {
let query = query_parser.parse_query("text:hello").unwrap();
assert_eq!(
format!("{:?}", query),
"Boost(query=TermQuery(Term(field=1,bytes=[104, 101, 108, 108, 111])), boost=2)"
r#"Boost(query=TermQuery(Term(type=Str, field=1, val="hello")), boost=2)"#
);
}
@@ -704,7 +704,7 @@ mod test {
let query = query_parser.parse_query("text:hello^2").unwrap();
assert_eq!(
format!("{:?}", query),
"Boost(query=Boost(query=TermQuery(Term(field=1,bytes=[104, 101, 108, 108, 111])), boost=2), boost=2)"
r#"Boost(query=Boost(query=TermQuery(Term(type=Str, field=1, val="hello")), boost=2), boost=2)"#
);
}
@@ -739,8 +739,7 @@ mod test {
pub fn test_parse_query_untokenized() {
test_parse_query_to_logical_ast_helper(
"nottokenized:\"wordone wordtwo\"",
"Term(field=7,bytes=[119, 111, 114, 100, 111, 110, \
101, 32, 119, 111, 114, 100, 116, 119, 111])",
r#"Term(type=Str, field=7, val="wordone wordtwo")"#,
false,
);
}
@@ -783,7 +782,7 @@ mod test {
.is_ok());
test_parse_query_to_logical_ast_helper(
"unsigned:2324",
"Term(field=3,bytes=[0, 0, 0, 0, 0, 0, 9, 20])",
"Term(type=U64, field=3, val=2324)",
false,
);
@@ -810,7 +809,7 @@ mod test {
fn test_parse_bytes() {
test_parse_query_to_logical_ast_helper(
"bytes:YnVidQ==",
"Term(field=12,bytes=[98, 117, 98, 117])",
"Term(type=Bytes, field=12, val=[98, 117, 98, 117])",
false,
);
}
@@ -825,7 +824,7 @@ mod test {
fn test_parse_bytes_phrase() {
test_parse_query_to_logical_ast_helper(
"bytes:\"YnVidQ==\"",
"Term(field=12,bytes=[98, 117, 98, 117])",
"Term(type=Bytes, field=12, val=[98, 117, 98, 117])",
false,
);
}
@@ -841,12 +840,12 @@ mod test {
fn test_parse_query_to_ast_ab_c() {
test_parse_query_to_logical_ast_helper(
"(+title:a +title:b) title:c",
"((+Term(field=0,bytes=[97]) +Term(field=0,bytes=[98])) Term(field=0,bytes=[99]))",
r#"((+Term(type=Str, field=0, val="a") +Term(type=Str, field=0, val="b")) Term(type=Str, field=0, val="c"))"#,
false,
);
test_parse_query_to_logical_ast_helper(
"(+title:a +title:b) title:c",
"(+(+Term(field=0,bytes=[97]) +Term(field=0,bytes=[98])) +Term(field=0,bytes=[99]))",
r#"(+(+Term(type=Str, field=0, val="a") +Term(type=Str, field=0, val="b")) +Term(type=Str, field=0, val="c"))"#,
true,
);
}
@@ -855,19 +854,17 @@ mod test {
pub fn test_parse_query_to_ast_single_term() {
test_parse_query_to_logical_ast_helper(
"title:toto",
"Term(field=0,bytes=[116, 111, 116, 111])",
r#"Term(type=Str, field=0, val="toto")"#,
false,
);
test_parse_query_to_logical_ast_helper(
"+title:toto",
"Term(field=0,bytes=[116, 111, 116, 111])",
r#"Term(type=Str, field=0, val="toto")"#,
false,
);
test_parse_query_to_logical_ast_helper(
"+title:toto -titi",
"(+Term(field=0,bytes=[116, 111, 116, 111]) \
-(Term(field=0,bytes=[116, 105, 116, 105]) \
Term(field=1,bytes=[116, 105, 116, 105])))",
r#"(+Term(type=Str, field=0, val="toto") -(Term(type=Str, field=0, val="titi") Term(type=Str, field=1, val="titi")))"#,
false,
);
}
@@ -884,13 +881,12 @@ mod test {
pub fn test_parse_query_to_ast_two_terms() {
test_parse_query_to_logical_ast_helper(
"title:a b",
"(Term(field=0,bytes=[97]) (Term(field=0,bytes=[98]) Term(field=1,bytes=[98])))",
r#"(Term(type=Str, field=0, val="a") (Term(type=Str, field=0, val="b") Term(type=Str, field=1, val="b")))"#,
false,
);
test_parse_query_to_logical_ast_helper(
"title:\"a b\"",
"\"[(0, Term(field=0,bytes=[97])), \
(1, Term(field=0,bytes=[98]))]\"",
r#"title:"a b""#,
r#""[(0, Term(type=Str, field=0, val="a")), (1, Term(type=Str, field=0, val="b"))]""#,
false,
);
}
@@ -899,46 +895,39 @@ mod test {
pub fn test_parse_query_to_ast_ranges() {
test_parse_query_to_logical_ast_helper(
"title:[a TO b]",
"(Included(Term(field=0,bytes=[97])) TO Included(Term(field=0,bytes=[98])))",
r#"(Included(Term(type=Str, field=0, val="a")) TO Included(Term(type=Str, field=0, val="b")))"#,
false,
);
test_parse_query_to_logical_ast_helper(
"[a TO b]",
"((Included(Term(field=0,bytes=[97])) TO \
Included(Term(field=0,bytes=[98]))) \
(Included(Term(field=1,bytes=[97])) TO \
Included(Term(field=1,bytes=[98]))))",
r#"((Included(Term(type=Str, field=0, val="a")) TO Included(Term(type=Str, field=0, val="b"))) (Included(Term(type=Str, field=1, val="a")) TO Included(Term(type=Str, field=1, val="b"))))"#,
false,
);
test_parse_query_to_logical_ast_helper(
"title:{titi TO toto}",
"(Excluded(Term(field=0,bytes=[116, 105, 116, 105])) TO \
Excluded(Term(field=0,bytes=[116, 111, 116, 111])))",
r#"(Excluded(Term(type=Str, field=0, val="titi")) TO Excluded(Term(type=Str, field=0, val="toto")))"#,
false,
);
test_parse_query_to_logical_ast_helper(
"title:{* TO toto}",
"(Unbounded TO Excluded(Term(field=0,bytes=[116, 111, 116, 111])))",
r#"(Unbounded TO Excluded(Term(type=Str, field=0, val="toto")))"#,
false,
);
test_parse_query_to_logical_ast_helper(
"title:{titi TO *}",
"(Excluded(Term(field=0,bytes=[116, 105, 116, 105])) TO Unbounded)",
r#"(Excluded(Term(type=Str, field=0, val="titi")) TO Unbounded)"#,
false,
);
test_parse_query_to_logical_ast_helper(
"signed:{-5 TO 3}",
"(Excluded(Term(field=2,bytes=[127, 255, 255, 255, 255, 255, 255, 251])) TO \
Excluded(Term(field=2,bytes=[128, 0, 0, 0, 0, 0, 0, 3])))",
r#"(Excluded(Term(type=I64, field=2, val=-5)) TO Excluded(Term(type=I64, field=2, val=3)))"#,
false,
);
test_parse_query_to_logical_ast_helper(
"float:{-1.5 TO 1.5}",
"(Excluded(Term(field=10,bytes=[64, 7, 255, 255, 255, 255, 255, 255])) TO \
Excluded(Term(field=10,bytes=[191, 248, 0, 0, 0, 0, 0, 0])))",
r#"(Excluded(Term(type=F64, field=10, val=-1.5)) TO Excluded(Term(type=F64, field=10, val=1.5)))"#,
false,
);
test_parse_query_to_logical_ast_helper("*", "*", false);
}
@@ -1065,32 +1054,27 @@ mod test {
pub fn test_parse_query_to_ast_conjunction() {
test_parse_query_to_logical_ast_helper(
"title:toto",
"Term(field=0,bytes=[116, 111, 116, 111])",
r#"Term(type=Str, field=0, val="toto")"#,
true,
);
test_parse_query_to_logical_ast_helper(
"+title:toto",
"Term(field=0,bytes=[116, 111, 116, 111])",
r#"Term(type=Str, field=0, val="toto")"#,
true,
);
test_parse_query_to_logical_ast_helper(
"+title:toto -titi",
"(+Term(field=0,bytes=[116, 111, 116, 111]) \
-(Term(field=0,bytes=[116, 105, 116, 105]) \
Term(field=1,bytes=[116, 105, 116, 105])))",
r#"(+Term(type=Str, field=0, val="toto") -(Term(type=Str, field=0, val="titi") Term(type=Str, field=1, val="titi")))"#,
true,
);
test_parse_query_to_logical_ast_helper(
"title:a b",
"(+Term(field=0,bytes=[97]) \
+(Term(field=0,bytes=[98]) \
Term(field=1,bytes=[98])))",
r#"(+Term(type=Str, field=0, val="a") +(Term(type=Str, field=0, val="b") Term(type=Str, field=1, val="b")))"#,
true,
);
test_parse_query_to_logical_ast_helper(
"title:\"a b\"",
"\"[(0, Term(field=0,bytes=[97])), \
(1, Term(field=0,bytes=[98]))]\"",
r#""[(0, Term(type=Str, field=0, val="a")), (1, Term(type=Str, field=0, val="b"))]""#,
true,
);
}
@@ -1099,8 +1083,8 @@ mod test {
pub fn test_query_parser_hyphen() {
test_parse_query_to_logical_ast_helper(
"title:www-form-encoded",
"\"[(0, Term(field=0,bytes=[119, 119, 119])), (1, Term(field=0,bytes=[102, 111, 114, 109])), (2, Term(field=0,bytes=[101, 110, 99, 111, 100, 101, 100]))]\"",
false
r#""[(0, Term(type=Str, field=0, val="www")), (1, Term(type=Str, field=0, val="form")), (2, Term(type=Str, field=0, val="encoded"))]""#,
false,
);
}
@@ -1109,7 +1093,7 @@ mod test {
for &default_conjunction in &[false, true] {
test_parse_query_to_logical_ast_helper(
"title:a AND title:b",
"(+Term(field=0,bytes=[97]) +Term(field=0,bytes=[98]))",
r#"(+Term(type=Str, field=0, val="a") +Term(type=Str, field=0, val="b"))"#,
default_conjunction,
);
}
@@ -1120,7 +1104,7 @@ mod test {
for &default_conjunction in &[false, true] {
test_parse_query_to_logical_ast_helper(
"title:a OR title:b",
"(Term(field=0,bytes=[97]) Term(field=0,bytes=[98]))",
r#"(Term(type=Str, field=0, val="a") Term(type=Str, field=0, val="b"))"#,
default_conjunction,
);
}

View File

@@ -175,7 +175,7 @@ mod tests {
);
assert_eq!(
format!("{:?}", term_query),
"TermQuery(Term(field=1,bytes=[104, 101, 108, 108, 111]))"
r#"TermQuery(Term(type=Str, field=1, val="hello"))"#
);
}

View File

@@ -31,11 +31,7 @@ impl Weight for TermWeight {
return Err(does_not_match(doc));
}
let mut explanation = scorer.explain();
explanation.add_context(format!(
"Term ={:?}:{:?}",
self.term.field(),
self.term.value_bytes()
));
explanation.add_context(format!("Term={:?}", self.term,));
Ok(explanation)
}

View File

@@ -85,7 +85,7 @@ impl FieldEntry {
assert!(is_valid_field_name(&field_name));
FieldEntry {
name: field_name,
field_type: FieldType::HierarchicalFacet(field_type),
field_type: FieldType::Facet(field_type),
}
}
@@ -138,7 +138,7 @@ impl FieldEntry {
| FieldType::F64(ref options)
| FieldType::Date(ref options) => options.is_stored(),
FieldType::Str(ref options) => options.is_stored(),
FieldType::HierarchicalFacet(ref options) => options.is_stored(),
FieldType::Facet(ref options) => options.is_stored(),
FieldType::Bytes(ref options) => options.is_stored(),
}
}

View File

@@ -30,21 +30,60 @@ pub enum ValueParsingError {
/// Contrary to FieldType, this does
/// not include the way the field must be indexed.
#[derive(Copy, Clone, Debug, Eq, PartialEq)]
#[repr(u8)]
pub enum Type {
/// `&str`
Str,
Str = b's',
/// `u64`
U64,
U64 = b'u',
/// `i64`
I64,
I64 = b'i',
/// `f64`
F64,
F64 = b'f',
/// `date(i64) timestamp`
Date,
Date = b'd',
/// `tantivy::schema::Facet`. Passed as a string in JSON.
HierarchicalFacet,
Facet = b'h',
/// `Vec<u8>`
Bytes,
Bytes = b'b',
}
const ALL_TYPES: [Type; 7] = [
Type::Str,
Type::U64,
Type::I64,
Type::F64,
Type::Date,
Type::Facet,
Type::Bytes,
];
impl Type {
/// Returns an iterator over the different values
/// the Type enum can tape.
pub fn iter_values() -> impl Iterator<Item = Type> {
ALL_TYPES.iter().cloned()
}
/// Returns a 1 byte code used to identify the type.
pub fn to_code(&self) -> u8 {
*self as u8
}
/// Interprets a 1byte code as a type.
/// Returns None if the code is invalid.
pub fn from_code(code: u8) -> Option<Self> {
match code {
b's' => Some(Type::Str),
b'u' => Some(Type::U64),
b'i' => Some(Type::I64),
b'f' => Some(Type::F64),
b'd' => Some(Type::Date),
b'h' => Some(Type::Facet),
b'b' => Some(Type::Bytes),
_ => None,
}
}
}
/// A `FieldType` describes the type (text, u64) of a field as well as
@@ -65,7 +104,7 @@ pub enum FieldType {
/// Signed 64-bits Date 64 field type configuration,
Date(IntOptions),
/// Hierachical Facet
HierarchicalFacet(FacetOptions),
Facet(FacetOptions),
/// Bytes (one per document)
Bytes(BytesOptions),
}
@@ -79,7 +118,7 @@ impl FieldType {
FieldType::I64(_) => Type::I64,
FieldType::F64(_) => Type::F64,
FieldType::Date(_) => Type::Date,
FieldType::HierarchicalFacet(_) => Type::HierarchicalFacet,
FieldType::Facet(_) => Type::Facet,
FieldType::Bytes(_) => Type::Bytes,
}
}
@@ -92,7 +131,7 @@ impl FieldType {
| FieldType::I64(ref int_options)
| FieldType::F64(ref int_options) => int_options.is_indexed(),
FieldType::Date(ref date_options) => date_options.is_indexed(),
FieldType::HierarchicalFacet(ref _facet_options) => true,
FieldType::Facet(ref _facet_options) => true,
FieldType::Bytes(ref bytes_options) => bytes_options.is_indexed(),
}
}
@@ -108,7 +147,7 @@ impl FieldType {
| FieldType::I64(ref int_options)
| FieldType::F64(ref int_options)
| FieldType::Date(ref int_options) => int_options.fieldnorms(),
FieldType::HierarchicalFacet(_) => false,
FieldType::Facet(_) => false,
FieldType::Bytes(ref bytes_options) => bytes_options.fieldnorms(),
}
}
@@ -132,7 +171,7 @@ impl FieldType {
None
}
}
FieldType::HierarchicalFacet(ref _facet_options) => Some(IndexRecordOption::Basic),
FieldType::Facet(ref _facet_options) => Some(IndexRecordOption::Basic),
FieldType::Bytes(ref bytes_options) => {
if bytes_options.is_indexed() {
Some(IndexRecordOption::Basic)
@@ -165,7 +204,7 @@ impl FieldType {
FieldType::U64(_) | FieldType::I64(_) | FieldType::F64(_) => Err(
ValueParsingError::TypeError(format!("Expected an integer, got {:?}", json)),
),
FieldType::HierarchicalFacet(_) => Ok(Value::Facet(Facet::from(field_text))),
FieldType::Facet(_) => Ok(Value::Facet(Facet::from(field_text))),
FieldType::Bytes(_) => base64::decode(field_text).map(Value::Bytes).map_err(|_| {
ValueParsingError::InvalidBase64(format!(
"Expected base64 string, got {:?}",
@@ -198,7 +237,7 @@ impl FieldType {
Err(ValueParsingError::OverflowError(msg))
}
}
FieldType::Str(_) | FieldType::HierarchicalFacet(_) | FieldType::Bytes(_) => {
FieldType::Str(_) | FieldType::Facet(_) | FieldType::Bytes(_) => {
let msg = format!("Expected a string, got {:?}", json);
Err(ValueParsingError::TypeError(msg))
}
@@ -241,6 +280,7 @@ mod tests {
use super::FieldType;
use crate::schema::field_type::ValueParsingError;
use crate::schema::TextOptions;
use crate::schema::Type;
use crate::schema::Value;
use crate::schema::{Schema, INDEXED};
use crate::tokenizer::{PreTokenizedString, Token};
@@ -358,4 +398,13 @@ mod tests {
assert_eq!(serialized_value_json, pre_tokenized_string_json);
}
#[test]
fn test_type_codes() {
for type_val in Type::iter_values() {
let code = type_val.to_code();
assert_eq!(Type::from_code(code), Some(type_val));
}
assert_eq!(Type::from_code(b'z'), None);
}
}

View File

@@ -1,18 +1,20 @@
use std::fmt;
use std::hash::{Hash, Hasher};
use super::Field;
use crate::schema::Facet;
use crate::fastfield::FastValue;
use crate::schema::{Facet, Type};
use crate::DateTime;
use common;
use std::str;
/// Size (in bytes) of the buffer of a int field.
const INT_TERM_LEN: usize = 4 + 8;
/// Size (in bytes) of the buffer of a fast value (u64, i64, f64, or date) term.
/// <field> + <type byte> + <value len>
const FAST_VALUE_TERM_LEN: usize = 4 + 1 + 8;
/// Term represents the value that the token can take.
///
/// It actually wraps a `Vec<u8>`.
#[derive(Clone, PartialEq, PartialOrd, Ord, Eq, Hash)]
#[derive(Clone)]
pub struct Term<B = Vec<u8>>(B)
where
B: AsRef<[u8]>;
@@ -22,98 +24,61 @@ impl Term {
Term(Vec::with_capacity(100))
}
/// Builds a term given a field, and a i64-value
///
/// Assuming the term has a field id of 1, and a i64 value of 3234,
/// the Term will have 12 bytes.
///
/// The first four byte are dedicated to storing the field id as a u64.
/// The 8 following bytes are encoding the u64 value.
pub fn from_field_i64(field: Field, val: i64) -> Term {
let val_u64: u64 = common::i64_to_u64(val);
Term::from_field_u64(field, val_u64)
}
/// Builds a term given a field, and a f64-value
///
/// Assuming the term has a field id of 1, and a f64 value of 1.5,
/// the Term will have 12 bytes.
///
/// The first four byte are dedicated to storing the field id as a u64.
/// The 8 following bytes are encoding the f64 as a u64 value.
pub fn from_field_f64(field: Field, val: f64) -> Term {
let val_u64: u64 = common::f64_to_u64(val);
Term::from_field_u64(field, val_u64)
}
/// Builds a term given a field, and a DateTime value
///
/// Assuming the term has a field id of 1, and a timestamp i64 value of 3234,
/// the Term will have 12 bytes.
///
/// The first four byte are dedicated to storing the field id as a u64.
/// The 8 following bytes are encoding the DateTime as i64 timestamp value.
pub fn from_field_date(field: Field, val: &DateTime) -> Term {
let val_timestamp = val.timestamp();
Term::from_field_i64(field, val_timestamp)
}
/// Creates a `Term` given a facet.
pub fn from_facet(field: Field, facet: &Facet) -> Term {
let bytes = facet.encoded_str().as_bytes();
let buffer = Vec::with_capacity(4 + bytes.len());
let mut term = Term(buffer);
term.set_field(field);
term.set_bytes(bytes);
term
}
/// Builds a term given a field, and a string value
///
/// Assuming the term has a field id of 2, and a text value of "abc",
/// the Term will have 4 bytes.
/// The first byte is 2, and the three following bytes are the utf-8
/// representation of "abc".
pub fn from_field_text(field: Field, text: &str) -> Term {
let buffer = Vec::with_capacity(4 + text.len());
let mut term = Term(buffer);
term.set_field(field);
term.set_text(text);
fn from_fast_value<T: FastValue>(field: Field, val: &T) -> Term {
let mut term = Term(vec![0u8; FAST_VALUE_TERM_LEN]);
term.set_field(T::to_type(), field);
term.set_u64(val.to_u64());
term
}
/// Builds a term given a field, and a u64-value
///
/// Assuming the term has a field id of 1, and a u64 value of 3234,
/// the Term will have 12 bytes.
///
/// The first four byte are dedicated to storing the field id as a u64.
/// The 8 following bytes are encoding the u64 value.
pub fn from_field_u64(field: Field, val: u64) -> Term {
let mut term = Term(vec![0u8; INT_TERM_LEN]);
term.set_field(field);
term.set_u64(val);
Term::from_fast_value(field, &val)
}
/// Builds a term given a field, and a i64-value
pub fn from_field_i64(field: Field, val: i64) -> Term {
Term::from_fast_value(field, &val)
}
/// Builds a term given a field, and a f64-value
pub fn from_field_f64(field: Field, val: f64) -> Term {
Term::from_fast_value(field, &val)
}
/// Builds a term given a field, and a DateTime value
pub fn from_field_date(field: Field, val: &DateTime) -> Term {
Term::from_fast_value(field, val)
}
/// Creates a `Term` given a facet.
pub fn from_facet(field: Field, facet: &Facet) -> Term {
let facet_encoded_str = facet.encoded_str();
Term::create_bytes_term(Type::Facet, field, facet_encoded_str.as_bytes())
}
/// Builds a term given a field, and a string value
pub fn from_field_text(field: Field, text: &str) -> Term {
Term::create_bytes_term(Type::Str, field, text.as_bytes())
}
fn create_bytes_term(typ: Type, field: Field, bytes: &[u8]) -> Term {
let mut term = Term(vec![0u8; 5 + bytes.len()]);
term.set_field(typ, field);
term.0.extend_from_slice(bytes);
term
}
/// Builds a term bytes.
pub fn from_field_bytes(field: Field, bytes: &[u8]) -> Term {
let mut term = Term::for_field(field);
term.set_bytes(bytes);
term
Term::create_bytes_term(Type::Bytes, field, bytes)
}
/// Creates a new Term for a given field.
pub(crate) fn for_field(field: Field) -> Term {
let mut term = Term(Vec::with_capacity(100));
term.set_field(field);
term
}
pub(crate) fn set_field(&mut self, field: Field) {
pub(crate) fn set_field(&mut self, typ: Type, field: Field) {
self.0.clear();
self.0
.extend_from_slice(field.field_id().to_be_bytes().as_ref());
self.0.push(typ.to_code());
}
/// Sets a u64 value in the term.
@@ -123,23 +88,33 @@ impl Term {
/// The use of BigEndian has the benefit of preserving
/// the natural order of the values.
pub fn set_u64(&mut self, val: u64) {
self.0.resize(INT_TERM_LEN, 0u8);
self.set_fast_value(val);
self.set_bytes(val.to_be_bytes().as_ref());
}
fn set_fast_value<T: FastValue>(&mut self, val: T) {
self.0.resize(FAST_VALUE_TERM_LEN, 0u8);
self.set_bytes(val.to_u64().to_be_bytes().as_ref());
}
/// Sets a `i64` value in the term.
pub fn set_i64(&mut self, val: i64) {
self.set_u64(common::i64_to_u64(val));
self.set_fast_value(val);
}
/// Sets a `i64` value in the term.
pub fn set_date(&mut self, date: crate::DateTime) {
self.set_fast_value(date);
}
/// Sets a `f64` value in the term.
pub fn set_f64(&mut self, val: f64) {
self.set_u64(common::f64_to_u64(val));
self.set_fast_value(val);
}
/// Sets the value of a `Bytes` field.
pub fn set_bytes(&mut self, bytes: &[u8]) {
self.0.resize(4, 0u8);
self.0.resize(5, 0u8);
self.0.extend(bytes);
}
@@ -149,6 +124,44 @@ impl Term {
}
}
impl<B> Ord for Term<B>
where
B: AsRef<[u8]>,
{
fn cmp(&self, other: &Self) -> std::cmp::Ordering {
self.as_slice().cmp(other.as_slice())
}
}
impl<B> PartialOrd for Term<B>
where
B: AsRef<[u8]>,
{
fn partial_cmp(&self, other: &Self) -> Option<std::cmp::Ordering> {
Some(self.cmp(other))
}
}
impl<B> PartialEq for Term<B>
where
B: AsRef<[u8]>,
{
fn eq(&self, other: &Self) -> bool {
self.as_slice() == other.as_slice()
}
}
impl<B> Eq for Term<B> where B: AsRef<[u8]> {}
impl<B> Hash for Term<B>
where
B: AsRef<[u8]>,
{
fn hash<H: Hasher>(&self, state: &mut H) {
self.0.as_ref().hash(state)
}
}
impl<B> Term<B>
where
B: AsRef<[u8]>,
@@ -158,6 +171,15 @@ where
Term(data)
}
/// Return the type of the term.
pub fn typ(&self) -> Type {
assert!(
self.as_slice().len() >= 5,
"the type does byte representation is too short"
);
Type::from_code(self.as_slice()[4]).expect("The term has an invalid type code")
}
/// Returns the field.
pub fn field(&self) -> Field {
let mut field_id_bytes = [0u8; 4];
@@ -167,41 +189,86 @@ where
/// Returns the `u64` value stored in a term.
///
/// # Panics
/// ... or returns an invalid value
/// if the term is not a `u64` field.
pub fn get_u64(&self) -> u64 {
let mut field_id_bytes = [0u8; 8];
field_id_bytes.copy_from_slice(self.value_bytes());
u64::from_be_bytes(field_id_bytes)
/// Returns None if the term is not of the u64 type, or if the term byte representation
/// is invalid.
pub fn as_u64(&self) -> Option<u64> {
self.get_fast_type::<u64>()
}
fn get_fast_type<T: FastValue>(&self) -> Option<T> {
if self.typ() != T::to_type() || self.as_slice().len() != FAST_VALUE_TERM_LEN {
return None;
}
let mut value_bytes = [0u8; 8];
value_bytes.copy_from_slice(self.value_bytes());
let value_u64 = u64::from_be_bytes(value_bytes);
Some(FastValue::from_u64(value_u64))
}
/// Returns the `i64` value stored in a term.
///
/// # Panics
/// ... or returns an invalid value
/// if the term is not a `i64` field.
pub fn get_i64(&self) -> i64 {
common::u64_to_i64(self.get_u64())
/// Returns None if the term is not of the i64 type, or if the term byte representation
/// is invalid.
pub fn as_i64(&self) -> Option<i64> {
self.get_fast_type::<i64>()
}
/// Returns the `f64` value stored in a term.
///
/// # Panics
/// ... or returns an invalid value
/// if the term is not a `f64` field.
pub fn get_f64(&self) -> f64 {
common::u64_to_f64(self.get_u64())
/// Returns None if the term is not of the f64 type, or if the term byte representation
/// is invalid.
pub fn as_f64(&self) -> Option<f64> {
self.get_fast_type::<f64>()
}
/// Returns the `Date` value stored in a term.
///
/// Returns None if the term is not of the Date type, or if the term byte representation
/// is invalid.
pub fn as_date(&self) -> Option<crate::DateTime> {
self.get_fast_type::<crate::DateTime>()
}
/// Returns the text associated with the term.
///
/// # Panics
/// If the value is not valid utf-8. This may happen
/// if the index is corrupted or if you try to
/// call this method on a non-string type.
pub fn text(&self) -> &str {
str::from_utf8(self.value_bytes()).expect("Term does not contain valid utf-8.")
/// 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;
}
str::from_utf8(self.value_bytes()).ok()
}
/// Returns the facet associated with the term.
///
/// Returns None if the field is not of facet type
/// or if the bytes are not valid utf-8.
pub fn as_facet(&self) -> Option<Facet> {
if self.as_slice().len() < 5 {
return None;
}
if self.typ() != Type::Facet {
return None;
}
let facet_encode_str = str::from_utf8(self.value_bytes()).ok()?;
Some(Facet::from_encoded_string(facet_encode_str.to_string()))
}
/// Returns the bytes associated with the term.
///
/// Returns None if the field is not of bytes type.
pub fn as_bytes(&self) -> Option<&[u8]> {
if self.as_slice().len() < 5 {
return None;
}
if self.typ() != Type::Bytes {
return None;
}
Some(self.value_bytes())
}
/// Returns the serialized value of the term.
@@ -211,32 +278,61 @@ where
/// If the term is a u64, its value is encoded according
/// to `byteorder::LittleEndian`.
pub fn value_bytes(&self) -> &[u8] {
&self.0.as_ref()[4..]
&self.0.as_ref()[5..]
}
/// Returns the underlying `&[u8]`
pub fn as_slice(&self) -> &[u8] {
/// 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.0.as_ref()
}
}
impl<B> AsRef<[u8]> for Term<B>
where
B: AsRef<[u8]>,
{
fn as_ref(&self) -> &[u8] {
self.0.as_ref()
fn write_opt<T: std::fmt::Debug>(f: &mut fmt::Formatter, val_opt: Option<T>) -> fmt::Result {
if let Some(val) = val_opt {
write!(f, "{:?}", val)?;
}
Ok(())
}
impl fmt::Debug for Term {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
write!(
f,
"Term(field={},bytes={:?})",
self.field().field_id(),
self.value_bytes()
)
let field_id = self.field().field_id();
let typ = self.typ();
write!(f, "Term(type={:?}, field={}, val=", typ, field_id,)?;
match typ {
Type::Str => {
let s = str::from_utf8(self.value_bytes()).ok();
write_opt(f, s)?;
}
Type::U64 => {
write_opt(f, self.as_u64())?;
}
Type::I64 => {
let val_i64 = self.as_i64();
write_opt(f, val_i64)?;
}
Type::F64 => {
let val_f64 = self.as_f64();
write_opt(f, val_f64)?;
}
// TODO pretty print these types too.
Type::Date => {
let val_date = self.as_date();
write_opt(f, val_date)?;
}
Type::Facet => {
let facet = self.as_facet().map(|facet| facet.to_path_string());
write_opt(f, facet)?;
}
Type::Bytes => {
write_opt(f, self.as_bytes())?;
}
}
write!(f, ")",)?;
Ok(())
}
}
@@ -246,30 +342,24 @@ mod tests {
use crate::schema::*;
#[test]
pub fn test_term() {
pub fn test_term_str() {
let mut schema_builder = Schema::builder();
schema_builder.add_text_field("text", STRING);
let title_field = schema_builder.add_text_field("title", STRING);
let count_field = schema_builder.add_text_field("count", STRING);
{
let term = Term::from_field_text(title_field, "test");
assert_eq!(term.field(), title_field);
assert_eq!(&term.as_slice()[0..4], &[0u8, 0u8, 0u8, 1u8]);
assert_eq!(&term.as_slice()[4..], "test".as_bytes());
}
{
let term = Term::from_field_u64(count_field, 983u64);
assert_eq!(term.field(), count_field);
assert_eq!(&term.as_slice()[0..4], &[0u8, 0u8, 0u8, 2u8]);
assert_eq!(term.as_slice().len(), 4 + 8);
assert_eq!(term.as_slice()[4], 0u8);
assert_eq!(term.as_slice()[5], 0u8);
assert_eq!(term.as_slice()[6], 0u8);
assert_eq!(term.as_slice()[7], 0u8);
assert_eq!(term.as_slice()[8], 0u8);
assert_eq!(term.as_slice()[9], 0u8);
assert_eq!(term.as_slice()[10], (933u64 / 256u64) as u8);
assert_eq!(term.as_slice()[11], (983u64 % 256u64) as u8);
}
let term = Term::from_field_text(title_field, "test");
assert_eq!(term.field(), title_field);
assert_eq!(term.typ(), Type::Str);
assert_eq!(term.as_str(), Some("test"))
}
#[test]
pub fn test_term_u64() {
let mut schema_builder = Schema::builder();
let count_field = schema_builder.add_u64_field("count", INDEXED);
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.as_u64(), Some(983u64))
}
}

View File

@@ -21,7 +21,7 @@ pub enum Value {
F64(f64),
/// Signed 64-bits Date time stamp `date`
Date(DateTime),
/// Hierarchical Facet
/// Facet
Facet(Facet),
/// Arbitrarily sized byte array
Bytes(Vec<u8>),

View File

@@ -243,10 +243,15 @@ impl SnippetGenerator {
if term.field() != field {
continue;
}
let term_str = if let Some(term_str) = term.as_str() {
term_str
} else {
continue;
};
let doc_freq = searcher.doc_freq(&term)?;
if doc_freq > 0 {
let score = 1.0 / (1.0 + doc_freq as Score);
terms_text.insert(term.text().to_string(), score);
terms_text.insert(term_str.to_string(), score);
}
}
let tokenizer = searcher.index().tokenizer_for_field(field)?;

View File

@@ -157,8 +157,8 @@ pub use self::tokenizer_manager::TokenizerManager;
///
/// Tokenizer are in charge of not emitting tokens larger than this value.
/// Currently, if a faulty tokenizer implementation emits tokens with a length larger than
/// `2^16 - 1 - 4`, the token will simply be ignored downstream.
pub const MAX_TOKEN_LEN: usize = u16::max_value() as usize - 4;
/// `2^16 - 1 - 5`, the token will simply be ignored downstream.
pub const MAX_TOKEN_LEN: usize = u16::max_value() as usize - 5;
#[cfg(test)]
pub mod tests {