Issue/922b (#1233)

* Add a NORMED options on field

Make fieldnorm indexation optional:

* for all types except text => added a NORMED options
* for text field
** if STRING, field has not fieldnorm retained
** if TEXT, field has fieldnorm computed

* Finalize making fieldnorm optional for all field types.

- Using Option for fieldnorm readers.
This commit is contained in:
Paul Masurel
2021-12-10 21:12:29 +09:00
committed by GitHub
parent 9e66c75fc6
commit c81b3030fa
20 changed files with 671 additions and 112 deletions

View File

@@ -4,6 +4,7 @@ Tantivy 0.17
- Facets are necessarily indexed. Existing index with indexed facets should work out of the box. Index without facets that are marked with index: false should be broken (but they were already broken in a sense). (@fulmicoton) #1195 .
- Bugfix that could in theory impact durability in theory on some filesystems [#1224](https://github.com/quickwit-inc/tantivy/issues/1224)
- Reduce the number of fsync calls [#1225](https://github.com/quickwit-inc/tantivy/issues/1225)
- Schema now offers not indexing fieldnorms (@lpouget) [#922](https://github.com/quickwit-inc/tantivy/issues/922)
Tantivy 0.16.2
================================

View File

@@ -394,7 +394,7 @@ mod tests {
let json = serde_json::ser::to_string(&index_metas).expect("serialization failed");
assert_eq!(
json,
r#"{"index_settings":{"sort_by_field":{"field":"text","order":"Asc"},"docstore_compression":"lz4"},"segments":[],"schema":[{"name":"text","type":"text","options":{"indexing":{"record":"position","tokenizer":"default"},"stored":false}}],"opstamp":0}"#
r#"{"index_settings":{"sort_by_field":{"field":"text","order":"Asc"},"docstore_compression":"lz4"},"segments":[],"schema":[{"name":"text","type":"text","options":{"indexing":{"record":"position","fieldnorms":true,"tokenizer":"default"},"stored":false}}],"opstamp":0}"#
);
}
}

View File

@@ -127,13 +127,17 @@ impl SegmentReader {
self.fieldnorm_readers.get_field(field)?.ok_or_else(|| {
let field_name = self.schema.get_field_name(field);
let err_msg = format!(
"Field norm not found for field {:?}. Was it marked as indexed during indexing?",
"Field norm not found for field {:?}. Was the field set to record norm during indexing?",
field_name
);
crate::TantivyError::SchemaError(err_msg)
})
}
pub(crate) fn fieldnorms_readers(&self) -> &FieldNormReaders {
&self.fieldnorm_readers
}
/// Accessor to the segment's `StoreReader`.
pub fn get_store_reader(&self) -> io::Result<StoreReader> {
StoreReader::open(self.store_file.clone())

View File

@@ -68,6 +68,7 @@ mod tests {
IntOptions::default()
.set_fast(Cardinality::MultiValues)
.set_indexed()
.set_fieldnorm()
.set_stored(),
);
let time_i =

View File

@@ -26,3 +26,137 @@ pub use self::serializer::FieldNormsSerializer;
pub use self::writer::FieldNormsWriter;
use self::code::{fieldnorm_to_id, id_to_fieldnorm};
#[cfg(test)]
mod tests {
use crate::directory::CompositeFile;
use crate::directory::{Directory, RamDirectory, WritePtr};
use crate::fieldnorm::FieldNormReader;
use crate::fieldnorm::FieldNormsSerializer;
use crate::fieldnorm::FieldNormsWriter;
use crate::query::Query;
use crate::query::TermQuery;
use crate::schema::IndexRecordOption;
use crate::schema::TextFieldIndexing;
use crate::schema::TextOptions;
use crate::schema::TEXT;
use crate::Index;
use crate::Term;
use crate::TERMINATED;
use once_cell::sync::Lazy;
use std::path::Path;
use crate::schema::{Field, Schema, STORED};
pub static SCHEMA: Lazy<Schema> = Lazy::new(|| {
let mut schema_builder = Schema::builder();
schema_builder.add_text_field("field", STORED);
schema_builder.add_text_field("txt_field", TEXT);
schema_builder.add_text_field(
"str_field",
TextOptions::default().set_indexing_options(
TextFieldIndexing::default()
.set_index_option(IndexRecordOption::Basic)
.set_fieldnorms(false),
),
);
schema_builder.build()
});
pub static FIELD: Lazy<Field> = Lazy::new(|| SCHEMA.get_field("field").unwrap());
pub static TXT_FIELD: Lazy<Field> = Lazy::new(|| SCHEMA.get_field("txt_field").unwrap());
pub static STR_FIELD: Lazy<Field> = Lazy::new(|| SCHEMA.get_field("str_field").unwrap());
#[test]
#[should_panic(expected = "Cannot register a given fieldnorm twice")]
pub fn test_should_panic_when_recording_fieldnorm_twice_for_same_doc() {
let mut fieldnorm_writers = FieldNormsWriter::for_schema(&SCHEMA);
fieldnorm_writers.record(0u32, *TXT_FIELD, 5);
fieldnorm_writers.record(0u32, *TXT_FIELD, 3);
}
#[test]
pub fn test_fieldnorm() -> crate::Result<()> {
let path = Path::new("test");
let directory: RamDirectory = RamDirectory::create();
{
let write: WritePtr = directory.open_write(Path::new("test"))?;
let serializer = FieldNormsSerializer::from_write(write)?;
let mut fieldnorm_writers = FieldNormsWriter::for_schema(&SCHEMA);
fieldnorm_writers.record(2u32, *TXT_FIELD, 5);
fieldnorm_writers.record(3u32, *TXT_FIELD, 3);
fieldnorm_writers.serialize(serializer, None)?;
}
let file = directory.open_read(&path)?;
{
let fields_composite = CompositeFile::open(&file)?;
assert!(fields_composite.open_read(*FIELD).is_none());
assert!(fields_composite.open_read(*STR_FIELD).is_none());
let data = fields_composite.open_read(*TXT_FIELD).unwrap();
let fieldnorm_reader = FieldNormReader::open(data)?;
assert_eq!(fieldnorm_reader.fieldnorm(0u32), 0u32);
assert_eq!(fieldnorm_reader.fieldnorm(1u32), 0u32);
assert_eq!(fieldnorm_reader.fieldnorm(2u32), 5u32);
assert_eq!(fieldnorm_reader.fieldnorm(3u32), 3u32);
}
Ok(())
}
#[test]
fn test_fieldnorm_disabled() -> crate::Result<()> {
let mut schema_builder = Schema::builder();
let text_options = TextOptions::default()
.set_indexing_options(TextFieldIndexing::default().set_fieldnorms(false));
let text = schema_builder.add_text_field("text", text_options);
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
let mut writer = index.writer_for_tests()?;
writer.add_document(doc!(text=>"hello"))?;
writer.add_document(doc!(text=>"hello hello hello"))?;
writer.commit()?;
let reader = index.reader()?;
let searcher = reader.searcher();
let query = TermQuery::new(
Term::from_field_text(text, "hello"),
IndexRecordOption::WithFreqs,
);
let weight = query.weight(&*searcher, true)?;
let mut scorer = weight.scorer(searcher.segment_reader(0), 1.0f32)?;
assert_eq!(scorer.doc(), 0);
assert!((scorer.score() - 0.22920431).abs() < 0.001f32);
assert_eq!(scorer.advance(), 1);
assert_eq!(scorer.doc(), 1);
assert!((scorer.score() - 0.22920431).abs() < 0.001f32);
assert_eq!(scorer.advance(), TERMINATED);
Ok(())
}
#[test]
fn test_fieldnorm_enabled() -> crate::Result<()> {
let mut schema_builder = Schema::builder();
let text_options = TextOptions::default()
.set_indexing_options(TextFieldIndexing::default().set_fieldnorms(true));
let text = schema_builder.add_text_field("text", text_options);
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
let mut writer = index.writer_for_tests()?;
writer.add_document(doc!(text=>"hello"))?;
writer.add_document(doc!(text=>"hello hello hello"))?;
writer.commit()?;
let reader = index.reader()?;
let searcher = reader.searcher();
let query = TermQuery::new(
Term::from_field_text(text, "hello"),
IndexRecordOption::WithFreqs,
);
let weight = query.weight(&*searcher, true)?;
let mut scorer = weight.scorer(searcher.segment_reader(0), 1.0f32)?;
assert_eq!(scorer.doc(), 0);
assert!((scorer.score() - 0.22920431).abs() < 0.001f32);
assert_eq!(scorer.advance(), 1);
assert_eq!(scorer.doc(), 1);
assert!((scorer.score() - 0.15136132).abs() < 0.001f32);
assert_eq!(scorer.advance(), TERMINATED);
Ok(())
}
}

View File

@@ -4,6 +4,7 @@ use super::fieldnorm_to_id;
use super::FieldNormsSerializer;
use crate::schema::Field;
use crate::schema::Schema;
use std::cmp::Ordering;
use std::{io, iter};
/// The `FieldNormsWriter` is in charge of tracking the fieldnorm byte
@@ -12,8 +13,7 @@ use std::{io, iter};
/// `FieldNormsWriter` stores a Vec<u8> for each tracked field, using a
/// byte per document per field.
pub struct FieldNormsWriter {
fields: Vec<Field>,
fieldnorms_buffer: Vec<Vec<u8>>,
fieldnorms_buffers: Vec<Option<Vec<u8>>>,
}
impl FieldNormsWriter {
@@ -23,7 +23,7 @@ impl FieldNormsWriter {
schema
.fields()
.filter_map(|(field, field_entry)| {
if field_entry.is_indexed() {
if field_entry.is_indexed() && field_entry.has_fieldnorms() {
Some(field)
} else {
None
@@ -35,25 +35,20 @@ impl FieldNormsWriter {
/// Initialize with state for tracking the field norm fields
/// specified in the schema.
pub fn for_schema(schema: &Schema) -> FieldNormsWriter {
let fields = FieldNormsWriter::fields_with_fieldnorm(schema);
let max_field = fields
.iter()
.map(Field::field_id)
.max()
.map(|max_field_id| max_field_id as usize + 1)
.unwrap_or(0);
FieldNormsWriter {
fields,
fieldnorms_buffer: iter::repeat_with(Vec::new)
.take(max_field)
.collect::<Vec<_>>(),
let mut fieldnorms_buffers: Vec<Option<Vec<u8>>> = iter::repeat_with(|| None)
.take(schema.num_fields())
.collect();
for field in FieldNormsWriter::fields_with_fieldnorm(schema) {
fieldnorms_buffers[field.field_id() as usize] = Some(Vec::with_capacity(1_000));
}
FieldNormsWriter { fieldnorms_buffers }
}
/// The memory used inclusive childs
pub fn mem_usage(&self) -> usize {
self.fieldnorms_buffer
self.fieldnorms_buffers
.iter()
.flatten()
.map(|buf| buf.capacity())
.sum()
}
@@ -62,8 +57,10 @@ impl FieldNormsWriter {
///
/// Will extend with 0-bytes for documents that have not been seen.
pub fn fill_up_to_max_doc(&mut self, max_doc: DocId) {
for field in self.fields.iter() {
self.fieldnorms_buffer[field.field_id() as usize].resize(max_doc as usize, 0u8);
for fieldnorms_buffer_opt in self.fieldnorms_buffers.iter_mut() {
if let Some(fieldnorms_buffer) = fieldnorms_buffer_opt.as_mut() {
fieldnorms_buffer.resize(max_doc as usize, 0u8);
}
}
}
@@ -76,14 +73,24 @@ impl FieldNormsWriter {
/// * field - the field being set
/// * fieldnorm - the number of terms present in document `doc` in field `field`
pub fn record(&mut self, doc: DocId, field: Field, fieldnorm: u32) {
let fieldnorm_buffer: &mut Vec<u8> = &mut self.fieldnorms_buffer[field.field_id() as usize];
assert!(
fieldnorm_buffer.len() <= doc as usize,
"Cannot register a given fieldnorm twice"
);
if let Some(fieldnorm_buffer) = self
.fieldnorms_buffers
.get_mut(field.field_id() as usize)
.map(Option::as_mut)
.flatten()
{
match fieldnorm_buffer.len().cmp(&(doc as usize)) {
Ordering::Less => {
// we fill intermediary `DocId` as having a fieldnorm of 0.
fieldnorm_buffer.resize(doc as usize + 1, 0u8);
fieldnorm_buffer[doc as usize] = fieldnorm_to_id(fieldnorm);
fieldnorm_buffer.resize(doc as usize, 0u8);
}
Ordering::Equal => {}
Ordering::Greater => {
panic!("Cannot register a given fieldnorm twice")
}
}
fieldnorm_buffer.push(fieldnorm_to_id(fieldnorm));
}
}
/// Serialize the seen fieldnorm values to the serializer for all fields.
@@ -92,17 +99,22 @@ impl FieldNormsWriter {
mut fieldnorms_serializer: FieldNormsSerializer,
doc_id_map: Option<&DocIdMapping>,
) -> io::Result<()> {
for &field in self.fields.iter() {
let fieldnorm_values: &[u8] = &self.fieldnorms_buffer[field.field_id() as usize][..];
for (field, fieldnorms_buffer) in self
.fieldnorms_buffers
.iter()
.enumerate()
.map(|(field_id, fieldnorms_buffer_opt)| {
fieldnorms_buffer_opt.as_ref().map(|fieldnorms_buffer| {
(Field::from_field_id(field_id as u32), fieldnorms_buffer)
})
})
.flatten()
{
if let Some(doc_id_map) = doc_id_map {
let mut mapped_fieldnorm_values = vec![];
mapped_fieldnorm_values.resize(fieldnorm_values.len(), 0u8);
for (new_doc_id, old_doc_id) in doc_id_map.iter_old_doc_ids().enumerate() {
mapped_fieldnorm_values[new_doc_id] = fieldnorm_values[old_doc_id as usize];
}
fieldnorms_serializer.serialize_field(field, &mapped_fieldnorm_values)?;
let remapped_fieldnorm_buffer = doc_id_map.remap(fieldnorms_buffer);
fieldnorms_serializer.serialize_field(field, &remapped_fieldnorm_buffer)?;
} else {
fieldnorms_serializer.serialize_field(field, fieldnorm_values)?;
fieldnorms_serializer.serialize_field(field, fieldnorms_buffer)?;
}
}
fieldnorms_serializer.close()?;

View File

@@ -63,6 +63,24 @@ pub struct DocIdMapping {
}
impl DocIdMapping {
pub fn from_new_id_to_old_id(new_doc_id_to_old: Vec<DocId>) -> Self {
let max_doc = new_doc_id_to_old.len();
let old_max_doc = new_doc_id_to_old
.iter()
.cloned()
.max()
.map(|n| n + 1)
.unwrap_or(0);
let mut old_doc_id_to_new = vec![0; old_max_doc as usize];
for i in 0..max_doc {
old_doc_id_to_new[new_doc_id_to_old[i] as usize] = i as DocId;
}
DocIdMapping {
new_doc_id_to_old,
old_doc_id_to_new,
}
}
/// returns the new doc_id for the old doc_id
pub fn get_new_doc_id(&self, doc_id: DocId) -> DocId {
self.old_doc_id_to_new[doc_id as usize]
@@ -75,6 +93,13 @@ impl DocIdMapping {
pub fn iter_old_doc_ids(&self) -> impl Iterator<Item = DocId> + Clone + '_ {
self.new_doc_id_to_old.iter().cloned()
}
/// Remaps a given array to the new doc ids.
pub fn remap<T: Copy>(&self, els: &[T]) -> Vec<T> {
self.new_doc_id_to_old
.iter()
.map(|old_doc| els[*old_doc as usize])
.collect()
}
}
pub(crate) fn expect_field_id_for_sort_field(
@@ -122,23 +147,13 @@ pub(crate) fn get_doc_id_mapping_from_field(
.into_iter()
.map(|el| el.0)
.collect::<Vec<_>>();
// create old doc_id to new doc_id index (used in posting recorder)
let max_doc = new_doc_id_to_old.len();
let mut old_doc_id_to_new = vec![0; max_doc];
for i in 0..max_doc {
old_doc_id_to_new[new_doc_id_to_old[i] as usize] = i as DocId;
}
let doc_id_map = DocIdMapping {
new_doc_id_to_old,
old_doc_id_to_new,
};
Ok(doc_id_map)
Ok(DocIdMapping::from_new_id_to_old_id(new_doc_id_to_old))
}
#[cfg(test)]
mod tests_indexsorting {
use crate::fastfield::FastFieldReader;
use crate::indexer::doc_id_mapping::DocIdMapping;
use crate::{collector::TopDocs, query::QueryParser, schema::*};
use crate::{schema::Schema, DocAddress};
use crate::{Index, IndexSettings, IndexSortByField, Order};
@@ -475,4 +490,27 @@ mod tests_indexsorting {
assert_eq!(vals, &[3]);
Ok(())
}
#[test]
fn test_doc_mapping() {
let doc_mapping = DocIdMapping::from_new_id_to_old_id(vec![3, 2, 5]);
assert_eq!(doc_mapping.get_old_doc_id(0), 3);
assert_eq!(doc_mapping.get_old_doc_id(1), 2);
assert_eq!(doc_mapping.get_old_doc_id(2), 5);
assert_eq!(doc_mapping.get_new_doc_id(0), 0);
assert_eq!(doc_mapping.get_new_doc_id(1), 0);
assert_eq!(doc_mapping.get_new_doc_id(2), 1);
assert_eq!(doc_mapping.get_new_doc_id(3), 0);
assert_eq!(doc_mapping.get_new_doc_id(4), 0);
assert_eq!(doc_mapping.get_new_doc_id(5), 2);
}
#[test]
fn test_doc_mapping_remap() {
let doc_mapping = DocIdMapping::from_new_id_to_old_id(vec![2, 8, 3]);
assert_eq!(
&doc_mapping.remap(&[0, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000]),
&[2000, 8000, 3000]
);
}
}

View File

@@ -41,31 +41,54 @@ use tantivy_bitpacker::minmax;
/// We do not allow segments with more than
pub const MAX_DOC_LIMIT: u32 = 1 << 31;
fn compute_total_num_tokens(readers: &[SegmentReader], field: Field) -> crate::Result<u64> {
let mut total_tokens = 0u64;
fn estimate_total_num_tokens_in_single_segment(
reader: &SegmentReader,
field: Field,
) -> crate::Result<u64> {
// There are no deletes. We can simply use the exact value saved into the posting list.
// Note that this value is not necessarily exact as it could have been the result of a merge between
// segments themselves containing deletes.
if !reader.has_deletes() {
return Ok(reader.inverted_index(field)?.total_num_tokens());
}
// When there are deletes, we use an approximation either
// by using the fieldnorm.
if let Some(fieldnorm_reader) = reader.fieldnorms_readers().get_field(field)? {
let mut count: [usize; 256] = [0; 256];
for reader in readers {
if reader.has_deletes() {
// if there are deletes, then we use an approximation
// using the fieldnorm
let fieldnorms_reader = reader.get_fieldnorms_reader(field)?;
for doc in reader.doc_ids_alive() {
let fieldnorm_id = fieldnorms_reader.fieldnorm_id(doc);
let fieldnorm_id = fieldnorm_reader.fieldnorm_id(doc);
count[fieldnorm_id as usize] += 1;
}
} else {
total_tokens += reader.inverted_index(field)?.total_num_tokens();
}
}
Ok(total_tokens
+ count
let total_num_tokens = count
.iter()
.cloned()
.enumerate()
.map(|(fieldnorm_ord, count)| {
count as u64 * u64::from(FieldNormReader::id_to_fieldnorm(fieldnorm_ord as u8))
})
.sum::<u64>())
.sum::<u64>();
return Ok(total_num_tokens);
}
// There are no fieldnorms available.
// Here we just do a pro-rata with the overall number of tokens an the ratio of
// documents alive.
let segment_num_tokens = reader.inverted_index(field)?.total_num_tokens();
if reader.max_doc() == 0 {
// That supposedly never happens, but let's be a bit defensive here.
return Ok(0u64);
}
let ratio = reader.num_docs() as f64 / reader.max_doc() as f64;
Ok((segment_num_tokens as f64 * ratio) as u64)
}
fn estimate_total_num_tokens(readers: &[SegmentReader], field: Field) -> crate::Result<u64> {
let mut total_num_tokens: u64 = 0;
for reader in readers {
total_num_tokens += estimate_total_num_tokens_in_single_segment(reader, field)?;
}
Ok(total_num_tokens)
}
pub struct IndexMerger {
@@ -851,10 +874,9 @@ impl IndexMerger {
segment_map[*old_doc_id as usize] = Some(new_doc_id as DocId);
}
// The total number of tokens will only be exact when there has been no deletes.
//
// Otherwise, we approximate by removing deleted documents proportionally.
let total_num_tokens: u64 = compute_total_num_tokens(&self.readers, indexed_field)?;
// Note that the total number of tokens is not exact.
// It is only used as a parameter in the BM25 formula.
let total_num_tokens: u64 = estimate_total_num_tokens(&self.readers, indexed_field)?;
// Create the total list of doc ids
// by stacking the doc ids from the different segment.

View File

@@ -234,7 +234,6 @@ impl SegmentWriter {
term_buffer,
)
};
self.fieldnorms_writer.record(doc_id, field, num_tokens);
}
FieldType::U64(_) => {

View File

@@ -308,8 +308,6 @@ pub struct PostingsSerializer<W: Write> {
fieldnorm_reader: Option<FieldNormReader>,
bm25_weight: Option<Bm25Weight>,
num_docs: u32, // Number of docs in the segment
avg_fieldnorm: Score, // Average number of term in the field for that segment.
// this value is used to compute the block wand information.
}
@@ -321,10 +319,6 @@ impl<W: Write> PostingsSerializer<W> {
mode: IndexRecordOption,
fieldnorm_reader: Option<FieldNormReader>,
) -> PostingsSerializer<W> {
let num_docs = fieldnorm_reader
.as_ref()
.map(|fieldnorm_reader| fieldnorm_reader.num_docs())
.unwrap_or(0u32);
PostingsSerializer {
output_write: CountingWriter::wrap(write),
@@ -339,21 +333,33 @@ impl<W: Write> PostingsSerializer<W> {
fieldnorm_reader,
bm25_weight: None,
num_docs,
avg_fieldnorm,
}
}
pub fn new_term(&mut self, term_doc_freq: u32) {
if self.mode.has_freq() && self.num_docs > 0 {
let bm25_weight = Bm25Weight::for_one_term(
term_doc_freq as u64,
self.num_docs as u64,
self.avg_fieldnorm,
);
self.bm25_weight = Some(bm25_weight);
self.bm25_weight = None;
if !self.mode.has_freq() {
return;
}
let num_docs_in_segment: u64 =
if let Some(fieldnorm_reader) = self.fieldnorm_reader.as_ref() {
fieldnorm_reader.num_docs() as u64
} else {
return;
};
if num_docs_in_segment == 0 {
return;
}
self.bm25_weight = Some(Bm25Weight::for_one_term(
term_doc_freq as u64,
num_docs_in_segment,
self.avg_fieldnorm,
));
}
fn write_block(&mut self) {

View File

@@ -92,21 +92,16 @@ impl TermQuery {
searcher: &Searcher,
scoring_enabled: bool,
) -> crate::Result<TermWeight> {
let term = self.term.clone();
let field_entry = searcher.schema().get_field_entry(term.field());
let field_entry = searcher.schema().get_field_entry(self.term.field());
if !field_entry.is_indexed() {
return Err(crate::TantivyError::SchemaError(format!(
"Field {:?} is not indexed",
field_entry.name()
)));
let error_msg = format!("Field {:?} is not indexed.", field_entry.name());
return Err(crate::TantivyError::SchemaError(error_msg));
}
let bm25_weight;
if scoring_enabled {
bm25_weight = Bm25Weight::for_terms(searcher, &[term])?;
let bm25_weight = if scoring_enabled {
Bm25Weight::for_terms(searcher, &[self.term.clone()])?
} else {
bm25_weight =
Bm25Weight::new(Explanation::new("<no score>".to_string(), 1.0f32), 1.0f32);
}
Bm25Weight::new(Explanation::new("<no score>".to_string(), 1.0f32), 1.0f32)
};
let index_record_option = if scoring_enabled {
self.index_record_option
} else {

View File

@@ -106,11 +106,13 @@ impl TermWeight {
) -> crate::Result<TermScorer> {
let field = self.term.field();
let inverted_index = reader.inverted_index(field)?;
let fieldnorm_reader = if self.scoring_enabled {
reader.get_fieldnorms_reader(field)?
let fieldnorm_reader_opt = if self.scoring_enabled {
reader.fieldnorms_readers().get_field(field)?
} else {
FieldNormReader::constant(reader.max_doc(), 1)
None
};
let fieldnorm_reader =
fieldnorm_reader_opt.unwrap_or_else(|| FieldNormReader::constant(reader.max_doc(), 1));
let similarity_weight = self.similarity_weight.boost_by(boost);
let postings_opt: Option<SegmentPostings> =
inverted_index.read_postings(&self.term, self.index_record_option)?;

View File

@@ -3,19 +3,51 @@ use std::ops::BitOr;
use super::flags::{FastFlag, IndexedFlag, SchemaFlagList, StoredFlag};
/// Define how an a bytes field should be handled by tantivy.
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize, Default)]
#[derive(Clone, Debug, Default, PartialEq, Eq, Serialize, Deserialize)]
#[serde(from = "BytesOptionsDeser")]
pub struct BytesOptions {
indexed: bool,
fieldnorms: bool,
fast: bool,
stored: bool,
}
/// For backward compability we add an intermediary to interpret the
/// lack of fieldnorms attribute as "true" iff indexed.
///
/// (Downstream, for the moment, this attribute is not used anyway if not indexed...)
/// Note that: newly serialized IntOptions will include the new attribute.
#[derive(Deserialize)]
struct BytesOptionsDeser {
indexed: bool,
#[serde(default)]
fieldnorms: Option<bool>,
fast: bool,
stored: bool,
}
impl From<BytesOptionsDeser> for BytesOptions {
fn from(deser: BytesOptionsDeser) -> Self {
BytesOptions {
indexed: deser.indexed,
fieldnorms: deser.fieldnorms.unwrap_or(deser.indexed),
fast: deser.fast,
stored: deser.stored,
}
}
}
impl BytesOptions {
/// Returns true iff the value is indexed.
pub fn is_indexed(&self) -> bool {
self.indexed
}
/// Returns true iff the value is normed.
pub fn fieldnorms(&self) -> bool {
self.fieldnorms
}
/// Returns true iff the value is a fast field.
pub fn is_fast(&self) -> bool {
self.fast
@@ -35,6 +67,15 @@ impl BytesOptions {
self
}
/// Set the field as normed.
///
/// Setting an integer as normed will generate
/// the fieldnorm data for it.
pub fn set_fieldnorms(mut self) -> BytesOptions {
self.fieldnorms = true;
self
}
/// Set the field as a single-valued fast field.
///
/// Fast fields are designed for random access.
@@ -63,6 +104,7 @@ impl<T: Into<BytesOptions>> BitOr<T> for BytesOptions {
let other = other.into();
BytesOptions {
indexed: self.indexed | other.indexed,
fieldnorms: self.fieldnorms | other.fieldnorms,
stored: self.stored | other.stored,
fast: self.fast | other.fast,
}
@@ -79,6 +121,7 @@ impl From<FastFlag> for BytesOptions {
fn from(_: FastFlag) -> Self {
BytesOptions {
indexed: false,
fieldnorms: false,
stored: false,
fast: true,
}
@@ -89,6 +132,7 @@ impl From<StoredFlag> for BytesOptions {
fn from(_: StoredFlag) -> Self {
BytesOptions {
indexed: false,
fieldnorms: false,
stored: true,
fast: false,
}
@@ -99,6 +143,7 @@ impl From<IndexedFlag> for BytesOptions {
fn from(_: IndexedFlag) -> Self {
BytesOptions {
indexed: true,
fieldnorms: true,
stored: false,
fast: false,
}
@@ -123,7 +168,10 @@ mod tests {
#[test]
fn test_bytes_option_fast_flag() {
assert_eq!(BytesOptions::default().set_fast(), FAST.into());
assert_eq!(BytesOptions::default().set_indexed(), INDEXED.into());
assert_eq!(
BytesOptions::default().set_indexed().set_fieldnorms(),
INDEXED.into()
);
assert_eq!(BytesOptions::default().set_stored(), STORED.into());
}
#[test]
@@ -133,11 +181,17 @@ mod tests {
(FAST | STORED).into()
);
assert_eq!(
BytesOptions::default().set_indexed().set_fast(),
BytesOptions::default()
.set_indexed()
.set_fieldnorms()
.set_fast(),
(INDEXED | FAST).into()
);
assert_eq!(
BytesOptions::default().set_stored().set_indexed(),
BytesOptions::default()
.set_stored()
.set_fieldnorms()
.set_indexed(),
(STORED | INDEXED).into()
);
}
@@ -147,8 +201,89 @@ mod tests {
assert!(!BytesOptions::default().is_stored());
assert!(!BytesOptions::default().is_fast());
assert!(!BytesOptions::default().is_indexed());
assert!(!BytesOptions::default().fieldnorms());
assert!(BytesOptions::default().set_stored().is_stored());
assert!(BytesOptions::default().set_fast().is_fast());
assert!(BytesOptions::default().set_indexed().is_indexed());
assert!(BytesOptions::default().set_fieldnorms().fieldnorms());
}
#[test]
fn test_bytes_options_deser_if_fieldnorm_missing_indexed_true() {
let json = r#"{
"indexed": true,
"fast": false,
"stored": false
}"#;
let bytes_options: BytesOptions = serde_json::from_str(json).unwrap();
assert_eq!(
&bytes_options,
&BytesOptions {
indexed: true,
fieldnorms: true,
fast: false,
stored: false
}
);
}
#[test]
fn test_bytes_options_deser_if_fieldnorm_missing_indexed_false() {
let json = r#"{
"indexed": false,
"stored": false,
"fast": false
}"#;
let bytes_options: BytesOptions = serde_json::from_str(json).unwrap();
assert_eq!(
&bytes_options,
&BytesOptions {
indexed: false,
fieldnorms: false,
fast: false,
stored: false
}
);
}
#[test]
fn test_bytes_options_deser_if_fieldnorm_false_indexed_true() {
let json = r#"{
"indexed": true,
"fieldnorms": false,
"fast": false,
"stored": false
}"#;
let bytes_options: BytesOptions = serde_json::from_str(json).unwrap();
assert_eq!(
&bytes_options,
&BytesOptions {
indexed: true,
fieldnorms: false,
fast: false,
stored: false
}
);
}
#[test]
fn test_bytes_options_deser_if_fieldnorm_true_indexed_false() {
// this one is kind of useless, at least at the moment
let json = r#"{
"indexed": false,
"fieldnorms": true,
"fast": false,
"stored": false
}"#;
let bytes_options: BytesOptions = serde_json::from_str(json).unwrap();
assert_eq!(
&bytes_options,
&BytesOptions {
indexed: false,
fieldnorms: true,
fast: false,
stored: false
}
);
}
}

View File

@@ -114,6 +114,11 @@ impl FieldEntry {
self.field_type.is_indexed()
}
/// Returns true iff the field is normed
pub fn has_fieldnorms(&self) -> bool {
self.field_type.has_fieldnorms()
}
/// Returns true iff the field is a int (signed or unsigned) fast field
pub fn is_fast(&self) -> bool {
match self.field_type {
@@ -142,7 +147,10 @@ impl FieldEntry {
#[cfg(test)]
mod tests {
use super::*;
use crate::schema::TEXT;
use crate::{
schema::{Schema, TextFieldIndexing, TEXT},
Index,
};
use serde_json;
#[test]
@@ -161,6 +169,7 @@ mod tests {
"options": {
"indexing": {
"record": "position",
"fieldnorms": true,
"tokenizer": "default"
},
"stored": false
@@ -187,6 +196,7 @@ mod tests {
"options": {
"indexing": {
"record": "position",
"fieldnorms": true,
"tokenizer": "default"
},
"stored": false
@@ -199,4 +209,21 @@ mod tests {
_ => panic!("expected FieldType::Str"),
}
}
#[test]
fn test_missing_fieldnorms() -> crate::Result<()> {
let mut schema_builder = Schema::builder();
let no_field_norm = TextOptions::default()
.set_indexing_options(TextFieldIndexing::default().set_fieldnorms(false));
let text = schema_builder.add_text_field("text", no_field_norm);
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
let mut index_writer = index.writer_for_tests()?;
index_writer.add_document(doc!(text=>"abc"))?;
index_writer.commit()?;
let searcher = index.reader()?.searcher();
let err = searcher.segment_reader(0u32).get_fieldnorms_reader(text);
assert!(matches!(err, Err(crate::TantivyError::SchemaError(_))));
Ok(())
}
}

View File

@@ -97,6 +97,22 @@ impl FieldType {
}
}
/// returns true iff the field is normed.
pub fn has_fieldnorms(&self) -> bool {
match *self {
FieldType::Str(ref text_options) => text_options
.get_indexing_options()
.map(|options| options.fieldnorms())
.unwrap_or(false),
FieldType::U64(ref int_options)
| FieldType::I64(ref int_options)
| FieldType::F64(ref int_options)
| FieldType::Date(ref int_options) => int_options.fieldnorms(),
FieldType::HierarchicalFacet(_) => false,
FieldType::Bytes(ref bytes_options) => bytes_options.fieldnorms(),
}
}
/// Given a field configuration, return the maximal possible
/// `IndexRecordOption` available.
///

View File

@@ -20,7 +20,7 @@ pub const STORED: SchemaFlagList<StoredFlag, ()> = SchemaFlagList {
#[derive(Clone)]
pub struct IndexedFlag;
/// Flag to mark the field as indexed. An indexed field is searchable.
/// Flag to mark the field as indexed. An indexed field is searchable and has a fieldnorm.
///
/// The `INDEXED` flag can only be used when building `IntOptions` (`u64`, `i64` and `f64` fields)
/// Of course, text fields can also be indexed... But this is expressed by using either the

View File

@@ -16,13 +16,42 @@ pub enum Cardinality {
/// Define how an u64, i64, of f64 field should be handled by tantivy.
#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize, Default)]
#[serde(from = "IntOptionsDeser")]
pub struct IntOptions {
indexed: bool,
// This boolean has no effect if the field is not marked as indexed too.
fieldnorms: bool, // This attribute only has an effect if indexed is true.
#[serde(skip_serializing_if = "Option::is_none")]
fast: Option<Cardinality>,
stored: bool,
}
/// For backward compability we add an intermediary to interpret the
/// lack of fieldnorms attribute as "true" iff indexed.
///
/// (Downstream, for the moment, this attribute is not used anyway if not indexed...)
/// Note that: newly serialized IntOptions will include the new attribute.
#[derive(Deserialize)]
struct IntOptionsDeser {
indexed: bool,
#[serde(default)]
fieldnorms: Option<bool>, // This attribute only has an effect if indexed is true.
#[serde(default)]
fast: Option<Cardinality>,
stored: bool,
}
impl From<IntOptionsDeser> for IntOptions {
fn from(deser: IntOptionsDeser) -> Self {
IntOptions {
indexed: deser.indexed,
fieldnorms: deser.fieldnorms.unwrap_or(deser.indexed),
fast: deser.fast,
stored: deser.stored,
}
}
}
impl IntOptions {
/// Returns true iff the value is stored.
pub fn is_stored(&self) -> bool {
@@ -34,6 +63,11 @@ impl IntOptions {
self.indexed
}
/// Returns true iff the field has fieldnorm.
pub fn fieldnorms(&self) -> bool {
self.fieldnorms && self.indexed
}
/// Returns true iff the value is a fast field.
pub fn is_fast(&self) -> bool {
self.fast.is_some()
@@ -59,6 +93,15 @@ impl IntOptions {
self
}
/// Set the field with fieldnorm.
///
/// Setting an integer as fieldnorm will generate
/// the fieldnorm data for it.
pub fn set_fieldnorm(mut self) -> IntOptions {
self.fieldnorms = true;
self
}
/// Set the field as a single-valued fast field.
///
/// Fast fields are designed for random access.
@@ -89,6 +132,7 @@ impl From<FastFlag> for IntOptions {
fn from(_: FastFlag) -> Self {
IntOptions {
indexed: false,
fieldnorms: false,
stored: false,
fast: Some(Cardinality::SingleValue),
}
@@ -99,6 +143,7 @@ impl From<StoredFlag> for IntOptions {
fn from(_: StoredFlag) -> Self {
IntOptions {
indexed: false,
fieldnorms: false,
stored: true,
fast: None,
}
@@ -109,6 +154,7 @@ impl From<IndexedFlag> for IntOptions {
fn from(_: IndexedFlag) -> Self {
IntOptions {
indexed: true,
fieldnorms: true,
stored: false,
fast: None,
}
@@ -122,6 +168,7 @@ impl<T: Into<IntOptions>> BitOr<T> for IntOptions {
let other = other.into();
IntOptions {
indexed: self.indexed | other.indexed,
fieldnorms: self.fieldnorms | other.fieldnorms,
stored: self.stored | other.stored,
fast: self.fast.or(other.fast),
}
@@ -138,3 +185,83 @@ where
Self::from(head_tail.head) | Self::from(head_tail.tail)
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn test_int_options_deser_if_fieldnorm_missing_indexed_true() {
let json = r#"{
"indexed": true,
"stored": false
}"#;
let int_options: IntOptions = serde_json::from_str(json).unwrap();
assert_eq!(
&int_options,
&IntOptions {
indexed: true,
fieldnorms: true,
fast: None,
stored: false
}
);
}
#[test]
fn test_int_options_deser_if_fieldnorm_missing_indexed_false() {
let json = r#"{
"indexed": false,
"stored": false
}"#;
let int_options: IntOptions = serde_json::from_str(json).unwrap();
assert_eq!(
&int_options,
&IntOptions {
indexed: false,
fieldnorms: false,
fast: None,
stored: false
}
);
}
#[test]
fn test_int_options_deser_if_fieldnorm_false_indexed_true() {
let json = r#"{
"indexed": true,
"fieldnorms": false,
"stored": false
}"#;
let int_options: IntOptions = serde_json::from_str(json).unwrap();
assert_eq!(
&int_options,
&IntOptions {
indexed: true,
fieldnorms: false,
fast: None,
stored: false
}
);
}
#[test]
fn test_int_options_deser_if_fieldnorm_true_indexed_false() {
// this one is kind of useless, at least at the moment
let json = r#"{
"indexed": false,
"fieldnorms": true,
"stored": false
}"#;
let int_options: IntOptions = serde_json::from_str(json).unwrap();
assert_eq!(
&int_options,
&IntOptions {
indexed: false,
fieldnorms: true,
fast: None,
stored: false
}
);
}
}

View File

@@ -240,6 +240,11 @@ impl Schema {
self.get_field_entry(field).name()
}
/// Returns the number of fields in the schema.
pub fn num_fields(&self) -> usize {
self.0.fields.len()
}
/// Return the list of all the `Field`s.
pub fn fields(&self) -> impl Iterator<Item = (Field, &FieldEntry)> {
self.0
@@ -427,9 +432,17 @@ mod tests {
.set_fast(Cardinality::SingleValue);
let score_options = IntOptions::default()
.set_indexed()
.set_fieldnorm()
.set_fast(Cardinality::SingleValue);
schema_builder.add_text_field("title", TEXT);
schema_builder.add_text_field("author", STRING);
schema_builder.add_text_field(
"author",
TextOptions::default().set_indexing_options(
TextFieldIndexing::default()
.set_tokenizer("raw")
.set_fieldnorms(false),
),
);
schema_builder.add_u64_field("count", count_options);
schema_builder.add_i64_field("popularity", popularity_options);
schema_builder.add_f64_field("score", score_options);
@@ -442,6 +455,7 @@ mod tests {
"options": {
"indexing": {
"record": "position",
"fieldnorms": true,
"tokenizer": "default"
},
"stored": false
@@ -453,6 +467,7 @@ mod tests {
"options": {
"indexing": {
"record": "basic",
"fieldnorms": false,
"tokenizer": "raw"
},
"stored": false
@@ -463,6 +478,7 @@ mod tests {
"type": "u64",
"options": {
"indexed": false,
"fieldnorms": false,
"fast": "single",
"stored": true
}
@@ -472,6 +488,7 @@ mod tests {
"type": "i64",
"options": {
"indexed": false,
"fieldnorms": false,
"fast": "single",
"stored": true
}
@@ -481,6 +498,7 @@ mod tests {
"type": "f64",
"options": {
"indexed": true,
"fieldnorms": true,
"fast": "single",
"stored": false
}
@@ -743,6 +761,7 @@ mod tests {
let timestamp_options = IntOptions::default()
.set_stored()
.set_indexed()
.set_fieldnorm()
.set_fast(SingleValue);
schema_builder.add_text_field("_id", id_options);
schema_builder.add_date_field("_timestamp", timestamp_options);
@@ -754,6 +773,7 @@ mod tests {
"options": {
"indexing": {
"record": "position",
"fieldnorms": true,
"tokenizer": "default"
},
"stored": false
@@ -764,6 +784,7 @@ mod tests {
"type": "i64",
"options": {
"indexed": false,
"fieldnorms": false,
"fast": "single",
"stored": true
}
@@ -784,6 +805,7 @@ mod tests {
"options": {
"indexing": {
"record": "basic",
"fieldnorms": true,
"tokenizer": "raw"
},
"stored": true
@@ -794,6 +816,7 @@ mod tests {
"type": "date",
"options": {
"indexed": true,
"fieldnorms": true,
"fast": "single",
"stored": true
}
@@ -804,6 +827,7 @@ mod tests {
"options": {
"indexing": {
"record": "position",
"fieldnorms": true,
"tokenizer": "default"
},
"stored": false
@@ -814,6 +838,7 @@ mod tests {
"type": "i64",
"options": {
"indexed": false,
"fieldnorms": false,
"fast": "single",
"stored": true
}

View File

@@ -45,6 +45,7 @@ impl TextOptions {
#[derive(Clone, PartialEq, Debug, Serialize, Deserialize)]
pub struct TextFieldIndexing {
record: IndexRecordOption,
fieldnorms: bool,
tokenizer: Cow<'static, str>,
}
@@ -53,6 +54,7 @@ impl Default for TextFieldIndexing {
TextFieldIndexing {
tokenizer: Cow::Borrowed("default"),
record: IndexRecordOption::Basic,
fieldnorms: true,
}
}
}
@@ -69,6 +71,17 @@ impl TextFieldIndexing {
&self.tokenizer
}
/// Sets fieldnorms
pub fn set_fieldnorms(mut self, fieldnorms: bool) -> TextFieldIndexing {
self.fieldnorms = fieldnorms;
self
}
/// Returns true iff fieldnorms are stored.
pub fn fieldnorms(&self) -> bool {
self.fieldnorms
}
/// Sets which information should be indexed with the tokens.
///
/// See [IndexRecordOption](./enum.IndexRecordOption.html) for more detail.
@@ -89,6 +102,7 @@ impl TextFieldIndexing {
pub const STRING: TextOptions = TextOptions {
indexing: Some(TextFieldIndexing {
tokenizer: Cow::Borrowed("raw"),
fieldnorms: true,
record: IndexRecordOption::Basic,
}),
stored: false,
@@ -98,6 +112,7 @@ pub const STRING: TextOptions = TextOptions {
pub const TEXT: TextOptions = TextOptions {
indexing: Some(TextFieldIndexing {
tokenizer: Cow::Borrowed("default"),
fieldnorms: true,
record: IndexRecordOption::WithFreqsAndPositions,
}),
stored: false,

View File

@@ -47,7 +47,7 @@ mod tests {
use crate::directory::OwnedBytes;
use crate::indexer::NoMergePolicy;
use crate::schema::{SchemaBuilder, STORED, STRING};
use crate::schema::{SchemaBuilder, STORED, TEXT};
use crate::store::index::Checkpoint;
use crate::{DocAddress, DocId, Index, Term};
@@ -128,7 +128,7 @@ mod tests {
#[test]
fn test_merge_store_with_stacking_reproducing_issue969() -> crate::Result<()> {
let mut schema_builder = SchemaBuilder::default();
let text = schema_builder.add_text_field("text", STORED | STRING);
let text = schema_builder.add_text_field("text", STORED | TEXT);
let body = schema_builder.add_text_field("body", STORED);
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);