Compare commits

..

21 Commits

Author SHA1 Message Date
trinity-1686a
2b686ffa54 fix columnar tests 2023-11-10 11:46:58 +01:00
trinity-1686a
04b3a27a0a increment sstable version number 2023-11-10 11:38:58 +01:00
trinity-1686a
710cf1efa6 implement multilayer sstable writer 2023-11-10 11:09:50 +01:00
trinity-1686a
8103790c16 define and implement reading multi layer index sstable 2023-11-09 15:42:00 +01:00
trinity-1686a
7a0064db1f bump index version (#2237)
* bump index version

and add constant for lowest supported version

* use range instead of handcoded bounds
2023-11-06 19:02:37 +01:00
PSeitz
2e7327205d fix coverage run (#2232)
coverage run uses the compare_hash_only feature which is not compativle
with the test_hashmap_size test
2023-11-06 11:18:38 +00:00
Paul Masurel
7bc5bf78e2 Fixing functional tests. (#2239) 2023-11-05 18:18:39 +09:00
giovannicuccu
ef603c8c7e rename ReloadPolicy onCommit to onCommitWithDelay (#2235)
* rename ReloadPolicy onCommit to onCommitWithDelay

* fix format issues

---------

Co-authored-by: Giovanni Cuccu <gcuccu@imolainformatica.it>
2023-11-03 12:22:10 +01:00
PSeitz
28dd6b6546 collect json paths in indexing (#2231)
* collect json paths in indexing

* remove unsafe iter_mut_keys
2023-11-01 11:25:17 +01:00
trinity-1686a
1dda2bb537 handle * inside term in query parser (#2228) 2023-10-27 08:57:02 +02:00
PSeitz
bf6544cf28 fix mmap::Advice reexport (#2230) 2023-10-27 14:09:25 +09:00
PSeitz
ccecf946f7 tantivy 0.21.1 (#2227) 2023-10-27 05:01:44 +02:00
PSeitz
19a859d6fd term hashmap remove copy in is_empty, unused unordered_id (#2229) 2023-10-27 05:01:32 +02:00
PSeitz
83af14caa4 Fix range query (#2226)
Fix range query end check in advance
Rename vars to reduce ambiguity
add tests

Fixes #2225
2023-10-25 09:17:31 +02:00
PSeitz
4feeb2323d fix clippy (#2223) 2023-10-24 10:05:22 +02:00
PSeitz
07bf66a197 json path writer (#2224)
* refactor logic to JsonPathWriter

* use in encode_column_name

* add inlines

* move unsafe block
2023-10-24 09:45:50 +02:00
trinity-1686a
0d4589219b encode some part of posting list as -1 instead of direct values (#2185)
* add support for delta-1 encoding posting list

* encode term frequency minus one

* don't emit tf for json integer terms

* make skipreader not pub(crate) mutable
2023-10-20 16:58:26 +02:00
PSeitz
c2b0469180 improve docs, rework exports (#2220)
* rework exports

move snippet and advice
make indexer pub, remove indexer reexports

* add deprecation warning

* add architecture overview
2023-10-18 09:22:24 +02:00
PSeitz
7e1980b218 run coverage only after merge (#2212)
* run coverage only after merge

coverage is a quite slow step in CI. It can be run only after merging

* Apply suggestions from code review

Co-authored-by: Paul Masurel <paul@quickwit.io>

---------

Co-authored-by: Paul Masurel <paul@quickwit.io>
2023-10-18 07:19:36 +02:00
PSeitz
ecb9a89a9f add compat mode for JSON (#2219) 2023-10-17 10:00:55 +02:00
PSeitz
5e06e504e6 split into ReferenceValueLeaf (#2217) 2023-10-16 16:31:30 +02:00
74 changed files with 2088 additions and 869 deletions

View File

@@ -3,8 +3,6 @@ name: Coverage
on:
push:
branches: [main]
pull_request:
branches: [main]
# Ensures that we cancel running jobs for the same PR / same workflow.
concurrency:

View File

@@ -39,6 +39,13 @@ jobs:
- name: Check Formatting
run: cargo +nightly fmt --all -- --check
- name: Check Stable Compilation
run: cargo build --all-features
- name: Check Bench Compilation
run: cargo +nightly bench --no-run --profile=dev --all-features
- uses: actions-rs/clippy-check@v1
with:

View File

@@ -1,3 +1,9 @@
Tantivy 0.21.1
================================
#### Bugfixes
- Range queries on fast fields with less values on that field than documents had an invalid end condition, leading to missing results. [#2226](https://github.com/quickwit-oss/tantivy/issues/2226)(@appaquet @PSeitz)
- Increase the minimum memory budget from 3MB to 15MB to avoid single doc segments (API fix). [#2176](https://github.com/quickwit-oss/tantivy/issues/2176)(@PSeitz)
Tantivy 0.21
================================
#### Bugfixes

View File

@@ -19,7 +19,6 @@ oneshot = "0.1.5"
base64 = "0.21.0"
byteorder = "1.4.3"
crc32fast = "1.3.2"
tracing = "0.1"
once_cell = "1.10.0"
regex = { version = "1.5.5", default-features = false, features = ["std", "unicode"] }
aho-corasick = "1.0"
@@ -38,7 +37,7 @@ uuid = { version = "1.0.0", features = ["v4", "serde"] }
crossbeam-channel = "0.5.4"
rust-stemmers = "1.2.0"
downcast-rs = "1.2.0"
bitpacking = { version = "0.8.4", default-features = false, features = ["bitpacker4x"] }
bitpacking = { git = "https://github.com/quickwit-oss/bitpacking", rev = "f730b75", default-features = false, features = ["bitpacker4x"] }
census = "0.4.0"
rustc-hash = "1.1.0"
thiserror = "1.0.30"
@@ -117,6 +116,11 @@ unstable = [] # useful for benches.
quickwit = ["sstable", "futures-util"]
# Compares only the hash of a string when indexing data.
# Increases indexing speed, but may lead to extremely rare missing terms, when there's a hash collision.
# Uses 64bit ahash.
compare_hash_only = ["stacker/compare_hash_only"]
[workspace]
members = ["query-grammar", "bitpacker", "common", "ownedbytes", "stacker", "sstable", "tokenizer-api", "columnar"]

View File

@@ -367,7 +367,7 @@ mod test {
let mut output: Vec<u32> = Vec::new();
for len in [0, 1, 2, 32, 33, 34, 64] {
for start_idx in 0u32..32u32 {
output.resize(len as usize, 0);
output.resize(len, 0);
bitunpacker.get_batch_u32s(start_idx, &buffer, &mut output);
for i in 0..len {
let expected = (start_idx + i as u32) & mask;

View File

@@ -215,12 +215,12 @@ mod bench {
let vals: Vec<RowId> = (0..TOTAL_NUM_VALUES)
.map(|_| rng.gen_bool(fill_ratio))
.enumerate()
.filter(|(pos, val)| *val)
.filter(|(_pos, val)| *val)
.map(|(pos, _)| pos as RowId)
.collect();
serialize_optional_index(&&vals[..], TOTAL_NUM_VALUES, &mut out).unwrap();
let codec = open_optional_index(OwnedBytes::new(out)).unwrap();
codec
open_optional_index(OwnedBytes::new(out)).unwrap()
}
fn random_range_iterator(
@@ -242,7 +242,7 @@ mod bench {
}
fn n_percent_step_iterator(percent: f32, num_values: u32) -> impl Iterator<Item = u32> {
let ratio = percent as f32 / 100.0;
let ratio = percent / 100.0;
let step_size = (1f32 / ratio) as u32;
let deviation = step_size - 1;
random_range_iterator(0, num_values, step_size, deviation)

View File

@@ -338,7 +338,7 @@ impl ColumnarWriter {
let mut columns: Vec<(&[u8], ColumnType, Addr)> = self
.numerical_field_hash_map
.iter()
.map(|(column_name, addr, _)| {
.map(|(column_name, addr)| {
let numerical_column_writer: NumericalColumnWriter =
self.numerical_field_hash_map.read(addr);
let column_type = numerical_column_writer.numerical_type().into();
@@ -348,27 +348,27 @@ impl ColumnarWriter {
columns.extend(
self.bytes_field_hash_map
.iter()
.map(|(term, addr, _)| (term, ColumnType::Bytes, addr)),
.map(|(term, addr)| (term, ColumnType::Bytes, addr)),
);
columns.extend(
self.str_field_hash_map
.iter()
.map(|(column_name, addr, _)| (column_name, ColumnType::Str, addr)),
.map(|(column_name, addr)| (column_name, ColumnType::Str, addr)),
);
columns.extend(
self.bool_field_hash_map
.iter()
.map(|(column_name, addr, _)| (column_name, ColumnType::Bool, addr)),
.map(|(column_name, addr)| (column_name, ColumnType::Bool, addr)),
);
columns.extend(
self.ip_addr_field_hash_map
.iter()
.map(|(column_name, addr, _)| (column_name, ColumnType::IpAddr, addr)),
.map(|(column_name, addr)| (column_name, ColumnType::IpAddr, addr)),
);
columns.extend(
self.datetime_field_hash_map
.iter()
.map(|(column_name, addr, _)| (column_name, ColumnType::DateTime, addr)),
.map(|(column_name, addr)| (column_name, ColumnType::DateTime, addr)),
);
columns.sort_unstable_by_key(|(column_name, col_type, _)| (*column_name, *col_type));

View File

@@ -26,7 +26,7 @@ fn test_dataframe_writer_str() {
assert_eq!(columnar.num_columns(), 1);
let cols: Vec<DynamicColumnHandle> = columnar.read_columns("my_string").unwrap();
assert_eq!(cols.len(), 1);
assert_eq!(cols[0].num_bytes(), 87);
assert_eq!(cols[0].num_bytes(), 99);
}
#[test]
@@ -40,7 +40,7 @@ fn test_dataframe_writer_bytes() {
assert_eq!(columnar.num_columns(), 1);
let cols: Vec<DynamicColumnHandle> = columnar.read_columns("my_string").unwrap();
assert_eq!(cols.len(), 1);
assert_eq!(cols[0].num_bytes(), 87);
assert_eq!(cols[0].num_bytes(), 99);
}
#[test]
@@ -330,9 +330,9 @@ fn bytes_strategy() -> impl Strategy<Value = &'static [u8]> {
// A random column value
fn column_value_strategy() -> impl Strategy<Value = ColumnValue> {
prop_oneof![
10 => string_strategy().prop_map(|s| ColumnValue::Str(s)),
1 => bytes_strategy().prop_map(|b| ColumnValue::Bytes(b)),
40 => num_strategy().prop_map(|n| ColumnValue::Numerical(n)),
10 => string_strategy().prop_map(ColumnValue::Str),
1 => bytes_strategy().prop_map(ColumnValue::Bytes),
40 => num_strategy().prop_map(ColumnValue::Numerical),
1 => (1u16..3u16).prop_map(|ip_addr_byte| ColumnValue::IpAddr(Ipv6Addr::new(
127,
0,
@@ -343,7 +343,7 @@ fn column_value_strategy() -> impl Strategy<Value = ColumnValue> {
0,
ip_addr_byte
))),
1 => any::<bool>().prop_map(|b| ColumnValue::Bool(b)),
1 => any::<bool>().prop_map(ColumnValue::Bool),
1 => (0_679_723_993i64..1_679_723_995i64)
.prop_map(|val| { ColumnValue::DateTime(DateTime::from_timestamp_secs(val)) })
]
@@ -419,8 +419,8 @@ fn build_columnar_with_mapping(
columnar_writer
.serialize(num_docs, old_to_new_row_ids_opt, &mut buffer)
.unwrap();
let columnar_reader = ColumnarReader::open(buffer).unwrap();
columnar_reader
ColumnarReader::open(buffer).unwrap()
}
fn build_columnar(docs: &[Vec<(&'static str, ColumnValue)>]) -> ColumnarReader {
@@ -746,7 +746,7 @@ proptest! {
let stack_merge_order = StackMergeOrder::stack(&columnar_readers_arr[..]).into();
crate::merge_columnar(&columnar_readers_arr[..], &[], stack_merge_order, &mut output).unwrap();
let merged_columnar = ColumnarReader::open(output).unwrap();
let concat_rows: Vec<Vec<(&'static str, ColumnValue)>> = columnar_docs.iter().cloned().flatten().collect();
let concat_rows: Vec<Vec<(&'static str, ColumnValue)>> = columnar_docs.iter().flatten().cloned().collect();
let expected_merged_columnar = build_columnar(&concat_rows[..]);
assert_columnar_eq_strict(&merged_columnar, &expected_merged_columnar);
}
@@ -772,7 +772,7 @@ fn test_columnar_merging_empty_columnar() {
.unwrap();
let merged_columnar = ColumnarReader::open(output).unwrap();
let concat_rows: Vec<Vec<(&'static str, ColumnValue)>> =
columnar_docs.iter().cloned().flatten().collect();
columnar_docs.iter().flatten().cloned().collect();
let expected_merged_columnar = build_columnar(&concat_rows[..]);
assert_columnar_eq_strict(&merged_columnar, &expected_merged_columnar);
}
@@ -809,7 +809,7 @@ fn test_columnar_merging_number_columns() {
.unwrap();
let merged_columnar = ColumnarReader::open(output).unwrap();
let concat_rows: Vec<Vec<(&'static str, ColumnValue)>> =
columnar_docs.iter().cloned().flatten().collect();
columnar_docs.iter().flatten().cloned().collect();
let expected_merged_columnar = build_columnar(&concat_rows[..]);
assert_columnar_eq_strict(&merged_columnar, &expected_merged_columnar);
}

View File

@@ -0,0 +1,112 @@
use crate::replace_in_place;
/// Separates the different segments of a json path.
pub const JSON_PATH_SEGMENT_SEP: u8 = 1u8;
pub const JSON_PATH_SEGMENT_SEP_STR: &str =
unsafe { std::str::from_utf8_unchecked(&[JSON_PATH_SEGMENT_SEP]) };
/// Create a new JsonPathWriter, that creates flattened json paths for tantivy.
#[derive(Clone, Debug, Default)]
pub struct JsonPathWriter {
path: String,
indices: Vec<usize>,
expand_dots: bool,
}
impl JsonPathWriter {
pub fn new() -> Self {
JsonPathWriter {
path: String::new(),
indices: Vec::new(),
expand_dots: false,
}
}
/// When expand_dots is enabled, json object like
/// `{"k8s.node.id": 5}` is processed as if it was
/// `{"k8s": {"node": {"id": 5}}}`.
/// This option has the merit of allowing users to
/// write queries like `k8s.node.id:5`.
/// On the other, enabling that feature can lead to
/// ambiguity.
#[inline]
pub fn set_expand_dots(&mut self, expand_dots: bool) {
self.expand_dots = expand_dots;
}
/// Push a new segment to the path.
#[inline]
pub fn push(&mut self, segment: &str) {
let len_path = self.path.len();
self.indices.push(len_path);
if !self.path.is_empty() {
self.path.push_str(JSON_PATH_SEGMENT_SEP_STR);
}
self.path.push_str(segment);
if self.expand_dots {
// This might include the separation byte, which is ok because it is not a dot.
let appended_segment = &mut self.path[len_path..];
// The unsafe below is safe as long as b'.' and JSON_PATH_SEGMENT_SEP are
// valid single byte ut8 strings.
// By utf-8 design, they cannot be part of another codepoint.
unsafe {
replace_in_place(b'.', JSON_PATH_SEGMENT_SEP, appended_segment.as_bytes_mut())
};
}
}
/// Remove the last segment. Does nothing if the path is empty.
#[inline]
pub fn pop(&mut self) {
if let Some(last_idx) = self.indices.pop() {
self.path.truncate(last_idx);
}
}
/// Clear the path.
#[inline]
pub fn clear(&mut self) {
self.path.clear();
self.indices.clear();
}
/// Get the current path.
#[inline]
pub fn as_str(&self) -> &str {
&self.path
}
}
impl From<JsonPathWriter> for String {
#[inline]
fn from(value: JsonPathWriter) -> Self {
value.path
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn json_path_writer_test() {
let mut writer = JsonPathWriter::new();
writer.push("root");
assert_eq!(writer.as_str(), "root");
writer.push("child");
assert_eq!(writer.as_str(), "root\u{1}child");
writer.pop();
assert_eq!(writer.as_str(), "root");
writer.push("k8s.node.id");
assert_eq!(writer.as_str(), "root\u{1}k8s.node.id");
writer.set_expand_dots(true);
writer.pop();
writer.push("k8s.node.id");
assert_eq!(writer.as_str(), "root\u{1}k8s\u{1}node\u{1}id");
}
}

View File

@@ -9,6 +9,7 @@ mod byte_count;
mod datetime;
pub mod file_slice;
mod group_by;
mod json_path_writer;
mod serialize;
mod vint;
mod writer;
@@ -18,6 +19,7 @@ pub use byte_count::ByteCount;
pub use datetime::DatePrecision;
pub use datetime::{DateTime, DateTimePrecision};
pub use group_by::GroupByIteratorExtended;
pub use json_path_writer::JsonPathWriter;
pub use ownedbytes::{OwnedBytes, StableDeref};
pub use serialize::{BinarySerializable, DeserializeFrom, FixedSize};
pub use vint::{
@@ -116,6 +118,7 @@ pub fn u64_to_f64(val: u64) -> f64 {
///
/// This function assumes that the needle is rarely contained in the bytes string
/// and offers a fast path if the needle is not present.
#[inline]
pub fn replace_in_place(needle: u8, replacement: u8, bytes: &mut [u8]) {
if !bytes.contains(&needle) {
return;

View File

@@ -164,7 +164,7 @@ fn main() -> tantivy::Result<()> {
// will reload the index automatically after each commit.
let reader = index
.reader_builder()
.reload_policy(ReloadPolicy::OnCommit)
.reload_policy(ReloadPolicy::OnCommitWithDelay)
.try_into()?;
// We now need to acquire a searcher.

View File

@@ -123,7 +123,7 @@ fn main() -> tantivy::Result<()> {
// will reload the index automatically after each commit.
let reader = index
.reader_builder()
.reload_policy(ReloadPolicy::OnCommit)
.reload_policy(ReloadPolicy::OnCommitWithDelay)
.try_into()?;
// We now need to acquire a searcher.

View File

@@ -51,7 +51,7 @@ fn main() -> Result<()> {
let reader = index
.reader_builder()
.reload_policy(ReloadPolicy::OnCommit)
.reload_policy(ReloadPolicy::OnCommitWithDelay)
.try_into()?;
let searcher = reader.searcher();

View File

@@ -94,7 +94,7 @@ fn main() -> tantivy::Result<()> {
let reader = index
.reader_builder()
.reload_policy(ReloadPolicy::OnCommit)
.reload_policy(ReloadPolicy::OnCommitWithDelay)
.try_into()?;
let searcher = reader.searcher();

View File

@@ -10,7 +10,8 @@
use tantivy::collector::TopDocs;
use tantivy::query::QueryParser;
use tantivy::schema::*;
use tantivy::{doc, Index, IndexWriter, Snippet, SnippetGenerator};
use tantivy::snippet::{Snippet, SnippetGenerator};
use tantivy::{doc, Index, IndexWriter};
use tempfile::TempDir;
fn main() -> tantivy::Result<()> {

View File

@@ -1,5 +1,5 @@
use std::convert::TryInto;
use std::ops::{Deref, Range};
use std::ops::Deref;
use std::sync::Arc;
use std::{fmt, io};
@@ -37,7 +37,7 @@ impl OwnedBytes {
/// creates a fileslice that is just a view over a slice of the data.
#[must_use]
#[inline]
pub fn slice(&self, range: Range<usize>) -> Self {
pub fn slice(&self, range: impl std::slice::SliceIndex<[u8], Output = [u8]>) -> Self {
OwnedBytes {
data: &self.data[range],
box_stable_deref: self.box_stable_deref.clone(),

View File

@@ -185,7 +185,7 @@ fn term_or_phrase(inp: &str) -> IResult<&str, UserInputLeaf> {
fn term_or_phrase_infallible(inp: &str) -> JResult<&str, Option<UserInputLeaf>> {
map(
// ~* for slop/prefix, ) inside group or ast tree, ^ if boost
tuple_infallible((simple_term_infallible("*)^"), slop_or_prefix_val)),
tuple_infallible((simple_term_infallible(")^"), slop_or_prefix_val)),
|((delimiter_phrase, (slop, prefix)), errors)| {
let leaf = if let Some((delimiter, phrase)) = delimiter_phrase {
Some(
@@ -1113,6 +1113,9 @@ mod test {
test_parse_query_to_ast_helper("'www-form-encoded'", "'www-form-encoded'");
test_parse_query_to_ast_helper("www-form-encoded", "www-form-encoded");
test_parse_query_to_ast_helper("www-form-encoded", "www-form-encoded");
test_parse_query_to_ast_helper("mr james bo?d", "(*mr *james *bo?d)");
test_parse_query_to_ast_helper("mr james bo*", "(*mr *james *bo*)");
test_parse_query_to_ast_helper("mr james b*d", "(*mr *james *b*d)");
}
#[test]

View File

@@ -48,7 +48,7 @@ mod bench {
let score_field_f64 = schema_builder.add_f64_field("score_f64", score_fieldtype.clone());
let score_field_i64 = schema_builder.add_i64_field("score_i64", score_fieldtype);
let index = Index::create_from_tempdir(schema_builder.build())?;
let few_terms_data = vec!["INFO", "ERROR", "WARN", "DEBUG"];
let few_terms_data = ["INFO", "ERROR", "WARN", "DEBUG"];
let lg_norm = rand_distr::LogNormal::new(2.996f64, 0.979f64).unwrap();
@@ -85,7 +85,7 @@ mod bench {
if cardinality == Cardinality::Sparse {
doc_with_value /= 20;
}
let val_max = 1_000_000.0;
let _val_max = 1_000_000.0;
for _ in 0..doc_with_value {
let val: f64 = rng.gen_range(0.0..1_000_000.0);
let json = if rng.gen_bool(0.1) {

View File

@@ -73,9 +73,9 @@ impl AggregationLimits {
/// Create a new ResourceLimitGuard, that will release the memory when dropped.
pub fn new_guard(&self) -> ResourceLimitGuard {
ResourceLimitGuard {
/// The counter which is shared between the aggregations for one request.
// The counter which is shared between the aggregations for one request.
memory_consumption: Arc::clone(&self.memory_consumption),
/// The memory_limit in bytes
// The memory_limit in bytes
memory_limit: self.memory_limit,
allocated_with_the_guard: 0,
}

View File

@@ -18,11 +18,11 @@ use crate::directory::{Directory, ManagedDirectory, RamDirectory, INDEX_WRITER_L
use crate::error::{DataCorruption, TantivyError};
use crate::indexer::index_writer::{MAX_NUM_THREAD, MEMORY_BUDGET_NUM_BYTES_MIN};
use crate::indexer::segment_updater::save_metas;
use crate::indexer::IndexWriter;
use crate::reader::{IndexReader, IndexReaderBuilder};
use crate::schema::document::Document;
use crate::schema::{Field, FieldType, Schema};
use crate::tokenizer::{TextAnalyzer, TokenizerManager};
use crate::IndexWriter;
fn load_metas(
directory: &dyn Directory,
@@ -566,7 +566,7 @@ impl Index {
/// Using a single thread gives us a deterministic allocation of DocId.
#[cfg(test)]
pub fn writer_for_tests<D: Document>(&self) -> crate::Result<IndexWriter<D>> {
self.writer_with_num_threads(1, 15_000_000)
self.writer_with_num_threads(1, MEMORY_BUDGET_NUM_BYTES_MIN)
}
/// Creates a multithreaded writer

View File

@@ -1,12 +1,11 @@
use columnar::MonotonicallyMappableToU64;
use common::replace_in_place;
use murmurhash32::murmurhash2;
use common::{replace_in_place, JsonPathWriter};
use rustc_hash::FxHashMap;
use crate::fastfield::FastValue;
use crate::postings::{IndexingContext, IndexingPosition, PostingsWriter};
use crate::schema::document::{ReferenceValue, Value};
use crate::schema::term::{JSON_PATH_SEGMENT_SEP, JSON_PATH_SEGMENT_SEP_STR};
use crate::schema::document::{ReferenceValue, ReferenceValueLeaf, Value};
use crate::schema::term::JSON_PATH_SEGMENT_SEP;
use crate::schema::{Field, Type, DATE_TIME_PRECISION_INDEXED};
use crate::time::format_description::well_known::Rfc3339;
use crate::time::{OffsetDateTime, UtcOffset};
@@ -58,13 +57,12 @@ struct IndexingPositionsPerPath {
}
impl IndexingPositionsPerPath {
fn get_position(&mut self, term: &Term) -> &mut IndexingPosition {
self.positions_per_path
.entry(murmurhash2(term.serialized_term()))
.or_default()
fn get_position_from_id(&mut self, id: u32) -> &mut IndexingPosition {
self.positions_per_path.entry(id).or_default()
}
}
#[allow(clippy::too_many_arguments)]
pub(crate) fn index_json_values<'a, V: Value<'a>>(
doc: DocId,
json_visitors: impl Iterator<Item = crate::Result<V::ObjectIter>>,
@@ -72,9 +70,11 @@ pub(crate) fn index_json_values<'a, V: Value<'a>>(
expand_dots_enabled: bool,
term_buffer: &mut Term,
postings_writer: &mut dyn PostingsWriter,
json_path_writer: &mut JsonPathWriter,
ctx: &mut IndexingContext,
) -> crate::Result<()> {
let mut json_term_writer = JsonTermWriter::wrap(term_buffer, expand_dots_enabled);
json_path_writer.clear();
json_path_writer.set_expand_dots(expand_dots_enabled);
let mut positions_per_path: IndexingPositionsPerPath = Default::default();
for json_visitor_res in json_visitors {
let json_visitor = json_visitor_res?;
@@ -82,7 +82,8 @@ pub(crate) fn index_json_values<'a, V: Value<'a>>(
doc,
json_visitor,
text_analyzer,
&mut json_term_writer,
term_buffer,
json_path_writer,
postings_writer,
ctx,
&mut positions_per_path,
@@ -91,94 +92,141 @@ pub(crate) fn index_json_values<'a, V: Value<'a>>(
Ok(())
}
#[allow(clippy::too_many_arguments)]
fn index_json_object<'a, V: Value<'a>>(
doc: DocId,
json_visitor: V::ObjectIter,
text_analyzer: &mut TextAnalyzer,
json_term_writer: &mut JsonTermWriter,
term_buffer: &mut Term,
json_path_writer: &mut JsonPathWriter,
postings_writer: &mut dyn PostingsWriter,
ctx: &mut IndexingContext,
positions_per_path: &mut IndexingPositionsPerPath,
) {
for (json_path_segment, json_value_visitor) in json_visitor {
json_term_writer.push_path_segment(json_path_segment);
json_path_writer.push(json_path_segment);
index_json_value(
doc,
json_value_visitor,
text_analyzer,
json_term_writer,
term_buffer,
json_path_writer,
postings_writer,
ctx,
positions_per_path,
);
json_term_writer.pop_path_segment();
json_path_writer.pop();
}
}
#[allow(clippy::too_many_arguments)]
fn index_json_value<'a, V: Value<'a>>(
doc: DocId,
json_value: V,
text_analyzer: &mut TextAnalyzer,
json_term_writer: &mut JsonTermWriter,
term_buffer: &mut Term,
json_path_writer: &mut JsonPathWriter,
postings_writer: &mut dyn PostingsWriter,
ctx: &mut IndexingContext,
positions_per_path: &mut IndexingPositionsPerPath,
) {
match json_value.as_value() {
ReferenceValue::Null => {}
ReferenceValue::Str(val) => {
let mut token_stream = text_analyzer.token_stream(val);
let set_path_id = |term_buffer: &mut Term, unordered_id: u32| {
term_buffer.truncate_value_bytes(0);
term_buffer.append_bytes(&unordered_id.to_be_bytes());
};
let set_type = |term_buffer: &mut Term, typ: Type| {
term_buffer.append_bytes(&[typ.to_code()]);
};
// TODO: make sure the chain position works out.
json_term_writer.close_path_and_set_type(Type::Str);
let indexing_position = positions_per_path.get_position(json_term_writer.term());
postings_writer.index_text(
doc,
&mut *token_stream,
json_term_writer.term_buffer,
ctx,
indexing_position,
);
}
ReferenceValue::U64(val) => {
json_term_writer.set_fast_value(val);
postings_writer.subscribe(doc, 0u32, json_term_writer.term(), ctx);
}
ReferenceValue::I64(val) => {
json_term_writer.set_fast_value(val);
postings_writer.subscribe(doc, 0u32, json_term_writer.term(), ctx);
}
ReferenceValue::F64(val) => {
json_term_writer.set_fast_value(val);
postings_writer.subscribe(doc, 0u32, json_term_writer.term(), ctx);
}
ReferenceValue::Bool(val) => {
json_term_writer.set_fast_value(val);
postings_writer.subscribe(doc, 0u32, json_term_writer.term(), ctx);
}
ReferenceValue::Facet(_) => {
unimplemented!("Facet support in dynamic fields is not yet implemented")
}
ReferenceValue::IpAddr(_) => {
unimplemented!("IP address support in dynamic fields is not yet implemented")
}
ReferenceValue::Date(val) => {
json_term_writer.set_fast_value(val);
postings_writer.subscribe(doc, 0u32, json_term_writer.term(), ctx);
}
ReferenceValue::PreTokStr(_) => {
unimplemented!("Pre-tokenized string support in dynamic fields is not yet implemented")
}
ReferenceValue::Bytes(_) => {
unimplemented!("Bytes support in dynamic fields is not yet implemented")
}
match json_value.as_value() {
ReferenceValue::Leaf(leaf) => match leaf {
ReferenceValueLeaf::Null => {}
ReferenceValueLeaf::Str(val) => {
let mut token_stream = text_analyzer.token_stream(val);
let unordered_id = ctx
.path_to_unordered_id
.get_or_allocate_unordered_id(json_path_writer.as_str());
// TODO: make sure the chain position works out.
set_path_id(term_buffer, unordered_id);
set_type(term_buffer, Type::Str);
let indexing_position = positions_per_path.get_position_from_id(unordered_id);
postings_writer.index_text(
doc,
&mut *token_stream,
term_buffer,
ctx,
indexing_position,
);
}
ReferenceValueLeaf::U64(val) => {
set_path_id(
term_buffer,
ctx.path_to_unordered_id
.get_or_allocate_unordered_id(json_path_writer.as_str()),
);
term_buffer.append_type_and_fast_value(val);
postings_writer.subscribe(doc, 0u32, term_buffer, ctx);
}
ReferenceValueLeaf::I64(val) => {
set_path_id(
term_buffer,
ctx.path_to_unordered_id
.get_or_allocate_unordered_id(json_path_writer.as_str()),
);
term_buffer.append_type_and_fast_value(val);
postings_writer.subscribe(doc, 0u32, term_buffer, ctx);
}
ReferenceValueLeaf::F64(val) => {
set_path_id(
term_buffer,
ctx.path_to_unordered_id
.get_or_allocate_unordered_id(json_path_writer.as_str()),
);
term_buffer.append_type_and_fast_value(val);
postings_writer.subscribe(doc, 0u32, term_buffer, ctx);
}
ReferenceValueLeaf::Bool(val) => {
set_path_id(
term_buffer,
ctx.path_to_unordered_id
.get_or_allocate_unordered_id(json_path_writer.as_str()),
);
term_buffer.append_type_and_fast_value(val);
postings_writer.subscribe(doc, 0u32, term_buffer, ctx);
}
ReferenceValueLeaf::Date(val) => {
set_path_id(
term_buffer,
ctx.path_to_unordered_id
.get_or_allocate_unordered_id(json_path_writer.as_str()),
);
term_buffer.append_type_and_fast_value(val);
postings_writer.subscribe(doc, 0u32, term_buffer, ctx);
}
ReferenceValueLeaf::PreTokStr(_) => {
unimplemented!(
"Pre-tokenized string support in dynamic fields is not yet implemented"
)
}
ReferenceValueLeaf::Bytes(_) => {
unimplemented!("Bytes support in dynamic fields is not yet implemented")
}
ReferenceValueLeaf::Facet(_) => {
unimplemented!("Facet support in dynamic fields is not yet implemented")
}
ReferenceValueLeaf::IpAddr(_) => {
unimplemented!("IP address support in dynamic fields is not yet implemented")
}
},
ReferenceValue::Array(elements) => {
for val in elements {
index_json_value(
doc,
val,
text_analyzer,
json_term_writer,
term_buffer,
json_path_writer,
postings_writer,
ctx,
positions_per_path,
@@ -190,7 +238,8 @@ fn index_json_value<'a, V: Value<'a>>(
doc,
object,
text_analyzer,
json_term_writer,
term_buffer,
json_path_writer,
postings_writer,
ctx,
positions_per_path,
@@ -311,17 +360,13 @@ pub(crate) fn encode_column_name(
json_path: &str,
expand_dots_enabled: bool,
) -> String {
let mut column_key: String = String::with_capacity(field_name.len() + json_path.len() + 1);
column_key.push_str(field_name);
for mut segment in split_json_path(json_path) {
column_key.push_str(JSON_PATH_SEGMENT_SEP_STR);
if expand_dots_enabled {
// We need to replace `.` by JSON_PATH_SEGMENT_SEP.
unsafe { replace_in_place(b'.', JSON_PATH_SEGMENT_SEP, segment.as_bytes_mut()) };
}
column_key.push_str(&segment);
let mut path = JsonPathWriter::default();
path.push(field_name);
path.set_expand_dots(expand_dots_enabled);
for segment in split_json_path(json_path) {
path.push(&segment);
}
column_key
path.into()
}
impl<'a> JsonTermWriter<'a> {
@@ -361,6 +406,7 @@ impl<'a> JsonTermWriter<'a> {
self.term_buffer.append_bytes(&[typ.to_code()]);
}
// TODO: Remove this function and use JsonPathWriter instead.
pub fn push_path_segment(&mut self, segment: &str) {
// the path stack should never be empty.
self.trim_to_end_of_path();

View File

@@ -5,7 +5,7 @@ use std::{fmt, io};
use crate::collector::Collector;
use crate::core::{Executor, SegmentReader};
use crate::query::{Bm25StatisticsProvider, EnableScoring, Query};
use crate::schema::document::{Document, DocumentDeserialize};
use crate::schema::document::DocumentDeserialize;
use crate::schema::{Schema, Term};
use crate::space_usage::SearcherSpaceUsage;
use crate::store::{CacheStats, StoreReader};

View File

@@ -121,7 +121,7 @@ fn test_index_on_commit_reload_policy() -> crate::Result<()> {
let index = Index::create_in_ram(schema);
let reader = index
.reader_builder()
.reload_policy(ReloadPolicy::OnCommit)
.reload_policy(ReloadPolicy::OnCommitWithDelay)
.try_into()
.unwrap();
assert_eq!(reader.searcher().num_docs(), 0);
@@ -147,7 +147,7 @@ mod mmap_specific {
let index = Index::create_in_dir(tempdir_path, schema).unwrap();
let reader = index
.reader_builder()
.reload_policy(ReloadPolicy::OnCommit)
.reload_policy(ReloadPolicy::OnCommitWithDelay)
.try_into()
.unwrap();
assert_eq!(reader.searcher().num_docs(), 0);
@@ -189,7 +189,7 @@ mod mmap_specific {
let read_index = Index::open_in_dir(&tempdir_path).unwrap();
let reader = read_index
.reader_builder()
.reload_policy(ReloadPolicy::OnCommit)
.reload_policy(ReloadPolicy::OnCommitWithDelay)
.try_into()
.unwrap();
assert_eq!(reader.searcher().num_docs(), 0);

View File

@@ -222,8 +222,8 @@ pub trait Directory: DirectoryClone + fmt::Debug + Send + Sync + 'static {
/// registered (and whose [`WatchHandle`] is still alive) are triggered.
///
/// Internally, tantivy only uses this API to detect new commits to implement the
/// `OnCommit` `ReloadPolicy`. Not implementing watch in a `Directory` only prevents the
/// `OnCommit` `ReloadPolicy` to work properly.
/// `OnCommitWithDelay` `ReloadPolicy`. Not implementing watch in a `Directory` only prevents
/// the `OnCommitWithDelay` `ReloadPolicy` to work properly.
fn watch(&self, watch_callback: WatchCallback) -> crate::Result<WatchHandle>;
}

View File

@@ -7,7 +7,7 @@ use serde::{Deserialize, Serialize};
use crate::directory::error::Incompatibility;
use crate::directory::{AntiCallToken, FileSlice, TerminatingWrite};
use crate::{Version, INDEX_FORMAT_VERSION};
use crate::{Version, INDEX_FORMAT_OLDEST_SUPPORTED_VERSION, INDEX_FORMAT_VERSION};
const FOOTER_MAX_LEN: u32 = 50_000;
@@ -102,10 +102,11 @@ impl Footer {
/// Confirms that the index will be read correctly by this version of tantivy
/// Has to be called after `extract_footer` to make sure it's not accessing uninitialised memory
pub fn is_compatible(&self) -> Result<(), Incompatibility> {
const SUPPORTED_INDEX_FORMAT_VERSION_RANGE: std::ops::RangeInclusive<u32> =
INDEX_FORMAT_OLDEST_SUPPORTED_VERSION..=INDEX_FORMAT_VERSION;
let library_version = crate::version();
if self.version.index_format_version < 4
|| self.version.index_format_version > INDEX_FORMAT_VERSION
{
if !SUPPORTED_INDEX_FORMAT_VERSION_RANGE.contains(&self.version.index_format_version) {
return Err(Incompatibility::IndexMismatch {
library_version: library_version.clone(),
index_version: self.version.clone(),

View File

@@ -8,6 +8,8 @@ use std::sync::{Arc, RwLock, Weak};
use common::StableDeref;
use fs4::FileExt;
#[cfg(all(feature = "mmap", unix))]
pub use memmap2::Advice;
use memmap2::Mmap;
use serde::{Deserialize, Serialize};
use tempfile::TempDir;
@@ -21,8 +23,6 @@ use crate::directory::{
AntiCallToken, Directory, DirectoryLock, FileHandle, Lock, OwnedBytes, TerminatingWrite,
WatchCallback, WatchHandle, WritePtr,
};
#[cfg(unix)]
use crate::Advice;
pub type ArcBytes = Arc<dyn Deref<Target = [u8]> + Send + Sync + 'static>;
pub type WeakArcBytes = Weak<dyn Deref<Target = [u8]> + Send + Sync + 'static>;

View File

@@ -42,6 +42,9 @@ pub struct GarbageCollectionResult {
pub failed_to_delete_files: Vec<PathBuf>,
}
#[cfg(all(feature = "mmap", unix))]
pub use memmap2::Advice;
pub use self::managed_directory::ManagedDirectory;
#[cfg(feature = "mmap")]
pub use self::mmap_directory::MmapDirectory;

View File

@@ -17,7 +17,7 @@ pub trait DocSet: Send {
///
/// The DocId of the next element is returned.
/// In other words we should always have :
/// ```ignore
/// ```compile_fail
/// let doc = docset.advance();
/// assert_eq!(doc, docset.doc());
/// ```

View File

@@ -131,7 +131,7 @@ mod tests {
}
let file = directory.open_read(path).unwrap();
assert_eq!(file.len(), 93);
assert_eq!(file.len(), 105);
let fast_field_readers = FastFieldReaders::open(file, SCHEMA.clone()).unwrap();
let column = fast_field_readers
.u64("field")
@@ -181,7 +181,7 @@ mod tests {
write.terminate().unwrap();
}
let file = directory.open_read(path).unwrap();
assert_eq!(file.len(), 121);
assert_eq!(file.len(), 133);
let fast_field_readers = FastFieldReaders::open(file, SCHEMA.clone()).unwrap();
let col = fast_field_readers
.u64("field")
@@ -214,7 +214,7 @@ mod tests {
write.terminate().unwrap();
}
let file = directory.open_read(path).unwrap();
assert_eq!(file.len(), 94);
assert_eq!(file.len(), 106);
let fast_field_readers = FastFieldReaders::open(file, SCHEMA.clone()).unwrap();
let fast_field_reader = fast_field_readers
.u64("field")
@@ -246,7 +246,7 @@ mod tests {
write.terminate().unwrap();
}
let file = directory.open_read(path).unwrap();
assert_eq!(file.len(), 4489);
assert_eq!(file.len(), 4501);
{
let fast_field_readers = FastFieldReaders::open(file, SCHEMA.clone()).unwrap();
let col = fast_field_readers
@@ -279,7 +279,7 @@ mod tests {
write.terminate().unwrap();
}
let file = directory.open_read(path).unwrap();
assert_eq!(file.len(), 265);
assert_eq!(file.len(), 277);
{
let fast_field_readers = FastFieldReaders::open(file, schema).unwrap();
@@ -773,7 +773,7 @@ mod tests {
write.terminate().unwrap();
}
let file = directory.open_read(path).unwrap();
assert_eq!(file.len(), 102);
assert_eq!(file.len(), 114);
let fast_field_readers = FastFieldReaders::open(file, schema).unwrap();
let bool_col = fast_field_readers.bool("field_bool").unwrap();
assert_eq!(bool_col.first(0), Some(true));
@@ -805,7 +805,7 @@ mod tests {
write.terminate().unwrap();
}
let file = directory.open_read(path).unwrap();
assert_eq!(file.len(), 114);
assert_eq!(file.len(), 126);
let readers = FastFieldReaders::open(file, schema).unwrap();
let bool_col = readers.bool("field_bool").unwrap();
for i in 0..25 {
@@ -830,7 +830,7 @@ mod tests {
write.terminate().unwrap();
}
let file = directory.open_read(path).unwrap();
assert_eq!(file.len(), 104);
assert_eq!(file.len(), 116);
let fastfield_readers = FastFieldReaders::open(file, schema).unwrap();
let col = fastfield_readers.bool("field_bool").unwrap();
assert_eq!(col.first(0), None);

View File

@@ -1,12 +1,11 @@
use std::io;
use columnar::{ColumnarWriter, NumericalValue};
use common::replace_in_place;
use common::JsonPathWriter;
use tokenizer_api::Token;
use crate::indexer::doc_id_mapping::DocIdMapping;
use crate::schema::document::{Document, ReferenceValue, Value};
use crate::schema::term::{JSON_PATH_SEGMENT_SEP, JSON_PATH_SEGMENT_SEP_STR};
use crate::schema::document::{Document, ReferenceValue, ReferenceValueLeaf, Value};
use crate::schema::{value_type_to_column_type, Field, FieldType, Schema, Type};
use crate::tokenizer::{TextAnalyzer, TokenizerManager};
use crate::{DateTimePrecision, DocId, TantivyError};
@@ -24,7 +23,7 @@ pub struct FastFieldsWriter {
expand_dots: Vec<bool>,
num_docs: DocId,
// Buffer that we recycle to avoid allocation.
json_path_buffer: String,
json_path_buffer: JsonPathWriter,
}
impl FastFieldsWriter {
@@ -98,7 +97,7 @@ impl FastFieldsWriter {
num_docs: 0u32,
date_precisions,
expand_dots,
json_path_buffer: String::new(),
json_path_buffer: JsonPathWriter::default(),
})
}
@@ -141,64 +140,68 @@ impl FastFieldsWriter {
};
match value.as_value() {
ReferenceValue::Null => {}
ReferenceValue::Str(val) => {
if let Some(tokenizer) = &mut self.per_field_tokenizer[field.field_id() as usize] {
let mut token_stream = tokenizer.token_stream(val);
token_stream.process(&mut |token: &Token| {
ReferenceValue::Leaf(leaf) => match leaf {
ReferenceValueLeaf::Null => {}
ReferenceValueLeaf::Str(val) => {
if let Some(tokenizer) =
&mut self.per_field_tokenizer[field.field_id() as usize]
{
let mut token_stream = tokenizer.token_stream(val);
token_stream.process(&mut |token: &Token| {
self.columnar_writer
.record_str(doc_id, field_name, &token.text);
})
} else {
self.columnar_writer.record_str(doc_id, field_name, val);
}
}
ReferenceValueLeaf::U64(val) => {
self.columnar_writer.record_numerical(
doc_id,
field_name,
NumericalValue::from(val),
);
}
ReferenceValueLeaf::I64(val) => {
self.columnar_writer.record_numerical(
doc_id,
field_name,
NumericalValue::from(val),
);
}
ReferenceValueLeaf::F64(val) => {
self.columnar_writer.record_numerical(
doc_id,
field_name,
NumericalValue::from(val),
);
}
ReferenceValueLeaf::Date(val) => {
let date_precision = self.date_precisions[field.field_id() as usize];
let truncated_datetime = val.truncate(date_precision);
self.columnar_writer
.record_datetime(doc_id, field_name, truncated_datetime);
}
ReferenceValueLeaf::Facet(val) => {
self.columnar_writer
.record_str(doc_id, field_name, val.encoded_str());
}
ReferenceValueLeaf::Bytes(val) => {
self.columnar_writer.record_bytes(doc_id, field_name, val);
}
ReferenceValueLeaf::IpAddr(val) => {
self.columnar_writer.record_ip_addr(doc_id, field_name, val);
}
ReferenceValueLeaf::Bool(val) => {
self.columnar_writer.record_bool(doc_id, field_name, val);
}
ReferenceValueLeaf::PreTokStr(val) => {
for token in &val.tokens {
self.columnar_writer
.record_str(doc_id, field_name, &token.text);
})
} else {
self.columnar_writer.record_str(doc_id, field_name, val);
}
}
}
ReferenceValue::U64(val) => {
self.columnar_writer.record_numerical(
doc_id,
field_name,
NumericalValue::from(val),
);
}
ReferenceValue::I64(val) => {
self.columnar_writer.record_numerical(
doc_id,
field_name,
NumericalValue::from(val),
);
}
ReferenceValue::F64(val) => {
self.columnar_writer.record_numerical(
doc_id,
field_name,
NumericalValue::from(val),
);
}
ReferenceValue::Date(val) => {
let date_precision = self.date_precisions[field.field_id() as usize];
let truncated_datetime = val.truncate(date_precision);
self.columnar_writer
.record_datetime(doc_id, field_name, truncated_datetime);
}
ReferenceValue::Facet(val) => {
self.columnar_writer
.record_str(doc_id, field_name, val.encoded_str());
}
ReferenceValue::Bytes(val) => {
self.columnar_writer.record_bytes(doc_id, field_name, val);
}
ReferenceValue::IpAddr(val) => {
self.columnar_writer.record_ip_addr(doc_id, field_name, val);
}
ReferenceValue::Bool(val) => {
self.columnar_writer.record_bool(doc_id, field_name, val);
}
ReferenceValue::PreTokStr(val) => {
for token in &val.tokens {
self.columnar_writer
.record_str(doc_id, field_name, &token.text);
}
}
},
ReferenceValue::Array(val) => {
// TODO: Check this is the correct behaviour we want.
for value in val {
@@ -208,14 +211,16 @@ impl FastFieldsWriter {
ReferenceValue::Object(val) => {
let expand_dots = self.expand_dots[field.field_id() as usize];
self.json_path_buffer.clear();
self.json_path_buffer.push_str(field_name);
// First field should not be expanded.
self.json_path_buffer.set_expand_dots(false);
self.json_path_buffer.push(field_name);
self.json_path_buffer.set_expand_dots(expand_dots);
let text_analyzer = &mut self.per_field_tokenizer[field.field_id() as usize];
record_json_obj_to_columnar_writer::<V>(
doc_id,
val,
expand_dots,
JSON_DEPTH_LIMIT,
&mut self.json_path_buffer,
&mut self.columnar_writer,
@@ -246,48 +251,30 @@ impl FastFieldsWriter {
fn record_json_obj_to_columnar_writer<'a, V: Value<'a>>(
doc: DocId,
json_visitor: V::ObjectIter,
expand_dots: bool,
remaining_depth_limit: usize,
json_path_buffer: &mut String,
json_path_buffer: &mut JsonPathWriter,
columnar_writer: &mut columnar::ColumnarWriter,
tokenizer: &mut Option<TextAnalyzer>,
) {
for (key, child) in json_visitor {
let len_path = json_path_buffer.len();
if !json_path_buffer.is_empty() {
json_path_buffer.push_str(JSON_PATH_SEGMENT_SEP_STR);
}
json_path_buffer.push_str(key);
if expand_dots {
// This might include the separation byte, which is ok because it is not a dot.
let appended_segment = &mut json_path_buffer[len_path..];
// The unsafe below is safe as long as b'.' and JSON_PATH_SEGMENT_SEP are
// valid single byte ut8 strings.
// By utf-8 design, they cannot be part of another codepoint.
replace_in_place(b'.', JSON_PATH_SEGMENT_SEP, unsafe {
appended_segment.as_bytes_mut()
});
}
json_path_buffer.push(key);
record_json_value_to_columnar_writer(
doc,
child,
expand_dots,
remaining_depth_limit,
json_path_buffer,
columnar_writer,
tokenizer,
);
// popping our sub path.
json_path_buffer.truncate(len_path);
json_path_buffer.pop();
}
}
fn record_json_value_to_columnar_writer<'a, V: Value<'a>>(
doc: DocId,
json_val: V,
expand_dots: bool,
mut remaining_depth_limit: usize,
json_path_writer: &mut String,
json_path_writer: &mut JsonPathWriter,
columnar_writer: &mut columnar::ColumnarWriter,
tokenizer: &mut Option<TextAnalyzer>,
) {
@@ -297,64 +284,67 @@ fn record_json_value_to_columnar_writer<'a, V: Value<'a>>(
remaining_depth_limit -= 1;
match json_val.as_value() {
ReferenceValue::Null => {} // TODO: Handle null
ReferenceValue::Str(val) => {
if let Some(text_analyzer) = tokenizer.as_mut() {
let mut token_stream = text_analyzer.token_stream(val);
token_stream.process(&mut |token| {
columnar_writer.record_str(doc, json_path_writer.as_str(), &token.text);
})
} else {
columnar_writer.record_str(doc, json_path_writer.as_str(), val);
ReferenceValue::Leaf(leaf) => match leaf {
ReferenceValueLeaf::Null => {} // TODO: Handle null
ReferenceValueLeaf::Str(val) => {
if let Some(text_analyzer) = tokenizer.as_mut() {
let mut token_stream = text_analyzer.token_stream(val);
token_stream.process(&mut |token| {
columnar_writer.record_str(doc, json_path_writer.as_str(), &token.text);
})
} else {
columnar_writer.record_str(doc, json_path_writer.as_str(), val);
}
}
}
ReferenceValue::U64(val) => {
columnar_writer.record_numerical(
doc,
json_path_writer.as_str(),
NumericalValue::from(val),
);
}
ReferenceValue::I64(val) => {
columnar_writer.record_numerical(
doc,
json_path_writer.as_str(),
NumericalValue::from(val),
);
}
ReferenceValue::F64(val) => {
columnar_writer.record_numerical(
doc,
json_path_writer.as_str(),
NumericalValue::from(val),
);
}
ReferenceValue::Bool(val) => {
columnar_writer.record_bool(doc, json_path_writer, val);
}
ReferenceValue::Date(val) => {
columnar_writer.record_datetime(doc, json_path_writer.as_str(), val);
}
ReferenceValue::Facet(_) => {
unimplemented!("Facet support in dynamic fields is not yet implemented")
}
ReferenceValue::Bytes(_) => {
// TODO: This can be re added once it is added to the JSON Utils section as well.
// columnar_writer.record_bytes(doc, json_path_writer.as_str(), val);
unimplemented!("Bytes support in dynamic fields is not yet implemented")
}
ReferenceValue::IpAddr(_) => {
unimplemented!("IP address support in dynamic fields is not yet implemented")
}
ReferenceValue::PreTokStr(_) => {
unimplemented!("Pre-tokenized string support in dynamic fields is not yet implemented")
}
ReferenceValueLeaf::U64(val) => {
columnar_writer.record_numerical(
doc,
json_path_writer.as_str(),
NumericalValue::from(val),
);
}
ReferenceValueLeaf::I64(val) => {
columnar_writer.record_numerical(
doc,
json_path_writer.as_str(),
NumericalValue::from(val),
);
}
ReferenceValueLeaf::F64(val) => {
columnar_writer.record_numerical(
doc,
json_path_writer.as_str(),
NumericalValue::from(val),
);
}
ReferenceValueLeaf::Bool(val) => {
columnar_writer.record_bool(doc, json_path_writer.as_str(), val);
}
ReferenceValueLeaf::Date(val) => {
columnar_writer.record_datetime(doc, json_path_writer.as_str(), val);
}
ReferenceValueLeaf::Facet(_) => {
unimplemented!("Facet support in dynamic fields is not yet implemented")
}
ReferenceValueLeaf::Bytes(_) => {
// TODO: This can be re added once it is added to the JSON Utils section as well.
// columnar_writer.record_bytes(doc, json_path_writer.as_str(), val);
unimplemented!("Bytes support in dynamic fields is not yet implemented")
}
ReferenceValueLeaf::IpAddr(_) => {
unimplemented!("IP address support in dynamic fields is not yet implemented")
}
ReferenceValueLeaf::PreTokStr(_) => {
unimplemented!(
"Pre-tokenized string support in dynamic fields is not yet implemented"
)
}
},
ReferenceValue::Array(elements) => {
for el in elements {
record_json_value_to_columnar_writer(
doc,
el,
expand_dots,
remaining_depth_limit,
json_path_writer,
columnar_writer,
@@ -366,7 +356,6 @@ fn record_json_value_to_columnar_writer<'a, V: Value<'a>>(
record_json_obj_to_columnar_writer::<V>(
doc,
object,
expand_dots,
remaining_depth_limit,
json_path_writer,
columnar_writer,
@@ -379,6 +368,7 @@ fn record_json_value_to_columnar_writer<'a, V: Value<'a>>(
#[cfg(test)]
mod tests {
use columnar::{Column, ColumnarReader, ColumnarWriter, StrColumn};
use common::JsonPathWriter;
use super::record_json_value_to_columnar_writer;
use crate::fastfield::writer::JSON_DEPTH_LIMIT;
@@ -389,12 +379,12 @@ mod tests {
expand_dots: bool,
) -> ColumnarReader {
let mut columnar_writer = ColumnarWriter::default();
let mut json_path = String::new();
let mut json_path = JsonPathWriter::default();
json_path.set_expand_dots(expand_dots);
for (doc, json_doc) in json_docs.iter().enumerate() {
record_json_value_to_columnar_writer(
doc as u32,
json_doc,
expand_dots,
JSON_DEPTH_LIMIT,
&mut json_path,
&mut columnar_writer,

View File

@@ -32,7 +32,7 @@ fn test_functional_store() -> crate::Result<()> {
let mut rng = thread_rng();
let mut index_writer: IndexWriter =
index.writer_with_num_threads(3, MEMORY_BUDGET_NUM_BYTES_MIN)?;
index.writer_with_num_threads(3, 3 * MEMORY_BUDGET_NUM_BYTES_MIN)?;
let mut doc_set: Vec<u64> = Vec::new();
@@ -92,7 +92,8 @@ fn test_functional_indexing_sorted() -> crate::Result<()> {
let mut rng = thread_rng();
let mut index_writer: IndexWriter = index.writer_with_num_threads(3, 120_000_000)?;
let mut index_writer: IndexWriter =
index.writer_with_num_threads(3, 3 * MEMORY_BUDGET_NUM_BYTES_MIN)?;
let mut committed_docs: HashSet<u64> = HashSet::new();
let mut uncommitted_docs: HashSet<u64> = HashSet::new();
@@ -167,7 +168,8 @@ fn test_functional_indexing_unsorted() -> crate::Result<()> {
let mut rng = thread_rng();
let mut index_writer: IndexWriter = index.writer_with_num_threads(3, 120_000_000)?;
let mut index_writer: IndexWriter =
index.writer_with_num_threads(3, 3 * MEMORY_BUDGET_NUM_BYTES_MIN)?;
let mut committed_docs: HashSet<u64> = HashSet::new();
let mut uncommitted_docs: HashSet<u64> = HashSet::new();

View File

@@ -1706,7 +1706,8 @@ mod tests {
let old_reader = index.reader()?;
let id_exists = |id| id % 3 != 0; // 0 does not exist
// Every 3rd doc has only id field
let id_is_full_doc = |id| id % 3 != 0;
let multi_text_field_text1 = "test1 test2 test3 test1 test2 test3";
// rotate left
@@ -1722,7 +1723,7 @@ mod tests {
let facet = Facet::from(&("/cola/".to_string() + &id.to_string()));
let ip = ip_from_id(id);
if !id_exists(id) {
if !id_is_full_doc(id) {
// every 3rd doc has no ip field
index_writer.add_document(doc!(
id_field=>id,
@@ -1842,7 +1843,7 @@ mod tests {
let num_docs_with_values = expected_ids_and_num_occurrences
.iter()
.filter(|(id, _id_occurrences)| id_exists(**id))
.filter(|(id, _id_occurrences)| id_is_full_doc(**id))
.map(|(_, id_occurrences)| *id_occurrences as usize)
.sum::<usize>();
@@ -1866,7 +1867,7 @@ mod tests {
if force_end_merge && num_segments_before_merge > 1 && num_segments_after_merge == 1 {
let mut expected_multi_ips: Vec<_> = id_list
.iter()
.filter(|id| id_exists(**id))
.filter(|id| id_is_full_doc(**id))
.flat_map(|id| vec![ip_from_id(*id), ip_from_id(*id)])
.collect();
assert_eq!(num_ips, expected_multi_ips.len() as u32);
@@ -1904,7 +1905,7 @@ mod tests {
let expected_ips = expected_ids_and_num_occurrences
.keys()
.flat_map(|id| {
if !id_exists(*id) {
if !id_is_full_doc(*id) {
None
} else {
Some(Ipv6Addr::from_u128(*id as u128))
@@ -1916,7 +1917,7 @@ mod tests {
let expected_ips = expected_ids_and_num_occurrences
.keys()
.filter_map(|id| {
if !id_exists(*id) {
if !id_is_full_doc(*id) {
None
} else {
Some(Ipv6Addr::from_u128(*id as u128))
@@ -1951,7 +1952,7 @@ mod tests {
let id = id_reader.first(doc).unwrap();
let vals: Vec<u64> = ff_reader.values_for_doc(doc).collect();
if id_exists(id) {
if id_is_full_doc(id) {
assert_eq!(vals.len(), 2);
assert_eq!(vals[0], vals[1]);
assert!(expected_ids_and_num_occurrences.contains_key(&vals[0]));
@@ -1961,7 +1962,7 @@ mod tests {
}
let bool_vals: Vec<bool> = bool_ff_reader.values_for_doc(doc).collect();
if id_exists(id) {
if id_is_full_doc(id) {
assert_eq!(bool_vals.len(), 2);
assert_ne!(bool_vals[0], bool_vals[1]);
} else {
@@ -1990,7 +1991,7 @@ mod tests {
.as_u64()
.unwrap();
assert!(expected_ids_and_num_occurrences.contains_key(&id));
if id_exists(id) {
if id_is_full_doc(id) {
let id2 = store_reader
.get::<TantivyDocument>(doc_id)
.unwrap()
@@ -2037,7 +2038,7 @@ mod tests {
let (existing_id, count) = (*id, *count);
let get_num_hits = |field| do_search(&existing_id.to_string(), field).len() as u64;
assert_eq!(get_num_hits(id_field), count);
if !id_exists(existing_id) {
if !id_is_full_doc(existing_id) {
continue;
}
assert_eq!(get_num_hits(text_field), count);
@@ -2087,7 +2088,7 @@ mod tests {
//
for (existing_id, count) in &expected_ids_and_num_occurrences {
let (existing_id, count) = (*existing_id, *count);
if !id_exists(existing_id) {
if !id_is_full_doc(existing_id) {
continue;
}
let do_search_ip_field = |term: &str| do_search(term, ip_field).len() as u64;
@@ -2104,34 +2105,84 @@ mod tests {
}
}
// assert data is like expected
// Range query
//
for (existing_id, count) in expected_ids_and_num_occurrences.iter().take(10) {
let (existing_id, count) = (*existing_id, *count);
if !id_exists(existing_id) {
continue;
}
let gen_query_inclusive = |field: &str, from: Ipv6Addr, to: Ipv6Addr| {
format!("{}:[{} TO {}]", field, &from.to_string(), &to.to_string())
// Take half as sample
let mut sample: Vec<_> = expected_ids_and_num_occurrences.iter().collect();
sample.sort_by_key(|(k, _num_occurences)| *k);
// sample.truncate(sample.len() / 2);
if !sample.is_empty() {
let (left_sample, right_sample) = sample.split_at(sample.len() / 2);
let expected_count = |sample: &[(&u64, &u64)]| {
sample
.iter()
.filter(|(id, _)| id_is_full_doc(**id))
.map(|(_id, num_occurences)| **num_occurences)
.sum::<u64>()
};
let ip = ip_from_id(existing_id);
fn gen_query_inclusive<T1: ToString, T2: ToString>(
field: &str,
from: T1,
to: T2,
) -> String {
format!("{}:[{} TO {}]", field, &from.to_string(), &to.to_string())
}
let do_search_ip_field = |term: &str| do_search(term, ip_field).len() as u64;
// Range query on single value field
let query = gen_query_inclusive("ip", ip, ip);
assert_eq!(do_search_ip_field(&query), count);
// Query first half
if !left_sample.is_empty() {
let expected_count = expected_count(left_sample);
// Range query on multi value field
let query = gen_query_inclusive("ips", ip, ip);
let start_range = *left_sample[0].0;
let end_range = *left_sample.last().unwrap().0;
let query = gen_query_inclusive("id_opt", start_range, end_range);
assert_eq!(do_search(&query, id_opt_field).len() as u64, expected_count);
assert_eq!(do_search_ip_field(&query), count);
// Range query on ip field
let ip1 = ip_from_id(start_range);
let ip2 = ip_from_id(end_range);
let do_search_ip_field = |term: &str| do_search(term, ip_field).len() as u64;
let query = gen_query_inclusive("ip", ip1, ip2);
assert_eq!(do_search_ip_field(&query), expected_count);
let query = gen_query_inclusive("ip", "*", ip2);
assert_eq!(do_search_ip_field(&query), expected_count);
// Range query on multi value field
let query = gen_query_inclusive("ips", ip1, ip2);
assert_eq!(do_search_ip_field(&query), expected_count);
let query = gen_query_inclusive("ips", "*", ip2);
assert_eq!(do_search_ip_field(&query), expected_count);
}
// Query second half
if !right_sample.is_empty() {
let expected_count = expected_count(right_sample);
let start_range = *right_sample[0].0;
let end_range = *right_sample.last().unwrap().0;
// Range query on id opt field
let query =
gen_query_inclusive("id_opt", start_range.to_string(), end_range.to_string());
assert_eq!(do_search(&query, id_opt_field).len() as u64, expected_count);
// Range query on ip field
let ip1 = ip_from_id(start_range);
let ip2 = ip_from_id(end_range);
let do_search_ip_field = |term: &str| do_search(term, ip_field).len() as u64;
let query = gen_query_inclusive("ip", ip1, ip2);
assert_eq!(do_search_ip_field(&query), expected_count);
let query = gen_query_inclusive("ip", ip1, "*");
assert_eq!(do_search_ip_field(&query), expected_count);
// Range query on multi value field
let query = gen_query_inclusive("ips", ip1, ip2);
assert_eq!(do_search_ip_field(&query), expected_count);
let query = gen_query_inclusive("ips", ip1, "*");
assert_eq!(do_search_ip_field(&query), expected_count);
}
}
// ip range query on fast field
//
for (existing_id, count) in expected_ids_and_num_occurrences.iter().take(10) {
let (existing_id, count) = (*existing_id, *count);
if !id_exists(existing_id) {
if !id_is_full_doc(existing_id) {
continue;
}
let gen_query_inclusive = |field: &str, from: Ipv6Addr, to: Ipv6Addr| {
@@ -2159,7 +2210,7 @@ mod tests {
.first_or_default_col(9999);
for doc_id in segment_reader.doc_ids_alive() {
let id = ff_reader.get_val(doc_id);
if !id_exists(id) {
if !id_is_full_doc(id) {
continue;
}
let facet_ords: Vec<u64> = facet_reader.facet_ords(doc_id).collect();
@@ -2197,6 +2248,12 @@ mod tests {
Ok(index)
}
#[test]
fn test_fast_field_range() {
let ops: Vec<_> = (0..1000).map(|id| IndexingOp::AddDoc { id }).collect();
assert!(test_operation_strategy(&ops, false, true).is_ok());
}
#[test]
fn test_sort_index_on_opt_field_regression() {
assert!(test_operation_strategy(

View File

@@ -63,10 +63,13 @@ impl MergeOperation {
}
}
/// Returns the opstamp up to which we want to consume the delete queue and reflect their
/// deletes.
pub fn target_opstamp(&self) -> Opstamp {
self.inner.target_opstamp
}
/// Returns the list of segment to be merged.
pub fn segment_ids(&self) -> &[SegmentId] {
&self.inner.segment_ids[..]
}

View File

@@ -543,7 +543,7 @@ mod bench_sorted_index_merge {
//);
//(doc_addr.doc_id, reader, u64_reader)
//});
//// add values in order of the new doc_ids
/// add values in order of the new doc_ids
// let mut val = 0;
// for (doc_id, _reader, field_reader) in sorted_doc_ids {
// val = field_reader.get_val(doc_id);

View File

@@ -1,23 +1,30 @@
pub mod delete_queue;
//! Indexing and merging data.
//!
//! Contains code to create and merge segments.
//! `IndexWriter` is the main entry point for that, which created from
//! [`Index::writer`](crate::Index::writer).
pub mod doc_id_mapping;
pub(crate) mod delete_queue;
pub(crate) mod path_to_unordered_id;
pub(crate) mod doc_id_mapping;
mod doc_opstamp_mapping;
mod flat_map_with_buffer;
pub mod index_writer;
mod index_writer_status;
pub(crate) mod index_writer;
pub(crate) mod index_writer_status;
mod log_merge_policy;
mod merge_operation;
pub mod merge_policy;
pub mod merger;
pub(crate) mod merge_policy;
pub(crate) mod merger;
mod merger_sorted_index_test;
pub mod operation;
pub mod prepared_commit;
pub(crate) mod operation;
pub(crate) mod prepared_commit;
mod segment_entry;
mod segment_manager;
mod segment_register;
pub mod segment_serializer;
pub mod segment_updater;
mod segment_writer;
pub(crate) mod segment_serializer;
pub(crate) mod segment_updater;
pub(crate) mod segment_writer;
mod stamper;
use crossbeam_channel as channel;
@@ -27,10 +34,10 @@ pub use self::index_writer::IndexWriter;
pub use self::log_merge_policy::LogMergePolicy;
pub use self::merge_operation::MergeOperation;
pub use self::merge_policy::{MergeCandidate, MergePolicy, NoMergePolicy};
pub use self::operation::UserOperation;
pub use self::prepared_commit::PreparedCommit;
pub use self::segment_entry::SegmentEntry;
pub use self::segment_manager::SegmentManager;
pub use self::segment_serializer::SegmentSerializer;
pub(crate) use self::segment_serializer::SegmentSerializer;
pub use self::segment_updater::{merge_filtered_segments, merge_indices};
pub use self::segment_writer::SegmentWriter;
use crate::indexer::operation::AddOperation;
@@ -103,6 +110,35 @@ mod tests_mmap {
}
}
#[test]
fn test_json_field_number() {
// this test was added specifically to reach some cases related to using json fields, with
// frequency enabled, to store integers, with enough documents containing a single integer
// that the posting list can be bitpacked.
let mut schema_builder = Schema::builder();
let json_field = schema_builder.add_json_field("json", TEXT);
let index = Index::create_in_ram(schema_builder.build());
let mut index_writer = index.writer_for_tests().unwrap();
for _ in 0..256 {
let json = serde_json::json!({"somekey": 1u64, "otherkey": -2i64});
index_writer.add_document(doc!(json_field=>json)).unwrap();
let json = serde_json::json!({"somekey": "1str", "otherkey": "2str"});
index_writer.add_document(doc!(json_field=>json)).unwrap();
}
index_writer.commit().unwrap();
let reader = index.reader().unwrap();
let searcher = reader.searcher();
assert_eq!(searcher.num_docs(), 512);
let parse_query = QueryParser::for_index(&index, Vec::new());
{
let query = parse_query.parse_query(r"json.somekey:1").unwrap();
let num_docs = searcher.search(&query, &Count).unwrap();
assert_eq!(num_docs, 256);
}
}
#[test]
fn test_json_field_expand_dots_enabled_dot_escape_not_required() {
let mut schema_builder = Schema::builder();

View File

@@ -0,0 +1,92 @@
use fnv::FnvHashMap;
/// `Field` is represented by an unsigned 32-bit integer type.
/// The schema holds the mapping between field names and `Field` objects.
#[derive(Copy, Default, Clone, Debug, PartialEq, PartialOrd, Eq, Ord, Hash)]
pub struct OrderedPathId(u32);
impl OrderedPathId {
/// Create a new field object for the given PathId.
pub const fn from_ordered_id(field_id: u32) -> OrderedPathId {
OrderedPathId(field_id)
}
/// Returns a u32 identifying uniquely a path within a schema.
pub const fn path_id(self) -> u32 {
self.0
}
}
impl From<u32> for OrderedPathId {
fn from(id: u32) -> Self {
Self(id)
}
}
#[derive(Default)]
pub(crate) struct PathToUnorderedId {
map: FnvHashMap<String, u32>,
}
impl PathToUnorderedId {
#[inline]
pub(crate) fn get_or_allocate_unordered_id(&mut self, path: &str) -> u32 {
if let Some(id) = self.map.get(path) {
return *id;
}
self.insert_new_path(path)
}
#[cold]
fn insert_new_path(&mut self, path: &str) -> u32 {
let next_id = self.map.len() as u32;
self.map.insert(path.to_string(), next_id);
next_id
}
/// Retuns ids which reflect the lexical order of the paths.
///
/// The returned vec can be indexed with the unordered id to get the ordered id.
pub(crate) fn unordered_id_to_ordered_id(&self) -> Vec<OrderedPathId> {
let mut sorted_ids: Vec<(&str, &u32)> =
self.map.iter().map(|(k, v)| (k.as_str(), v)).collect();
sorted_ids.sort_unstable_by_key(|(path, _)| *path);
let mut result = vec![OrderedPathId::default(); sorted_ids.len()];
for (ordered, unordered) in sorted_ids.iter().map(|(_k, v)| v).enumerate() {
result[**unordered as usize] = OrderedPathId::from_ordered_id(ordered as u32);
}
result
}
/// Retuns the paths so they can be queried by the ordered id (which is the index).
pub(crate) fn ordered_id_to_path(&self) -> Vec<&str> {
let mut paths = self.map.keys().map(String::as_str).collect::<Vec<_>>();
paths.sort_unstable();
paths
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn path_to_unordered_test() {
let mut path_to_id = PathToUnorderedId::default();
let terms = vec!["b", "a", "b", "c"];
let ids = terms
.iter()
.map(|term| path_to_id.get_or_allocate_unordered_id(term))
.collect::<Vec<u32>>();
assert_eq!(ids, vec![0, 1, 0, 2]);
let ordered_ids = ids
.iter()
.map(|id| path_to_id.unordered_id_to_ordered_id()[*id as usize])
.collect::<Vec<OrderedPathId>>();
assert_eq!(ordered_ids, vec![1.into(), 0.into(), 1.into(), 2.into()]);
// Fetch terms
let terms_fetched = ordered_ids
.iter()
.map(|id| path_to_id.ordered_id_to_path()[id.path_id() as usize])
.collect::<Vec<&str>>();
assert_eq!(terms_fetched, terms);
}
}

View File

@@ -1,4 +1,5 @@
use columnar::MonotonicallyMappableToU64;
use common::JsonPathWriter;
use itertools::Itertools;
use tokenizer_api::BoxTokenStream;
@@ -66,6 +67,7 @@ pub struct SegmentWriter {
pub(crate) segment_serializer: SegmentSerializer,
pub(crate) fast_field_writers: FastFieldsWriter,
pub(crate) fieldnorms_writer: FieldNormsWriter,
pub(crate) json_path_writer: JsonPathWriter,
pub(crate) doc_opstamps: Vec<Opstamp>,
per_field_text_analyzers: Vec<TextAnalyzer>,
term_buffer: Term,
@@ -116,6 +118,7 @@ impl SegmentWriter {
ctx: IndexingContext::new(table_size),
per_field_postings_writers,
fieldnorms_writer: FieldNormsWriter::for_schema(&schema),
json_path_writer: JsonPathWriter::default(),
segment_serializer,
fast_field_writers: FastFieldsWriter::from_schema_and_tokenizer_manager(
&schema,
@@ -144,6 +147,7 @@ impl SegmentWriter {
.map(|sort_by_field| get_doc_id_mapping_from_field(sort_by_field, &self))
.transpose()?;
remap_and_write(
self.schema,
&self.per_field_postings_writers,
self.ctx,
self.fast_field_writers,
@@ -155,6 +159,8 @@ impl SegmentWriter {
Ok(doc_opstamps)
}
/// Returns an estimation of the current memory usage of the segment writer.
/// If the mem usage exceeds the `memory_budget`, the segment be serialized.
pub fn mem_usage(&self) -> usize {
self.ctx.mem_usage()
+ self.fieldnorms_writer.mem_usage()
@@ -353,6 +359,7 @@ impl SegmentWriter {
json_options.is_expand_dots_enabled(),
term_buffer,
postings_writer,
&mut self.json_path_writer,
ctx,
)?;
}
@@ -420,6 +427,7 @@ impl SegmentWriter {
///
/// `doc_id_map` is used to map to the new doc_id order.
fn remap_and_write(
schema: Schema,
per_field_postings_writers: &PerFieldPostingsWriter,
ctx: IndexingContext,
fast_field_writers: FastFieldsWriter,
@@ -437,6 +445,7 @@ fn remap_and_write(
let fieldnorm_readers = FieldNormReaders::open(fieldnorm_data)?;
serialize_postings(
ctx,
schema,
per_field_postings_writers,
fieldnorm_readers,
doc_id_map,
@@ -486,12 +495,11 @@ mod tests {
use tempfile::TempDir;
use super::compute_initial_table_size;
use crate::collector::Count;
use crate::collector::{Count, TopDocs};
use crate::core::json_utils::JsonTermWriter;
use crate::directory::RamDirectory;
use crate::postings::TermInfo;
use crate::query::PhraseQuery;
use crate::query::{PhraseQuery, QueryParser};
use crate::schema::document::Value;
use crate::schema::{
Document, IndexRecordOption, Schema, TextFieldIndexing, TextOptions, Type, STORED, STRING,
@@ -507,10 +515,12 @@ mod tests {
};
#[test]
#[cfg(not(feature = "compare_hash_only"))]
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(15_000_000).unwrap(), 1 << 18);
use super::compute_initial_table_size;
assert_eq!(compute_initial_table_size(100_000).unwrap(), 1 << 12);
assert_eq!(compute_initial_table_size(1_000_000).unwrap(), 1 << 15);
assert_eq!(compute_initial_table_size(15_000_000).unwrap(), 1 << 19);
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);
}
@@ -550,6 +560,43 @@ mod tests {
assert_eq!(doc.field_values()[0].value().as_str(), Some("A"));
assert_eq!(doc.field_values()[1].value().as_str(), Some("title"));
}
#[test]
fn test_simple_json_indexing() {
let mut schema_builder = Schema::builder();
let json_field = schema_builder.add_json_field("json", STORED | STRING);
let schema = schema_builder.build();
let index = Index::create_in_ram(schema.clone());
let mut writer = index.writer_for_tests().unwrap();
writer
.add_document(doc!(json_field=>json!({"my_field": "b"})))
.unwrap();
writer
.add_document(doc!(json_field=>json!({"my_field": "a"})))
.unwrap();
writer
.add_document(doc!(json_field=>json!({"my_field": "b"})))
.unwrap();
writer.commit().unwrap();
let query_parser = QueryParser::for_index(&index, vec![json_field]);
let text_query = query_parser.parse_query("my_field:a").unwrap();
let score_docs: Vec<(_, DocAddress)> = index
.reader()
.unwrap()
.searcher()
.search(&text_query, &TopDocs::with_limit(4))
.unwrap();
assert_eq!(score_docs.len(), 1);
let text_query = query_parser.parse_query("my_field:b").unwrap();
let score_docs: Vec<(_, DocAddress)> = index
.reader()
.unwrap()
.searcher()
.search(&text_query, &TopDocs::with_limit(4))
.unwrap();
assert_eq!(score_docs.len(), 2);
}
#[test]
fn test_json_indexing() {

View File

@@ -103,7 +103,48 @@
//! the example code (
//! [literate programming](https://tantivy-search.github.io/examples/basic_search.html) /
//! [source code](https://github.com/quickwit-oss/tantivy/blob/main/examples/basic_search.rs))
//!
//! # Tantivy Architecture Overview
//!
//! Tantivy is inspired by Lucene, the Architecture is very similar.
//!
//! ## Core Concepts
//!
//! - **[Index]**: A collection of segments. The top level entry point for tantivy users to search
//! and index data.
//!
//! - **[Segment]**: At the heart of Tantivy's indexing structure is the [Segment]. It contains
//! documents and indices and is the atomic unit of indexing and search.
//!
//! - **[Schema](schema)**: A schema is a set of fields in an index. Each field has a specific data
//! type and set of attributes.
//!
//! - **[IndexWriter]**: Responsible creating and merging segments. It executes the indexing
//! pipeline including tokenization, creating indices, and storing the index in the
//! [Directory](directory).
//!
//! - **Searching**: [Searcher] searches the segments with anything that implements
//! [Query](query::Query) and merges the results. The list of [supported
//! queries](query::Query#implementors). Custom Queries are supported by implementing the
//! [Query](query::Query) trait.
//!
//! - **[Directory](directory)**: Abstraction over the storage where the index data is stored.
//!
//! - **[Tokenizer](tokenizer)**: Breaks down text into individual tokens. Users can implement or
//! use provided tokenizers.
//!
//! ## Architecture Flow
//!
//! 1. **Document Addition**: Users create documents according to the defined schema. The documents
//! fields are tokenized, processed, and added to the current segment. See
//! [Document](schema::document) for the structure and usage.
//!
//! 2. **Segment Creation**: Once the memory limit threshold is reached or a commit is called, the
//! segment is written to the Directory. Documents are searchable after `commit`.
//!
//! 3. **Merging**: To optimize space and search speed, segments might be merged. This operation is
//! performed in the background. Customize the merge behaviour via
//! [IndexWriter::set_merge_policy].
#[cfg_attr(test, macro_use)]
extern crate serde_json;
#[macro_use]
@@ -137,7 +178,7 @@ pub use crate::future_result::FutureResult;
pub type Result<T> = std::result::Result<T, TantivyError>;
mod core;
mod indexer;
pub mod indexer;
#[allow(unused_doc_comments)]
pub mod error;
@@ -161,8 +202,7 @@ pub mod termdict;
mod reader;
pub use self::reader::{IndexReader, IndexReaderBuilder, ReloadPolicy, Warmer};
mod snippet;
pub use self::snippet::{Snippet, SnippetGenerator};
pub mod snippet;
mod docset;
use std::fmt;
@@ -173,6 +213,11 @@ use once_cell::sync::Lazy;
use serde::{Deserialize, Serialize};
pub use self::docset::{DocSet, TERMINATED};
#[deprecated(
since = "0.22.0",
note = "Will be removed in tantivy 0.23. Use export from snippet module instead"
)]
pub use self::snippet::{Snippet, SnippetGenerator};
#[doc(hidden)]
pub use crate::core::json_utils;
pub use crate::core::{
@@ -181,18 +226,21 @@ pub use crate::core::{
SegmentReader, SingleSegmentIndexWriter,
};
pub use crate::directory::Directory;
pub use crate::indexer::operation::UserOperation;
pub use crate::indexer::{merge_filtered_segments, merge_indices, IndexWriter, PreparedCommit};
pub use crate::indexer::IndexWriter;
#[deprecated(
since = "0.22.0",
note = "Will be removed in tantivy 0.23. Use export from indexer module instead"
)]
pub use crate::indexer::{merge_filtered_segments, merge_indices, PreparedCommit};
pub use crate::postings::Postings;
#[allow(deprecated)]
pub use crate::schema::DatePrecision;
pub use crate::schema::{DateOptions, DateTimePrecision, Document, TantivyDocument, Term};
/// Index format version.
const INDEX_FORMAT_VERSION: u32 = 5;
#[cfg(all(feature = "mmap", unix))]
pub use memmap2::Advice;
const INDEX_FORMAT_VERSION: u32 = 6;
/// Oldest index format version this tantivy version can read.
const INDEX_FORMAT_OLDEST_SUPPORTED_VERSION: u32 = 4;
/// Structure version for the index.
#[derive(Clone, PartialEq, Eq, Serialize, Deserialize)]

View File

@@ -92,7 +92,7 @@ impl PositionReader {
// that block is bitpacked.
let bit_width = bit_widths[block_rel_id];
self.block_decoder
.uncompress_block_unsorted(compressed_data, bit_width);
.uncompress_block_unsorted(compressed_data, bit_width, false);
} else {
// that block is vint encoded.
self.block_decoder

View File

@@ -62,8 +62,9 @@ impl<W: io::Write> PositionSerializer<W> {
return;
}
if self.block.len() == COMPRESSION_BLOCK_SIZE {
let (bit_width, block_encoded): (u8, &[u8]) =
self.block_encoder.compress_block_unsorted(&self.block[..]);
let (bit_width, block_encoded): (u8, &[u8]) = self
.block_encoder
.compress_block_unsorted(&self.block[..], false);
self.bit_widths.push(bit_width);
self.positions_buffer.extend(block_encoded);
} else {

View File

@@ -24,13 +24,13 @@ fn max_score<I: Iterator<Item = Score>>(mut it: I) -> Option<Score> {
#[derive(Clone)]
pub struct BlockSegmentPostings {
pub(crate) doc_decoder: BlockDecoder,
loaded_offset: usize,
block_loaded: bool,
freq_decoder: BlockDecoder,
freq_reading_option: FreqReadingOption,
block_max_score_cache: Option<Score>,
doc_freq: u32,
data: OwnedBytes,
pub(crate) skip_reader: SkipReader,
skip_reader: SkipReader,
}
fn decode_bitpacked_block(
@@ -40,10 +40,16 @@ fn decode_bitpacked_block(
doc_offset: DocId,
doc_num_bits: u8,
tf_num_bits: u8,
strict_delta: bool,
) {
let num_consumed_bytes = doc_decoder.uncompress_block_sorted(data, doc_offset, doc_num_bits);
let num_consumed_bytes =
doc_decoder.uncompress_block_sorted(data, doc_offset, doc_num_bits, strict_delta);
if let Some(freq_decoder) = freq_decoder_opt {
freq_decoder.uncompress_block_unsorted(&data[num_consumed_bytes..], tf_num_bits);
freq_decoder.uncompress_block_unsorted(
&data[num_consumed_bytes..],
tf_num_bits,
strict_delta,
);
}
}
@@ -57,11 +63,15 @@ fn decode_vint_block(
let num_consumed_bytes =
doc_decoder.uncompress_vint_sorted(data, doc_offset, num_vint_docs, TERMINATED);
if let Some(freq_decoder) = freq_decoder_opt {
freq_decoder.uncompress_vint_unsorted(
&data[num_consumed_bytes..],
num_vint_docs,
TERMINATED,
);
// if it's a json term with freq, containing less than 256 docs, we can reach here thinking
// we have a freq, despite not really having one.
if data.len() > num_consumed_bytes {
freq_decoder.uncompress_vint_unsorted(
&data[num_consumed_bytes..],
num_vint_docs,
TERMINATED,
);
}
}
}
@@ -78,28 +88,46 @@ fn split_into_skips_and_postings(
}
impl BlockSegmentPostings {
/// Opens a `BlockSegmentPostings`.
/// `doc_freq` is the number of documents in the posting list.
/// `record_option` represents the amount of data available according to the schema.
/// `requested_option` is the amount of data requested by the user.
/// If for instance, we do not request for term frequencies, this function will not decompress
/// term frequency blocks.
pub(crate) fn open(
doc_freq: u32,
data: FileSlice,
record_option: IndexRecordOption,
mut record_option: IndexRecordOption,
requested_option: IndexRecordOption,
) -> io::Result<BlockSegmentPostings> {
let bytes = data.read_bytes()?;
let (skip_data_opt, postings_data) = split_into_skips_and_postings(doc_freq, bytes)?;
let skip_reader = match skip_data_opt {
Some(skip_data) => {
let block_count = doc_freq as usize / COMPRESSION_BLOCK_SIZE;
// 8 is the minimum size of a block with frequency (can be more if pos are stored
// too)
if skip_data.len() < 8 * block_count {
// the field might be encoded with frequency, but this term in particular isn't.
// This can happen for JSON field with term frequencies:
// - text terms are encoded with term freqs.
// - numerical terms are encoded without term freqs.
record_option = IndexRecordOption::Basic;
}
SkipReader::new(skip_data, doc_freq, record_option)
}
None => SkipReader::new(OwnedBytes::empty(), doc_freq, record_option),
};
let freq_reading_option = match (record_option, requested_option) {
(IndexRecordOption::Basic, _) => FreqReadingOption::NoFreq,
(_, IndexRecordOption::Basic) => FreqReadingOption::SkipFreq,
(_, _) => FreqReadingOption::ReadFreq,
};
let bytes = data.read_bytes()?;
let (skip_data_opt, postings_data) = split_into_skips_and_postings(doc_freq, bytes)?;
let skip_reader = match skip_data_opt {
Some(skip_data) => SkipReader::new(skip_data, doc_freq, record_option),
None => SkipReader::new(OwnedBytes::empty(), doc_freq, record_option),
};
let mut block_segment_postings = BlockSegmentPostings {
doc_decoder: BlockDecoder::with_val(TERMINATED),
loaded_offset: usize::MAX,
block_loaded: false,
freq_decoder: BlockDecoder::with_val(1),
freq_reading_option,
block_max_score_cache: None,
@@ -169,7 +197,7 @@ impl BlockSegmentPostings {
split_into_skips_and_postings(doc_freq, postings_data)?;
self.data = postings_data;
self.block_max_score_cache = None;
self.loaded_offset = usize::MAX;
self.block_loaded = false;
if let Some(skip_data) = skip_data_opt {
self.skip_reader.reset(skip_data, doc_freq);
} else {
@@ -265,22 +293,23 @@ impl BlockSegmentPostings {
pub(crate) fn shallow_seek(&mut self, target_doc: DocId) {
if self.skip_reader.seek(target_doc) {
self.block_max_score_cache = None;
self.block_loaded = false;
}
}
pub(crate) fn block_is_loaded(&self) -> bool {
self.loaded_offset == self.skip_reader.byte_offset()
self.block_loaded
}
pub(crate) fn load_block(&mut self) {
let offset = self.skip_reader.byte_offset();
if self.loaded_offset == offset {
if self.block_is_loaded() {
return;
}
self.loaded_offset = offset;
match self.skip_reader.block_info() {
BlockInfo::BitPacked {
doc_num_bits,
strict_delta_encoded,
tf_num_bits,
..
} => {
@@ -295,6 +324,7 @@ impl BlockSegmentPostings {
self.skip_reader.last_doc_in_previous_block,
doc_num_bits,
tf_num_bits,
strict_delta_encoded,
);
}
BlockInfo::VInt { num_docs } => {
@@ -318,13 +348,13 @@ impl BlockSegmentPostings {
);
}
}
self.block_loaded = true;
}
/// Advance to the next block.
///
/// Returns false if and only if there is no remaining block.
pub fn advance(&mut self) {
self.skip_reader.advance();
self.block_loaded = false;
self.block_max_score_cache = None;
self.load_block();
}
@@ -333,7 +363,7 @@ impl BlockSegmentPostings {
pub fn empty() -> BlockSegmentPostings {
BlockSegmentPostings {
doc_decoder: BlockDecoder::with_val(TERMINATED),
loaded_offset: 0,
block_loaded: true,
freq_decoder: BlockDecoder::with_val(1),
freq_reading_option: FreqReadingOption::NoFreq,
block_max_score_cache: None,
@@ -342,6 +372,10 @@ impl BlockSegmentPostings {
skip_reader: SkipReader::new(OwnedBytes::empty(), 0, IndexRecordOption::Basic),
}
}
pub(crate) fn skip_reader(&self) -> &SkipReader {
&self.skip_reader
}
}
#[cfg(test)]

View File

@@ -33,14 +33,40 @@ impl BlockEncoder {
}
pub fn compress_block_sorted(&mut self, block: &[u32], offset: u32) -> (u8, &[u8]) {
let num_bits = self.bitpacker.num_bits_sorted(offset, block);
// if offset is zero, convert it to None. This is correct as long as we do the same when
// decompressing. It's required in case the block starts with an actual zero.
let offset = if offset == 0u32 { None } else { Some(offset) };
let num_bits = self.bitpacker.num_bits_strictly_sorted(offset, block);
let written_size =
self.bitpacker
.compress_sorted(offset, block, &mut self.output[..], num_bits);
.compress_strictly_sorted(offset, block, &mut self.output[..], num_bits);
(num_bits, &self.output[..written_size])
}
pub fn compress_block_unsorted(&mut self, block: &[u32]) -> (u8, &[u8]) {
/// Compress a single block of unsorted numbers.
///
/// If `minus_one_encoded` is set, each value must be >= 1, and will be encoded in a sligly
/// more compact format. This is useful for some values where 0 isn't a correct value, such
/// as term frequency, but isn't correct for some usages like position lists, where 0 can
/// appear.
pub fn compress_block_unsorted(
&mut self,
block: &[u32],
minus_one_encoded: bool,
) -> (u8, &[u8]) {
debug_assert!(!minus_one_encoded || !block.contains(&0));
let mut block_minus_one = [0; COMPRESSION_BLOCK_SIZE];
let block = if minus_one_encoded {
for (elem_min_one, elem) in block_minus_one.iter_mut().zip(block) {
*elem_min_one = elem - 1;
}
&block_minus_one
} else {
block
};
let num_bits = self.bitpacker.num_bits(block);
let written_size = self
.bitpacker
@@ -71,21 +97,55 @@ impl BlockDecoder {
}
}
/// Decompress block of sorted integers.
///
/// `strict_delta` depends on what encoding was used. Older version of tantivy never use strict
/// deltas, newer versions always use them.
pub fn uncompress_block_sorted(
&mut self,
compressed_data: &[u8],
offset: u32,
num_bits: u8,
strict_delta: bool,
) -> usize {
self.output_len = COMPRESSION_BLOCK_SIZE;
self.bitpacker
.decompress_sorted(offset, compressed_data, &mut self.output, num_bits)
if strict_delta {
let offset = std::num::NonZeroU32::new(offset).map(std::num::NonZeroU32::get);
self.output_len = COMPRESSION_BLOCK_SIZE;
self.bitpacker.decompress_strictly_sorted(
offset,
compressed_data,
&mut self.output,
num_bits,
)
} else {
self.output_len = COMPRESSION_BLOCK_SIZE;
self.bitpacker
.decompress_sorted(offset, compressed_data, &mut self.output, num_bits)
}
}
pub fn uncompress_block_unsorted(&mut self, compressed_data: &[u8], num_bits: u8) -> usize {
/// Decompress block of unsorted integers.
///
/// `minus_one_encoded` depends on what encoding was used. Older version of tantivy never use
/// that encoding. Newer version use it for some structures, but not all. See the corresponding
/// call to `BlockEncoder::compress_block_unsorted`.
pub fn uncompress_block_unsorted(
&mut self,
compressed_data: &[u8],
num_bits: u8,
minus_one_encoded: bool,
) -> usize {
self.output_len = COMPRESSION_BLOCK_SIZE;
self.bitpacker
.decompress(compressed_data, &mut self.output, num_bits)
let res = self
.bitpacker
.decompress(compressed_data, &mut self.output, num_bits);
if minus_one_encoded {
for val in &mut self.output {
*val += 1;
}
}
res
}
#[inline]
@@ -218,7 +278,8 @@ pub mod tests {
let (num_bits, compressed_data) = encoder.compress_block_sorted(&vals, 0);
let mut decoder = BlockDecoder::default();
{
let consumed_num_bytes = decoder.uncompress_block_sorted(compressed_data, 0, num_bits);
let consumed_num_bytes =
decoder.uncompress_block_sorted(compressed_data, 0, num_bits, true);
assert_eq!(consumed_num_bytes, compressed_data.len());
}
for i in 0..128 {
@@ -233,7 +294,8 @@ pub mod tests {
let (num_bits, compressed_data) = encoder.compress_block_sorted(&vals, 10);
let mut decoder = BlockDecoder::default();
{
let consumed_num_bytes = decoder.uncompress_block_sorted(compressed_data, 10, num_bits);
let consumed_num_bytes =
decoder.uncompress_block_sorted(compressed_data, 10, num_bits, true);
assert_eq!(consumed_num_bytes, compressed_data.len());
}
for i in 0..128 {
@@ -252,7 +314,8 @@ pub mod tests {
compressed.push(173u8);
let mut decoder = BlockDecoder::default();
{
let consumed_num_bytes = decoder.uncompress_block_sorted(&compressed, 10, num_bits);
let consumed_num_bytes =
decoder.uncompress_block_sorted(&compressed, 10, num_bits, true);
assert_eq!(consumed_num_bytes, compressed.len() - 1);
assert_eq!(compressed[consumed_num_bytes], 173u8);
}
@@ -263,21 +326,25 @@ pub mod tests {
#[test]
fn test_encode_unsorted_block_with_junk() {
let mut compressed: Vec<u8> = Vec::new();
let n = 128;
let vals: Vec<u32> = (0..n).map(|i| 11u32 + (i as u32) * 7u32 % 12).collect();
let mut encoder = BlockEncoder::default();
let (num_bits, compressed_data) = encoder.compress_block_unsorted(&vals);
compressed.extend_from_slice(compressed_data);
compressed.push(173u8);
let mut decoder = BlockDecoder::default();
{
let consumed_num_bytes = decoder.uncompress_block_unsorted(&compressed, num_bits);
assert_eq!(consumed_num_bytes + 1, compressed.len());
assert_eq!(compressed[consumed_num_bytes], 173u8);
}
for i in 0..n {
assert_eq!(vals[i], decoder.output(i));
for minus_one_encode in [false, true] {
let mut compressed: Vec<u8> = Vec::new();
let n = 128;
let vals: Vec<u32> = (0..n).map(|i| 11u32 + (i as u32) * 7u32 % 12).collect();
let mut encoder = BlockEncoder::default();
let (num_bits, compressed_data) =
encoder.compress_block_unsorted(&vals, minus_one_encode);
compressed.extend_from_slice(compressed_data);
compressed.push(173u8);
let mut decoder = BlockDecoder::default();
{
let consumed_num_bytes =
decoder.uncompress_block_unsorted(&compressed, num_bits, minus_one_encode);
assert_eq!(consumed_num_bytes + 1, compressed.len());
assert_eq!(compressed[consumed_num_bytes], 173u8);
}
for i in 0..n {
assert_eq!(vals[i], decoder.output(i));
}
}
}
@@ -344,7 +411,7 @@ mod bench {
let (num_bits, compressed) = encoder.compress_block_sorted(&data, 0u32);
let mut decoder = BlockDecoder::default();
b.iter(|| {
decoder.uncompress_block_sorted(compressed, 0u32, num_bits);
decoder.uncompress_block_sorted(compressed, 0u32, num_bits, true);
});
}

View File

@@ -1,5 +1,7 @@
use stacker::{ArenaHashMap, MemoryArena};
use crate::indexer::path_to_unordered_id::PathToUnorderedId;
/// IndexingContext contains all of the transient memory arenas
/// required for building the inverted index.
pub(crate) struct IndexingContext {
@@ -8,6 +10,7 @@ pub(crate) struct IndexingContext {
pub term_index: ArenaHashMap,
/// Arena is a memory arena that stores posting lists / term frequencies / positions.
pub arena: MemoryArena,
pub path_to_unordered_id: PathToUnorderedId,
}
impl IndexingContext {
@@ -17,6 +20,7 @@ impl IndexingContext {
IndexingContext {
arena: MemoryArena::default(),
term_index,
path_to_unordered_id: PathToUnorderedId::default(),
}
}

View File

@@ -3,10 +3,11 @@ use std::io;
use stacker::Addr;
use crate::indexer::doc_id_mapping::DocIdMapping;
use crate::indexer::path_to_unordered_id::OrderedPathId;
use crate::postings::postings_writer::SpecializedPostingsWriter;
use crate::postings::recorder::{BufferLender, DocIdRecorder, Recorder};
use crate::postings::{FieldSerializer, IndexingContext, IndexingPosition, PostingsWriter};
use crate::schema::Type;
use crate::schema::{Field, Type, JSON_END_OF_PATH};
use crate::tokenizer::TokenStream;
use crate::{DocId, Term};
@@ -54,18 +55,24 @@ impl<Rec: Recorder> PostingsWriter for JsonPostingsWriter<Rec> {
/// The actual serialization format is handled by the `PostingsSerializer`.
fn serialize(
&self,
term_addrs: &[(Term<&[u8]>, Addr)],
term_addrs: &[(Field, OrderedPathId, &[u8], Addr)],
ordered_id_to_path: &[&str],
doc_id_map: Option<&DocIdMapping>,
ctx: &IndexingContext,
serializer: &mut FieldSerializer,
) -> io::Result<()> {
let mut term_buffer = Term::with_capacity(48);
let mut buffer_lender = BufferLender::default();
for (term, addr) in term_addrs {
if let Some(json_value) = term.value().as_json_value_bytes() {
for (_field, path_id, term, addr) in term_addrs {
term_buffer.clear_with_field_and_type(Type::Json, Field::from_field_id(0));
term_buffer.append_bytes(ordered_id_to_path[path_id.path_id() as usize].as_bytes());
term_buffer.append_bytes(&[JSON_END_OF_PATH]);
term_buffer.append_bytes(term);
if let Some(json_value) = term_buffer.value().as_json_value_bytes() {
let typ = json_value.typ();
if typ == Type::Str {
SpecializedPostingsWriter::<Rec>::serialize_one_term(
term,
term_buffer.serialized_value_bytes(),
*addr,
doc_id_map,
&mut buffer_lender,
@@ -74,7 +81,7 @@ impl<Rec: Recorder> PostingsWriter for JsonPostingsWriter<Rec> {
)?;
} else {
SpecializedPostingsWriter::<DocIdRecorder>::serialize_one_term(
term,
term_buffer.serialized_value_bytes(),
*addr,
doc_id_map,
&mut buffer_lender,

View File

@@ -6,20 +6,23 @@ use stacker::Addr;
use crate::fieldnorm::FieldNormReaders;
use crate::indexer::doc_id_mapping::DocIdMapping;
use crate::indexer::path_to_unordered_id::OrderedPathId;
use crate::postings::recorder::{BufferLender, Recorder};
use crate::postings::{
FieldSerializer, IndexingContext, InvertedIndexSerializer, PerFieldPostingsWriter,
};
use crate::schema::{Field, Term};
use crate::schema::{Field, Schema, Term, Type};
use crate::tokenizer::{Token, TokenStream, MAX_TOKEN_LEN};
use crate::DocId;
const POSITION_GAP: u32 = 1;
fn make_field_partition(term_offsets: &[(Term<&[u8]>, Addr)]) -> Vec<(Field, Range<usize>)> {
fn make_field_partition(
term_offsets: &[(Field, OrderedPathId, &[u8], Addr)],
) -> Vec<(Field, Range<usize>)> {
let term_offsets_it = term_offsets
.iter()
.map(|(term, _)| term.field())
.map(|(field, _, _, _)| *field)
.enumerate();
let mut prev_field_opt = None;
let mut fields = vec![];
@@ -44,19 +47,36 @@ fn make_field_partition(term_offsets: &[(Term<&[u8]>, Addr)]) -> Vec<(Field, Ran
/// postings serializer.
pub(crate) fn serialize_postings(
ctx: IndexingContext,
schema: Schema,
per_field_postings_writers: &PerFieldPostingsWriter,
fieldnorm_readers: FieldNormReaders,
doc_id_map: Option<&DocIdMapping>,
serializer: &mut InvertedIndexSerializer,
) -> crate::Result<()> {
let mut term_offsets: Vec<(Term<&[u8]>, Addr)> = Vec::with_capacity(ctx.term_index.len());
term_offsets.extend(
ctx.term_index
.iter()
.map(|(bytes, addr, _unordered_id)| (Term::wrap(bytes), addr)),
);
term_offsets.sort_unstable_by_key(|(k, _)| k.clone());
// Replace unordered ids by ordered ids to be able to sort
let unordered_id_to_ordered_id: Vec<OrderedPathId> =
ctx.path_to_unordered_id.unordered_id_to_ordered_id();
let mut term_offsets: Vec<(Field, OrderedPathId, &[u8], Addr)> =
Vec::with_capacity(ctx.term_index.len());
term_offsets.extend(ctx.term_index.iter().map(|(key, addr)| {
let field = Term::wrap(key).field();
if schema.get_field_entry(field).field_type().value_type() == Type::Json {
let byte_range_path = 5..5 + 4;
let unordered_id = u32::from_be_bytes(key[byte_range_path.clone()].try_into().unwrap());
let path_id = unordered_id_to_ordered_id[unordered_id as usize];
(field, path_id, &key[byte_range_path.end..], addr)
} else {
(field, 0.into(), &key[5..], addr)
}
}));
// Sort by field, path, and term
term_offsets.sort_unstable_by(
|(field1, path_id1, bytes1, _), (field2, path_id2, bytes2, _)| {
(field1, path_id1, bytes1).cmp(&(field2, path_id2, bytes2))
},
);
let ordered_id_to_path = ctx.path_to_unordered_id.ordered_id_to_path();
let field_offsets = make_field_partition(&term_offsets);
for (field, byte_offsets) in field_offsets {
let postings_writer = per_field_postings_writers.get_for_field(field);
@@ -65,12 +85,14 @@ pub(crate) fn serialize_postings(
serializer.new_field(field, postings_writer.total_num_tokens(), fieldnorm_reader)?;
postings_writer.serialize(
&term_offsets[byte_offsets],
&ordered_id_to_path,
doc_id_map,
&ctx,
&mut field_serializer,
)?;
field_serializer.close()?;
}
Ok(())
}
@@ -98,7 +120,8 @@ pub(crate) trait PostingsWriter: Send + Sync {
/// The actual serialization format is handled by the `PostingsSerializer`.
fn serialize(
&self,
term_addrs: &[(Term<&[u8]>, Addr)],
term_addrs: &[(Field, OrderedPathId, &[u8], Addr)],
ordered_id_to_path: &[&str],
doc_id_map: Option<&DocIdMapping>,
ctx: &IndexingContext,
serializer: &mut FieldSerializer,
@@ -162,7 +185,7 @@ impl<Rec: Recorder> From<SpecializedPostingsWriter<Rec>> for Box<dyn PostingsWri
impl<Rec: Recorder> SpecializedPostingsWriter<Rec> {
#[inline]
pub(crate) fn serialize_one_term(
term: &Term<&[u8]>,
term: &[u8],
addr: Addr,
doc_id_map: Option<&DocIdMapping>,
buffer_lender: &mut BufferLender,
@@ -171,7 +194,7 @@ impl<Rec: Recorder> SpecializedPostingsWriter<Rec> {
) -> io::Result<()> {
let recorder: Rec = ctx.term_index.read(addr);
let term_doc_freq = recorder.term_doc_freq().unwrap_or(0u32);
serializer.new_term(term.serialized_value_bytes(), term_doc_freq)?;
serializer.new_term(term, term_doc_freq)?;
recorder.serialize(&ctx.arena, doc_id_map, serializer, buffer_lender);
serializer.close_term()?;
Ok(())
@@ -204,13 +227,14 @@ impl<Rec: Recorder> PostingsWriter for SpecializedPostingsWriter<Rec> {
fn serialize(
&self,
term_addrs: &[(Term<&[u8]>, Addr)],
term_addrs: &[(Field, OrderedPathId, &[u8], Addr)],
_ordered_id_to_path: &[&str],
doc_id_map: Option<&DocIdMapping>,
ctx: &IndexingContext,
serializer: &mut FieldSerializer,
) -> io::Result<()> {
let mut buffer_lender = BufferLender::default();
for (term, addr) in term_addrs {
for (_field, _path_id, term, addr) in term_addrs {
Self::serialize_one_term(term, *addr, doc_id_map, &mut buffer_lender, ctx, serializer)?;
}
Ok(())

View File

@@ -301,6 +301,7 @@ pub struct PostingsSerializer<W: Write> {
bm25_weight: Option<Bm25Weight>,
avg_fieldnorm: Score, /* Average number of term in the field for that segment.
* this value is used to compute the block wand information. */
term_has_freq: bool,
}
impl<W: Write> PostingsSerializer<W> {
@@ -325,13 +326,15 @@ impl<W: Write> PostingsSerializer<W> {
fieldnorm_reader,
bm25_weight: None,
avg_fieldnorm,
term_has_freq: false,
}
}
pub fn new_term(&mut self, term_doc_freq: u32) {
self.bm25_weight = None;
if !self.mode.has_freq() {
self.term_has_freq = self.mode.has_freq() && term_doc_freq != 0;
if !self.term_has_freq {
return;
}
@@ -365,10 +368,10 @@ impl<W: Write> PostingsSerializer<W> {
// last el block 0, offset block 1,
self.postings_write.extend(block_encoded);
}
if self.mode.has_freq() {
if self.term_has_freq {
let (num_bits, block_encoded): (u8, &[u8]) = self
.block_encoder
.compress_block_unsorted(self.block.term_freqs());
.compress_block_unsorted(self.block.term_freqs(), true);
self.postings_write.extend(block_encoded);
self.skip_write.write_term_freq(num_bits);
if self.mode.has_positions() {
@@ -432,7 +435,7 @@ impl<W: Write> PostingsSerializer<W> {
self.postings_write.write_all(block_encoded)?;
}
// ... Idem for term frequencies
if self.mode.has_freq() {
if self.term_has_freq {
let block_encoded = self
.block_encoder
.compress_vint_unsorted(self.block.term_freqs());

View File

@@ -6,6 +6,22 @@ use crate::query::Bm25Weight;
use crate::schema::IndexRecordOption;
use crate::{DocId, Score, TERMINATED};
// doc num bits uses the following encoding:
// given 0b a b cdefgh
// |1|2| 3 |
// - 1: unused
// - 2: is delta-1 encoded. 0 if not, 1, if yes
// - 3: a 6 bit number in 0..=32, the actual bitwidth
fn encode_bitwidth(bitwidth: u8, delta_1: bool) -> u8 {
bitwidth | ((delta_1 as u8) << 6)
}
fn decode_bitwidth(raw_bitwidth: u8) -> (u8, bool) {
let delta_1 = (raw_bitwidth >> 6 & 1) != 0;
let bitwidth = raw_bitwidth & 0x3f;
(bitwidth, delta_1)
}
#[inline]
fn encode_block_wand_max_tf(max_tf: u32) -> u8 {
max_tf.min(u8::MAX as u32) as u8
@@ -41,7 +57,7 @@ impl SkipSerializer {
pub fn write_doc(&mut self, last_doc: DocId, doc_num_bits: u8) {
write_u32(last_doc, &mut self.buffer);
self.buffer.push(doc_num_bits);
self.buffer.push(encode_bitwidth(doc_num_bits, true));
}
pub fn write_term_freq(&mut self, tf_num_bits: u8) {
@@ -85,6 +101,7 @@ pub(crate) struct SkipReader {
pub(crate) enum BlockInfo {
BitPacked {
doc_num_bits: u8,
strict_delta_encoded: bool,
tf_num_bits: u8,
tf_sum: u32,
block_wand_fieldnorm_id: u8,
@@ -172,12 +189,13 @@ impl SkipReader {
let bytes = self.owned_read.as_slice();
let advance_len: usize;
self.last_doc_in_block = read_u32(bytes);
let doc_num_bits = bytes[4];
let (doc_num_bits, strict_delta_encoded) = decode_bitwidth(bytes[4]);
match self.skip_info {
IndexRecordOption::Basic => {
advance_len = 5;
self.block_info = BlockInfo::BitPacked {
doc_num_bits,
strict_delta_encoded,
tf_num_bits: 0,
tf_sum: 0,
block_wand_fieldnorm_id: 0,
@@ -191,6 +209,7 @@ impl SkipReader {
advance_len = 8;
self.block_info = BlockInfo::BitPacked {
doc_num_bits,
strict_delta_encoded,
tf_num_bits,
tf_sum: 0,
block_wand_fieldnorm_id,
@@ -205,6 +224,7 @@ impl SkipReader {
advance_len = 12;
self.block_info = BlockInfo::BitPacked {
doc_num_bits,
strict_delta_encoded,
tf_num_bits,
tf_sum,
block_wand_fieldnorm_id,
@@ -268,7 +288,9 @@ impl SkipReader {
#[cfg(test)]
mod tests {
use super::{BlockInfo, IndexRecordOption, SkipReader, SkipSerializer};
use super::{
decode_bitwidth, encode_bitwidth, BlockInfo, IndexRecordOption, SkipReader, SkipSerializer,
};
use crate::directory::OwnedBytes;
use crate::postings::compression::COMPRESSION_BLOCK_SIZE;
@@ -310,6 +332,7 @@ mod tests {
skip_reader.block_info,
BlockInfo::BitPacked {
doc_num_bits: 2u8,
strict_delta_encoded: true,
tf_num_bits: 3u8,
tf_sum: 0,
block_wand_fieldnorm_id: 13,
@@ -322,6 +345,7 @@ mod tests {
skip_reader.block_info(),
BlockInfo::BitPacked {
doc_num_bits: 5u8,
strict_delta_encoded: true,
tf_num_bits: 2u8,
tf_sum: 0,
block_wand_fieldnorm_id: 8,
@@ -352,6 +376,7 @@ mod tests {
skip_reader.block_info(),
BlockInfo::BitPacked {
doc_num_bits: 2u8,
strict_delta_encoded: true,
tf_num_bits: 0,
tf_sum: 0u32,
block_wand_fieldnorm_id: 0,
@@ -364,6 +389,7 @@ mod tests {
skip_reader.block_info(),
BlockInfo::BitPacked {
doc_num_bits: 5u8,
strict_delta_encoded: true,
tf_num_bits: 0,
tf_sum: 0u32,
block_wand_fieldnorm_id: 0,
@@ -393,6 +419,7 @@ mod tests {
skip_reader.block_info(),
BlockInfo::BitPacked {
doc_num_bits: 2u8,
strict_delta_encoded: true,
tf_num_bits: 0,
tf_sum: 0u32,
block_wand_fieldnorm_id: 0,
@@ -402,4 +429,18 @@ mod tests {
skip_reader.advance();
assert_eq!(skip_reader.block_info(), BlockInfo::VInt { num_docs: 0u32 });
}
#[test]
fn test_encode_decode_bitwidth() {
for bitwidth in 0..=32 {
for delta_1 in [false, true] {
assert_eq!(
(bitwidth, delta_1),
decode_bitwidth(encode_bitwidth(bitwidth, delta_1))
);
}
}
assert_eq!(0b01000010, encode_bitwidth(0b10, true));
assert_eq!(0b00000010, encode_bitwidth(0b10, false));
}
}

View File

@@ -31,8 +31,8 @@ impl VecCursor {
self.current_pos = 0;
&mut self.docs
}
fn last_value(&self) -> Option<u32> {
self.docs.iter().last().cloned()
fn last_doc(&self) -> Option<u32> {
self.docs.last().cloned()
}
fn is_empty(&self) -> bool {
self.current().is_none()
@@ -112,15 +112,15 @@ impl<T: Send + Sync + PartialOrd + Copy + Debug + 'static> RangeDocSet<T> {
finished_to_end = true;
}
let last_value = self.loaded_docs.last_value();
let last_doc = self.loaded_docs.last_doc();
let doc_buffer: &mut Vec<DocId> = self.loaded_docs.get_cleared_data();
self.column.get_docids_for_value_range(
self.value_range.clone(),
self.next_fetch_start..end,
doc_buffer,
);
if let Some(last_value) = last_value {
while self.loaded_docs.current() == Some(last_value) {
if let Some(last_doc) = last_doc {
while self.loaded_docs.current() == Some(last_doc) {
self.loaded_docs.next();
}
}
@@ -136,7 +136,7 @@ impl<T: Send + Sync + PartialOrd + Copy + Debug + 'static> DocSet for RangeDocSe
if let Some(docid) = self.loaded_docs.next() {
return docid;
}
if self.next_fetch_start >= self.column.values.num_vals() {
if self.next_fetch_start >= self.column.num_docs() {
return TERMINATED;
}
self.fetch_block();
@@ -177,3 +177,54 @@ impl<T: Send + Sync + PartialOrd + Copy + Debug + 'static> DocSet for RangeDocSe
0 // heuristic possible by checking number of hits when fetching a block
}
}
#[cfg(test)]
mod tests {
use crate::collector::Count;
use crate::directory::RamDirectory;
use crate::query::RangeQuery;
use crate::{schema, IndexBuilder, TantivyDocument};
#[test]
fn range_query_fast_optional_field_minimum() {
let mut schema_builder = schema::SchemaBuilder::new();
let id_field = schema_builder.add_text_field("id", schema::STRING);
let score_field = schema_builder.add_u64_field("score", schema::FAST | schema::INDEXED);
let dir = RamDirectory::default();
let index = IndexBuilder::new()
.schema(schema_builder.build())
.open_or_create(dir)
.unwrap();
{
let mut writer = index.writer(15_000_000).unwrap();
let count = 1000;
for i in 0..count {
let mut doc = TantivyDocument::new();
doc.add_text(id_field, format!("doc{i}"));
let nb_scores = i % 2; // 0 or 1 scores
for _ in 0..nb_scores {
doc.add_u64(score_field, 80);
}
writer.add_document(doc).unwrap();
}
writer.commit().unwrap();
}
let reader = index.reader().unwrap();
let searcher = reader.searcher();
let query = RangeQuery::new_u64_bounds(
"score".to_string(),
std::ops::Bound::Included(70),
std::ops::Bound::Unbounded,
);
let count = searcher.search(&query, &Count).unwrap();
assert_eq!(count, 500);
}
}

View File

@@ -7,7 +7,7 @@ mod range_query;
mod range_query_ip_fastfield;
mod range_query_u64_fastfield;
pub use self::range_query::{RangeQuery, RangeWeight};
pub use self::range_query::RangeQuery;
pub use self::range_query_ip_fastfield::IPFastFieldRangeWeight;
pub use self::range_query_u64_fastfield::FastFieldRangeWeight;

View File

@@ -4,8 +4,6 @@ mod term_weight;
pub use self::term_query::TermQuery;
pub use self::term_scorer::TermScorer;
pub use self::term_weight::TermWeight;
#[cfg(test)]
mod tests {

View File

@@ -93,7 +93,7 @@ impl TermScorer {
}
pub fn last_doc_in_block(&self) -> DocId {
self.postings.block_cursor.skip_reader.last_doc_in_block()
self.postings.block_cursor.skip_reader().last_doc_in_block()
}
}
@@ -127,6 +127,7 @@ impl Scorer for TermScorer {
mod tests {
use proptest::prelude::*;
use crate::indexer::index_writer::MEMORY_BUDGET_NUM_BYTES_MIN;
use crate::merge_policy::NoMergePolicy;
use crate::postings::compression::COMPRESSION_BLOCK_SIZE;
use crate::query::term_query::TermScorer;
@@ -296,7 +297,8 @@ mod tests {
let text_field = schema_builder.add_text_field("text", TEXT);
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
let mut writer: IndexWriter = index.writer_with_num_threads(3, 30_000_000)?;
let mut writer: IndexWriter =
index.writer_with_num_threads(3, 3 * MEMORY_BUDGET_NUM_BYTES_MIN)?;
use rand::Rng;
let mut rng = rand::thread_rng();
writer.set_merge_policy(Box::new(NoMergePolicy));

View File

@@ -28,7 +28,7 @@ pub enum ReloadPolicy {
Manual,
/// The index is reloaded within milliseconds after a new commit is available.
/// This is made possible by watching changes in the `meta.json` file.
OnCommit, // TODO add NEAR_REAL_TIME(target_ms)
OnCommitWithDelay, // TODO add NEAR_REAL_TIME(target_ms)
}
/// [`IndexReader`] builder
@@ -51,7 +51,7 @@ impl IndexReaderBuilder {
#[must_use]
pub(crate) fn new(index: Index) -> IndexReaderBuilder {
IndexReaderBuilder {
reload_policy: ReloadPolicy::OnCommit,
reload_policy: ReloadPolicy::OnCommitWithDelay,
index,
warmers: Vec::new(),
num_warming_threads: 1,
@@ -83,7 +83,7 @@ impl IndexReaderBuilder {
// No need to set anything...
None
}
ReloadPolicy::OnCommit => {
ReloadPolicy::OnCommitWithDelay => {
let inner_reader_arc_clone = inner_reader_arc.clone();
let callback = move || {
if let Err(err) = inner_reader_arc_clone.reload() {
@@ -282,7 +282,7 @@ impl IndexReader {
/// Update searchers so that they reflect the state of the last
/// `.commit()`.
///
/// If you set up the [`ReloadPolicy::OnCommit`] (which is the default)
/// If you set up the [`ReloadPolicy::OnCommitWithDelay`] (which is the default)
/// every commit should be rapidly reflected on your `IndexReader` and you should
/// not need to call `reload()` at all.
///

View File

@@ -18,6 +18,8 @@ use std::sync::Arc;
use columnar::MonotonicallyMappableToU128;
use common::{u64_to_f64, BinarySerializable, DateTime, VInt};
use super::se::BinaryObjectSerializer;
use super::{OwnedValue, Value};
use crate::schema::document::type_codes;
use crate::schema::{Facet, Field};
use crate::tokenizer::PreTokenizedString;
@@ -157,6 +159,9 @@ pub enum ValueType {
Array,
/// A dynamic object value.
Object,
/// A JSON object value. Deprecated.
#[deprecated]
JSONObject,
}
/// A value visitor for deserializing a document value.
@@ -376,6 +381,8 @@ where R: Read
type_codes::NULL_CODE => ValueType::Null,
type_codes::ARRAY_CODE => ValueType::Array,
type_codes::OBJECT_CODE => ValueType::Object,
#[allow(deprecated)]
type_codes::JSON_OBJ_CODE => ValueType::JSONObject,
_ => {
return Err(DeserializeError::from(io::Error::new(
io::ErrorKind::InvalidData,
@@ -514,6 +521,26 @@ where R: Read
let access = BinaryObjectDeserializer::from_reader(self.reader)?;
visitor.visit_object(access)
}
#[allow(deprecated)]
ValueType::JSONObject => {
// This is a compatibility layer
// The implementation is slow, but is temporary anyways
let mut de = serde_json::Deserializer::from_reader(self.reader);
let json_map = <serde_json::Map::<String, serde_json::Value> as serde::Deserialize>::deserialize(&mut de).map_err(|err| DeserializeError::Custom(err.to_string()))?;
let mut out = Vec::new();
let mut serializer = BinaryObjectSerializer::begin(json_map.len(), &mut out)?;
for (key, val) in json_map {
let val: OwnedValue = val.into();
serializer.serialize_entry(&key, (&val).as_value())?;
}
serializer.end()?;
let out_rc = std::rc::Rc::new(out);
let mut slice: &[u8] = &out_rc;
let access = BinaryObjectDeserializer::from_reader(&mut slice)?;
visitor.visit_object(access)
}
}
}
}
@@ -791,7 +818,8 @@ mod tests {
use super::*;
use crate::schema::document::existing_type_impls::JsonObjectIter;
use crate::schema::document::se::BinaryValueSerializer;
use crate::schema::document::ReferenceValue;
use crate::schema::document::{ReferenceValue, ReferenceValueLeaf};
use crate::schema::OwnedValue;
fn serialize_value<'a>(value: ReferenceValue<'a, &'a serde_json::Value>) -> Vec<u8> {
let mut writer = Vec::new();
@@ -810,34 +838,35 @@ mod tests {
#[test]
fn test_simple_value_serialize() {
let result = serialize_value(ReferenceValue::Null);
let result = serialize_value(ReferenceValueLeaf::Null.into());
let value = deserialize_value(result);
assert_eq!(value, crate::schema::OwnedValue::Null);
let result = serialize_value(ReferenceValue::Str("hello, world"));
let result = serialize_value(ReferenceValueLeaf::Str("hello, world").into());
let value = deserialize_value(result);
assert_eq!(
value,
crate::schema::OwnedValue::Str(String::from("hello, world"))
);
let result = serialize_value(ReferenceValue::U64(123));
let result = serialize_value(ReferenceValueLeaf::U64(123).into());
let value = deserialize_value(result);
assert_eq!(value, crate::schema::OwnedValue::U64(123));
let result = serialize_value(ReferenceValue::I64(-123));
let result = serialize_value(ReferenceValueLeaf::I64(-123).into());
let value = deserialize_value(result);
assert_eq!(value, crate::schema::OwnedValue::I64(-123));
let result = serialize_value(ReferenceValue::F64(123.3845));
let result = serialize_value(ReferenceValueLeaf::F64(123.3845).into());
let value = deserialize_value(result);
assert_eq!(value, crate::schema::OwnedValue::F64(123.3845));
let result = serialize_value(ReferenceValue::Bool(false));
let result = serialize_value(ReferenceValueLeaf::Bool(false).into());
let value = deserialize_value(result);
assert_eq!(value, crate::schema::OwnedValue::Bool(false));
let result = serialize_value(ReferenceValue::Date(DateTime::from_timestamp_micros(100)));
let result =
serialize_value(ReferenceValueLeaf::Date(DateTime::from_timestamp_micros(100)).into());
let value = deserialize_value(result);
assert_eq!(
value,
@@ -845,7 +874,7 @@ mod tests {
);
let facet = Facet::from_text("/hello/world").unwrap();
let result = serialize_value(ReferenceValue::Facet(&facet));
let result = serialize_value(ReferenceValueLeaf::Facet(&facet).into());
let value = deserialize_value(result);
assert_eq!(value, crate::schema::OwnedValue::Facet(facet));
@@ -853,7 +882,7 @@ mod tests {
text: "hello, world".to_string(),
tokens: vec![Token::default(), Token::default()],
};
let result = serialize_value(ReferenceValue::PreTokStr(&pre_tok_str));
let result = serialize_value(ReferenceValueLeaf::PreTokStr(&pre_tok_str).into());
let value = deserialize_value(result);
assert_eq!(value, crate::schema::OwnedValue::PreTokStr(pre_tok_str));
}
@@ -953,6 +982,25 @@ mod tests {
assert_eq!(value, crate::schema::OwnedValue::Object(expected_object));
}
#[test]
fn test_json_compat() {
let data = [
8, 123, 34, 107, 101, 121, 97, 58, 34, 58, 34, 98, 108, 117, 98, 34, 44, 34, 118, 97,
108, 115, 34, 58, 123, 34, 104, 101, 121, 34, 58, 34, 104, 111, 34, 125, 125,
]
.to_vec();
let expected = json!({
"keya:": "blub",
"vals": {
"hey": "ho"
}
});
let expected_val: OwnedValue = expected.clone().into();
let value = deserialize_value(data);
assert_eq!(value, expected_val);
}
#[test]
fn test_nested_serialize() {
let mut object = serde_json::Map::new();
@@ -1025,11 +1073,11 @@ mod tests {
let mut expected_object = BTreeMap::new();
expected_object.insert(
"my-array".to_string(),
crate::schema::OwnedValue::Array(vec![crate::schema::OwnedValue::Array(vec![
crate::schema::OwnedValue::Array(vec![]),
crate::schema::OwnedValue::Array(vec![crate::schema::OwnedValue::Null]),
OwnedValue::Array(vec![OwnedValue::Array(vec![
OwnedValue::Array(vec![]),
OwnedValue::Array(vec![OwnedValue::Null]),
])]),
);
assert_eq!(value, crate::schema::OwnedValue::Object(expected_object));
assert_eq!(value, OwnedValue::Object(expected_object));
}
}

View File

@@ -8,6 +8,7 @@ use std::collections::{btree_map, hash_map, BTreeMap, HashMap};
use serde_json::Number;
use super::ReferenceValueLeaf;
use crate::schema::document::{
ArrayAccess, DeserializeError, Document, DocumentDeserialize, DocumentDeserializer,
ObjectAccess, ReferenceValue, Value, ValueDeserialize, ValueDeserializer, ValueVisitor,
@@ -21,20 +22,20 @@ impl<'a> Value<'a> for &'a serde_json::Value {
fn as_value(&self) -> ReferenceValue<'a, Self> {
match self {
serde_json::Value::Null => ReferenceValue::Null,
serde_json::Value::Bool(value) => ReferenceValue::Bool(*value),
serde_json::Value::Null => ReferenceValueLeaf::Null.into(),
serde_json::Value::Bool(value) => ReferenceValueLeaf::Bool(*value).into(),
serde_json::Value::Number(number) => {
if let Some(val) = number.as_i64() {
ReferenceValue::I64(val)
ReferenceValueLeaf::I64(val).into()
} else if let Some(val) = number.as_u64() {
ReferenceValue::U64(val)
ReferenceValueLeaf::U64(val).into()
} else if let Some(val) = number.as_f64() {
ReferenceValue::F64(val)
ReferenceValueLeaf::F64(val).into()
} else {
panic!("Unsupported serde_json number {number}");
}
}
serde_json::Value::String(val) => ReferenceValue::Str(val),
serde_json::Value::String(val) => ReferenceValueLeaf::Str(val).into(),
serde_json::Value::Array(elements) => ReferenceValue::Array(elements.iter()),
serde_json::Value::Object(object) => {
ReferenceValue::Object(JsonObjectIter(object.iter()))
@@ -77,7 +78,7 @@ impl ValueDeserialize for serde_json::Value {
}
fn visit_bool(&self, val: bool) -> Result<Self::Value, DeserializeError> {
Ok(serde_json::Value::Bool(val.into()))
Ok(serde_json::Value::Bool(val))
}
fn visit_array<'de, A>(&self, mut access: A) -> Result<Self::Value, DeserializeError>

View File

@@ -116,6 +116,7 @@
//!
//! ```
//! use tantivy::schema::document::ReferenceValue;
//! use tantivy::schema::document::ReferenceValueLeaf;
//! use tantivy::schema::{Value};
//!
//! #[derive(Debug)]
@@ -141,9 +142,9 @@
//! fn as_value(&self) -> ReferenceValue<'a, Self> {
//! // We can support any type that Tantivy itself supports.
//! match self {
//! MyCustomValue::String(val) => ReferenceValue::Str(val),
//! MyCustomValue::Float(val) => ReferenceValue::F64(*val),
//! MyCustomValue::Bool(val) => ReferenceValue::Bool(*val),
//! MyCustomValue::String(val) => ReferenceValue::Leaf(ReferenceValueLeaf::Str(*val)),
//! MyCustomValue::Float(val) => ReferenceValue::Leaf(ReferenceValueLeaf::F64(*val)),
//! MyCustomValue::Bool(val) => ReferenceValue::Leaf(ReferenceValueLeaf::Bool(*val)),
//! }
//! }
//!
@@ -170,7 +171,7 @@ pub use self::de::{
pub use self::default_document::{DocParsingError, TantivyDocument};
pub use self::owned_value::OwnedValue;
pub(crate) use self::se::BinaryDocumentSerializer;
pub use self::value::{ReferenceValue, Value};
pub use self::value::{ReferenceValue, ReferenceValueLeaf, Value};
use super::*;
/// The core trait representing a document within the index.
@@ -253,8 +254,9 @@ pub(crate) mod type_codes {
pub const DATE_CODE: u8 = 5;
pub const F64_CODE: u8 = 6;
pub const EXT_CODE: u8 = 7;
// Replaced by the `OBJECT_CODE`.
// -- pub const JSON_OBJ_CODE: u8 = 8;
#[deprecated]
pub const JSON_OBJ_CODE: u8 = 8; // Replaced by the `OBJECT_CODE`.
pub const BOOL_CODE: u8 = 9;
pub const IP_CODE: u8 = 10;
pub const NULL_CODE: u8 = 11;

View File

@@ -8,6 +8,7 @@ use serde::de::{MapAccess, SeqAccess};
use time::format_description::well_known::Rfc3339;
use time::OffsetDateTime;
use super::ReferenceValueLeaf;
use crate::schema::document::{
ArrayAccess, DeserializeError, ObjectAccess, ReferenceValue, Value, ValueDeserialize,
ValueDeserializer, ValueVisitor,
@@ -62,17 +63,17 @@ impl<'a> Value<'a> for &'a OwnedValue {
fn as_value(&self) -> ReferenceValue<'a, Self> {
match self {
OwnedValue::Null => ReferenceValue::Null,
OwnedValue::Str(val) => ReferenceValue::Str(val),
OwnedValue::PreTokStr(val) => ReferenceValue::PreTokStr(val),
OwnedValue::U64(val) => ReferenceValue::U64(*val),
OwnedValue::I64(val) => ReferenceValue::I64(*val),
OwnedValue::F64(val) => ReferenceValue::F64(*val),
OwnedValue::Bool(val) => ReferenceValue::Bool(*val),
OwnedValue::Date(val) => ReferenceValue::Date(*val),
OwnedValue::Facet(val) => ReferenceValue::Facet(val),
OwnedValue::Bytes(val) => ReferenceValue::Bytes(val),
OwnedValue::IpAddr(val) => ReferenceValue::IpAddr(*val),
OwnedValue::Null => ReferenceValueLeaf::Null.into(),
OwnedValue::Str(val) => ReferenceValueLeaf::Str(val).into(),
OwnedValue::PreTokStr(val) => ReferenceValueLeaf::PreTokStr(val).into(),
OwnedValue::U64(val) => ReferenceValueLeaf::U64(*val).into(),
OwnedValue::I64(val) => ReferenceValueLeaf::I64(*val).into(),
OwnedValue::F64(val) => ReferenceValueLeaf::F64(*val).into(),
OwnedValue::Bool(val) => ReferenceValueLeaf::Bool(*val).into(),
OwnedValue::Date(val) => ReferenceValueLeaf::Date(*val).into(),
OwnedValue::Facet(val) => ReferenceValueLeaf::Facet(val).into(),
OwnedValue::Bytes(val) => ReferenceValueLeaf::Bytes(val).into(),
OwnedValue::IpAddr(val) => ReferenceValueLeaf::IpAddr(*val).into(),
OwnedValue::Array(array) => ReferenceValue::Array(array.iter()),
OwnedValue::Object(object) => ReferenceValue::Object(ObjectMapIter(object.iter())),
}
@@ -264,17 +265,19 @@ impl<'de> serde::Deserialize<'de> for OwnedValue {
impl<'a, V: Value<'a>> From<ReferenceValue<'a, V>> for OwnedValue {
fn from(val: ReferenceValue<'a, V>) -> OwnedValue {
match val {
ReferenceValue::Null => OwnedValue::Null,
ReferenceValue::Str(val) => OwnedValue::Str(val.to_string()),
ReferenceValue::U64(val) => OwnedValue::U64(val),
ReferenceValue::I64(val) => OwnedValue::I64(val),
ReferenceValue::F64(val) => OwnedValue::F64(val),
ReferenceValue::Date(val) => OwnedValue::Date(val),
ReferenceValue::Facet(val) => OwnedValue::Facet(val.clone()),
ReferenceValue::Bytes(val) => OwnedValue::Bytes(val.to_vec()),
ReferenceValue::IpAddr(val) => OwnedValue::IpAddr(val),
ReferenceValue::Bool(val) => OwnedValue::Bool(val),
ReferenceValue::PreTokStr(val) => OwnedValue::PreTokStr(val.clone()),
ReferenceValue::Leaf(leaf) => match leaf {
ReferenceValueLeaf::Null => OwnedValue::Null,
ReferenceValueLeaf::Str(val) => OwnedValue::Str(val.to_string()),
ReferenceValueLeaf::U64(val) => OwnedValue::U64(val),
ReferenceValueLeaf::I64(val) => OwnedValue::I64(val),
ReferenceValueLeaf::F64(val) => OwnedValue::F64(val),
ReferenceValueLeaf::Date(val) => OwnedValue::Date(val),
ReferenceValueLeaf::Facet(val) => OwnedValue::Facet(val.clone()),
ReferenceValueLeaf::Bytes(val) => OwnedValue::Bytes(val.to_vec()),
ReferenceValueLeaf::IpAddr(val) => OwnedValue::IpAddr(val),
ReferenceValueLeaf::Bool(val) => OwnedValue::Bool(val),
ReferenceValueLeaf::PreTokStr(val) => OwnedValue::PreTokStr(val.clone()),
},
ReferenceValue::Array(val) => {
OwnedValue::Array(val.map(|v| v.as_value().into()).collect())
}
@@ -366,7 +369,7 @@ impl From<BTreeMap<String, OwnedValue>> for OwnedValue {
fn can_be_rfc3339_date_time(text: &str) -> bool {
if let Some(&first_byte) = text.as_bytes().first() {
if (b'0'..=b'9').contains(&first_byte) {
if first_byte.is_ascii_digit() {
return true;
}
}

View File

@@ -5,6 +5,7 @@ use std::io::Write;
use columnar::MonotonicallyMappableToU128;
use common::{f64_to_u64, BinarySerializable, VInt};
use super::{OwnedValue, ReferenceValueLeaf};
use crate::schema::document::{type_codes, Document, ReferenceValue, Value};
use crate::schema::Schema;
@@ -39,10 +40,10 @@ where W: Write
let mut serializer = BinaryValueSerializer::new(self.writer);
match value_access.as_value() {
ReferenceValue::PreTokStr(pre_tokenized_text) => {
serializer.serialize_value(ReferenceValue::Str::<
&'_ crate::schema::OwnedValue,
>(&pre_tokenized_text.text))?;
ReferenceValue::Leaf(ReferenceValueLeaf::PreTokStr(pre_tokenized_text)) => {
serializer.serialize_value(ReferenceValue::Leaf::<&'_ OwnedValue>(
ReferenceValueLeaf::Str(&pre_tokenized_text.text),
))?;
}
_ => {
serializer.serialize_value(value_access.as_value())?;
@@ -90,59 +91,61 @@ where W: Write
V: Value<'a>,
{
match value {
ReferenceValue::Null => self.write_type_code(type_codes::NULL_CODE),
ReferenceValue::Str(val) => {
self.write_type_code(type_codes::TEXT_CODE)?;
ReferenceValue::Leaf(leaf) => match leaf {
ReferenceValueLeaf::Null => self.write_type_code(type_codes::NULL_CODE),
ReferenceValueLeaf::Str(val) => {
self.write_type_code(type_codes::TEXT_CODE)?;
let temp_val = Cow::Borrowed(val);
temp_val.serialize(self.writer)
}
ReferenceValue::U64(val) => {
self.write_type_code(type_codes::U64_CODE)?;
let temp_val = Cow::Borrowed(val);
temp_val.serialize(self.writer)
}
ReferenceValueLeaf::U64(val) => {
self.write_type_code(type_codes::U64_CODE)?;
val.serialize(self.writer)
}
ReferenceValue::I64(val) => {
self.write_type_code(type_codes::I64_CODE)?;
val.serialize(self.writer)
}
ReferenceValueLeaf::I64(val) => {
self.write_type_code(type_codes::I64_CODE)?;
val.serialize(self.writer)
}
ReferenceValue::F64(val) => {
self.write_type_code(type_codes::F64_CODE)?;
val.serialize(self.writer)
}
ReferenceValueLeaf::F64(val) => {
self.write_type_code(type_codes::F64_CODE)?;
f64_to_u64(val).serialize(self.writer)
}
ReferenceValue::Date(val) => {
self.write_type_code(type_codes::DATE_CODE)?;
val.serialize(self.writer)
}
ReferenceValue::Facet(val) => {
self.write_type_code(type_codes::HIERARCHICAL_FACET_CODE)?;
f64_to_u64(val).serialize(self.writer)
}
ReferenceValueLeaf::Date(val) => {
self.write_type_code(type_codes::DATE_CODE)?;
val.serialize(self.writer)
}
ReferenceValueLeaf::Facet(val) => {
self.write_type_code(type_codes::HIERARCHICAL_FACET_CODE)?;
val.serialize(self.writer)
}
ReferenceValue::Bytes(val) => {
self.write_type_code(type_codes::BYTES_CODE)?;
val.serialize(self.writer)
}
ReferenceValueLeaf::Bytes(val) => {
self.write_type_code(type_codes::BYTES_CODE)?;
let temp_val = Cow::Borrowed(val);
temp_val.serialize(self.writer)
}
ReferenceValue::IpAddr(val) => {
self.write_type_code(type_codes::IP_CODE)?;
let temp_val = Cow::Borrowed(val);
temp_val.serialize(self.writer)
}
ReferenceValueLeaf::IpAddr(val) => {
self.write_type_code(type_codes::IP_CODE)?;
val.to_u128().serialize(self.writer)
}
ReferenceValue::Bool(val) => {
self.write_type_code(type_codes::BOOL_CODE)?;
val.to_u128().serialize(self.writer)
}
ReferenceValueLeaf::Bool(val) => {
self.write_type_code(type_codes::BOOL_CODE)?;
val.serialize(self.writer)
}
ReferenceValue::PreTokStr(val) => {
self.write_type_code(type_codes::EXT_CODE)?;
self.write_type_code(type_codes::TOK_STR_EXT_CODE)?;
val.serialize(self.writer)
}
ReferenceValueLeaf::PreTokStr(val) => {
self.write_type_code(type_codes::EXT_CODE)?;
self.write_type_code(type_codes::TOK_STR_EXT_CODE)?;
val.serialize(self.writer)
}
val.serialize(self.writer)
}
},
ReferenceValue::Array(elements) => {
self.write_type_code(type_codes::ARRAY_CODE)?;
@@ -272,7 +275,7 @@ where W: Write
// as we could avoid writing the extra byte per key. But the gain is
// largely not worth it for the extra complexity it brings.
self.inner
.serialize_value(ReferenceValue::<'a, V>::Str(key))?;
.serialize_value(ReferenceValue::<'a, V>::Leaf(ReferenceValueLeaf::Str(key)))?;
self.inner.serialize_value(value)?;
self.actual_length += 1;
@@ -361,7 +364,7 @@ mod tests {
#[test]
fn test_simple_value_serialize() {
let result = serialize_value(ReferenceValue::Null);
let result = serialize_value(ReferenceValueLeaf::Null.into());
let expected = binary_repr!(
type_codes::NULL_CODE => (),
);
@@ -370,7 +373,7 @@ mod tests {
"Expected serialized value to match the binary representation"
);
let result = serialize_value(ReferenceValue::Str("hello, world"));
let result = serialize_value(ReferenceValueLeaf::Str("hello, world").into());
let expected = binary_repr!(
type_codes::TEXT_CODE => String::from("hello, world"),
);
@@ -379,7 +382,7 @@ mod tests {
"Expected serialized value to match the binary representation"
);
let result = serialize_value(ReferenceValue::U64(123));
let result = serialize_value(ReferenceValueLeaf::U64(123).into());
let expected = binary_repr!(
type_codes::U64_CODE => 123u64,
);
@@ -388,7 +391,7 @@ mod tests {
"Expected serialized value to match the binary representation"
);
let result = serialize_value(ReferenceValue::I64(-123));
let result = serialize_value(ReferenceValueLeaf::I64(-123).into());
let expected = binary_repr!(
type_codes::I64_CODE => -123i64,
);
@@ -397,7 +400,7 @@ mod tests {
"Expected serialized value to match the binary representation"
);
let result = serialize_value(ReferenceValue::F64(123.3845));
let result = serialize_value(ReferenceValueLeaf::F64(123.3845f64).into());
let expected = binary_repr!(
type_codes::F64_CODE => f64_to_u64(123.3845f64),
);
@@ -406,7 +409,7 @@ mod tests {
"Expected serialized value to match the binary representation"
);
let result = serialize_value(ReferenceValue::Bool(false));
let result = serialize_value(ReferenceValueLeaf::Bool(false).into());
let expected = binary_repr!(
type_codes::BOOL_CODE => false,
);
@@ -415,7 +418,7 @@ mod tests {
"Expected serialized value to match the binary representation"
);
let result = serialize_value(ReferenceValue::Date(DateTime::MAX));
let result = serialize_value(ReferenceValueLeaf::Date(DateTime::MAX).into());
let expected = binary_repr!(
type_codes::DATE_CODE => DateTime::MAX,
);
@@ -425,7 +428,7 @@ mod tests {
);
let facet = Facet::from_text("/hello/world").unwrap();
let result = serialize_value(ReferenceValue::Facet(&facet));
let result = serialize_value(ReferenceValueLeaf::Facet(&facet).into());
let expected = binary_repr!(
type_codes::HIERARCHICAL_FACET_CODE => Facet::from_text("/hello/world").unwrap(),
);
@@ -438,7 +441,7 @@ mod tests {
text: "hello, world".to_string(),
tokens: vec![Token::default(), Token::default()],
};
let result = serialize_value(ReferenceValue::PreTokStr(&pre_tok_str));
let result = serialize_value(ReferenceValueLeaf::PreTokStr(&pre_tok_str).into());
let expected = binary_repr!(
type_codes::EXT_CODE, type_codes::TOK_STR_EXT_CODE => pre_tok_str,
);

View File

@@ -21,108 +21,81 @@ pub trait Value<'a>: Send + Sync + Debug {
#[inline]
/// Returns if the value is `null` or not.
fn is_null(&self) -> bool {
matches!(self.as_value(), ReferenceValue::Null)
matches!(
self.as_value(),
ReferenceValue::Leaf(ReferenceValueLeaf::Null)
)
}
#[inline]
/// If the Value is a String, returns the associated str. Returns None otherwise.
fn as_leaf(&self) -> Option<ReferenceValueLeaf<'a>> {
if let ReferenceValue::Leaf(val) = self.as_value() {
Some(val)
} else {
None
}
}
#[inline]
/// If the Value is a String, returns the associated str. Returns None otherwise.
fn as_str(&self) -> Option<&'a str> {
if let ReferenceValue::Str(val) = self.as_value() {
Some(val)
} else {
None
}
self.as_leaf().and_then(|leaf| leaf.as_str())
}
#[inline]
/// If the Value is a u64, returns the associated u64. Returns None otherwise.
fn as_u64(&self) -> Option<u64> {
if let ReferenceValue::U64(val) = self.as_value() {
Some(val)
} else {
None
}
self.as_leaf().and_then(|leaf| leaf.as_u64())
}
#[inline]
/// If the Value is a i64, returns the associated i64. Returns None otherwise.
fn as_i64(&self) -> Option<i64> {
if let ReferenceValue::I64(val) = self.as_value() {
Some(val)
} else {
None
}
self.as_leaf().and_then(|leaf| leaf.as_i64())
}
#[inline]
/// If the Value is a f64, returns the associated f64. Returns None otherwise.
fn as_f64(&self) -> Option<f64> {
if let ReferenceValue::F64(val) = self.as_value() {
Some(val)
} else {
None
}
self.as_leaf().and_then(|leaf| leaf.as_f64())
}
#[inline]
/// If the Value is a datetime, returns the associated datetime. Returns None otherwise.
fn as_datetime(&self) -> Option<DateTime> {
if let ReferenceValue::Date(val) = self.as_value() {
Some(val)
} else {
None
}
self.as_leaf().and_then(|leaf| leaf.as_datetime())
}
#[inline]
/// If the Value is a IP address, returns the associated IP. Returns None otherwise.
fn as_ip_addr(&self) -> Option<Ipv6Addr> {
if let ReferenceValue::IpAddr(val) = self.as_value() {
Some(val)
} else {
None
}
self.as_leaf().and_then(|leaf| leaf.as_ip_addr())
}
#[inline]
/// If the Value is a bool, returns the associated bool. Returns None otherwise.
fn as_bool(&self) -> Option<bool> {
if let ReferenceValue::Bool(val) = self.as_value() {
Some(val)
} else {
None
}
self.as_leaf().and_then(|leaf| leaf.as_bool())
}
#[inline]
/// If the Value is a pre-tokenized string, returns the associated string. Returns None
/// otherwise.
fn as_pre_tokenized_text(&self) -> Option<&'a PreTokenizedString> {
if let ReferenceValue::PreTokStr(val) = self.as_value() {
Some(val)
} else {
None
}
self.as_leaf().and_then(|leaf| leaf.as_pre_tokenized_text())
}
#[inline]
/// If the Value is a bytes value, returns the associated set of bytes. Returns None otherwise.
fn as_bytes(&self) -> Option<&'a [u8]> {
if let ReferenceValue::Bytes(val) = self.as_value() {
Some(val)
} else {
None
}
self.as_leaf().and_then(|leaf| leaf.as_bytes())
}
#[inline]
/// If the Value is a facet, returns the associated facet. Returns None otherwise.
fn as_facet(&self) -> Option<&'a Facet> {
if let ReferenceValue::Facet(val) = self.as_value() {
Some(val)
} else {
None
}
self.as_leaf().and_then(|leaf| leaf.as_facet())
}
#[inline]
@@ -158,11 +131,9 @@ pub trait Value<'a>: Send + Sync + Debug {
}
}
/// A enum representing a value for tantivy to index.
#[derive(Clone, Debug, PartialEq)]
pub enum ReferenceValue<'a, V>
where V: Value<'a> + ?Sized
{
/// A enum representing a leaf value for tantivy to index.
#[derive(Clone, Copy, Debug, PartialEq)]
pub enum ReferenceValueLeaf<'a> {
/// A null value.
Null,
/// The str type is used for any text information.
@@ -185,15 +156,32 @@ where V: Value<'a> + ?Sized
Bool(bool),
/// Pre-tokenized str type,
PreTokStr(&'a PreTokenizedString),
/// A an array containing multiple values.
Array(V::ArrayIter),
/// A nested / dynamic object.
Object(V::ObjectIter),
}
impl<'a, V> ReferenceValue<'a, V>
where V: Value<'a>
{
impl<'a, T: Value<'a> + ?Sized> From<ReferenceValueLeaf<'a>> for ReferenceValue<'a, T> {
#[inline]
fn from(value: ReferenceValueLeaf<'a>) -> Self {
match value {
ReferenceValueLeaf::Null => ReferenceValue::Leaf(ReferenceValueLeaf::Null),
ReferenceValueLeaf::Str(val) => ReferenceValue::Leaf(ReferenceValueLeaf::Str(val)),
ReferenceValueLeaf::U64(val) => ReferenceValue::Leaf(ReferenceValueLeaf::U64(val)),
ReferenceValueLeaf::I64(val) => ReferenceValue::Leaf(ReferenceValueLeaf::I64(val)),
ReferenceValueLeaf::F64(val) => ReferenceValue::Leaf(ReferenceValueLeaf::F64(val)),
ReferenceValueLeaf::Date(val) => ReferenceValue::Leaf(ReferenceValueLeaf::Date(val)),
ReferenceValueLeaf::Facet(val) => ReferenceValue::Leaf(ReferenceValueLeaf::Facet(val)),
ReferenceValueLeaf::Bytes(val) => ReferenceValue::Leaf(ReferenceValueLeaf::Bytes(val)),
ReferenceValueLeaf::IpAddr(val) => {
ReferenceValue::Leaf(ReferenceValueLeaf::IpAddr(val))
}
ReferenceValueLeaf::Bool(val) => ReferenceValue::Leaf(ReferenceValueLeaf::Bool(val)),
ReferenceValueLeaf::PreTokStr(val) => {
ReferenceValue::Leaf(ReferenceValueLeaf::PreTokStr(val))
}
}
}
}
impl<'a> ReferenceValueLeaf<'a> {
#[inline]
/// Returns if the value is `null` or not.
pub fn is_null(&self) -> bool {
@@ -300,11 +288,105 @@ where V: Value<'a>
None
}
}
}
/// A enum representing a value for tantivy to index.
#[derive(Clone, Debug, PartialEq)]
pub enum ReferenceValue<'a, V>
where V: Value<'a> + ?Sized
{
/// A null value.
Leaf(ReferenceValueLeaf<'a>),
/// A an array containing multiple values.
Array(V::ArrayIter),
/// A nested / dynamic object.
Object(V::ObjectIter),
}
impl<'a, V> ReferenceValue<'a, V>
where V: Value<'a>
{
#[inline]
/// Returns if the value is `null` or not.
pub fn is_null(&self) -> bool {
matches!(self, Self::Leaf(ReferenceValueLeaf::Null))
}
#[inline]
/// If the Value is a leaf, returns the associated leaf. Returns None otherwise.
pub fn as_leaf(&self) -> Option<&ReferenceValueLeaf<'a>> {
if let Self::Leaf(val) = self {
Some(val)
} else {
None
}
}
#[inline]
/// If the Value is a String, returns the associated str. Returns None otherwise.
pub fn as_str(&self) -> Option<&'a str> {
self.as_leaf().and_then(|leaf| leaf.as_str())
}
#[inline]
/// If the Value is a u64, returns the associated u64. Returns None otherwise.
pub fn as_u64(&self) -> Option<u64> {
self.as_leaf().and_then(|leaf| leaf.as_u64())
}
#[inline]
/// If the Value is a i64, returns the associated i64. Returns None otherwise.
pub fn as_i64(&self) -> Option<i64> {
self.as_leaf().and_then(|leaf| leaf.as_i64())
}
#[inline]
/// If the Value is a f64, returns the associated f64. Returns None otherwise.
pub fn as_f64(&self) -> Option<f64> {
self.as_leaf().and_then(|leaf| leaf.as_f64())
}
#[inline]
/// If the Value is a datetime, returns the associated datetime. Returns None otherwise.
pub fn as_datetime(&self) -> Option<DateTime> {
self.as_leaf().and_then(|leaf| leaf.as_datetime())
}
#[inline]
/// If the Value is a IP address, returns the associated IP. Returns None otherwise.
pub fn as_ip_addr(&self) -> Option<Ipv6Addr> {
self.as_leaf().and_then(|leaf| leaf.as_ip_addr())
}
#[inline]
/// If the Value is a bool, returns the associated bool. Returns None otherwise.
pub fn as_bool(&self) -> Option<bool> {
self.as_leaf().and_then(|leaf| leaf.as_bool())
}
#[inline]
/// If the Value is a pre-tokenized string, returns the associated string. Returns None
/// otherwise.
pub fn as_pre_tokenized_text(&self) -> Option<&'a PreTokenizedString> {
self.as_leaf().and_then(|leaf| leaf.as_pre_tokenized_text())
}
#[inline]
/// If the Value is a bytes value, returns the associated set of bytes. Returns None otherwise.
pub fn as_bytes(&self) -> Option<&'a [u8]> {
self.as_leaf().and_then(|leaf| leaf.as_bytes())
}
#[inline]
/// If the Value is a facet, returns the associated facet. Returns None otherwise.
pub fn as_facet(&self) -> Option<&'a Facet> {
self.as_leaf().and_then(|leaf| leaf.as_facet())
}
#[inline]
/// Returns true if the Value is an array.
pub fn is_array(&self) -> bool {
matches!(self, Self::Object(_))
matches!(self, Self::Array(_))
}
#[inline]

View File

@@ -93,6 +93,7 @@ impl Type {
}
/// Returns a 1 byte code used to identify the type.
#[inline]
pub fn to_code(&self) -> u8 {
*self as u8
}
@@ -115,6 +116,7 @@ impl Type {
/// Interprets a 1byte code as a type.
/// Returns `None` if the code is invalid.
#[inline]
pub fn from_code(code: u8) -> Option<Self> {
match code {
b's' => Some(Type::Str),

View File

@@ -278,6 +278,7 @@ fn locate_splitting_dots(field_path: &str) -> Vec<usize> {
impl Schema {
/// Return the `FieldEntry` associated with a `Field`.
#[inline]
pub fn get_field_entry(&self, field: Field) -> &FieldEntry {
&self.0.fields[field.field_id() as usize]
}

View File

@@ -3,7 +3,7 @@ use std::hash::{Hash, Hasher};
use std::net::Ipv6Addr;
use std::{fmt, str};
use columnar::MonotonicallyMappableToU128;
use columnar::{MonotonicallyMappableToU128, MonotonicallyMappableToU64};
use super::date_time_options::DATE_TIME_PRECISION_INDEXED;
use super::Field;
@@ -170,6 +170,18 @@ impl Term {
self.set_bytes(val.to_u64().to_be_bytes().as_ref());
}
pub(crate) fn append_type_and_fast_value<T: FastValue>(&mut self, val: T) {
self.0.push(T::to_type().to_code());
let value = if T::to_type() == Type::Date {
DateTime::from_u64(val.to_u64())
.truncate(DATE_TIME_PRECISION_INDEXED)
.to_u64()
} else {
val.to_u64()
};
self.0.extend(value.to_be_bytes().as_ref());
}
/// Sets a `Ipv6Addr` value in the term.
pub fn set_ip_addr(&mut self, val: Ipv6Addr) {
self.set_bytes(val.to_u128().to_be_bytes().as_ref());

View File

@@ -1,3 +1,59 @@
//! [`SnippetGenerator`]
//! Generates a text snippet for a given document, and some highlighted parts inside it.
//! Imagine you doing a text search in a document
//! and want to show a preview of where in the document the search terms occur,
//! along with some surrounding text to give context, and the search terms highlighted.
//!
//! [`SnippetGenerator`] serves this purpose.
//! It scans a document and constructs a snippet, which consists of sections where the search terms
//! have been found, stitched together with "..." in between sections if necessary.
//!
//! ## Example
//!
//! ```rust
//! # use tantivy::query::QueryParser;
//! # use tantivy::schema::{Schema, TEXT};
//! # use tantivy::{doc, Index};
//! use tantivy::snippet::SnippetGenerator;
//!
//! # fn main() -> tantivy::Result<()> {
//! # let mut schema_builder = Schema::builder();
//! # let text_field = schema_builder.add_text_field("text", TEXT);
//! # let schema = schema_builder.build();
//! # let index = Index::create_in_ram(schema);
//! # let mut index_writer = index.writer_with_num_threads(1, 20_000_000)?;
//! # let doc = doc!(text_field => r#"Comme je descendais des Fleuves impassibles,
//! # Je ne me sentis plus guidé par les haleurs :
//! # Des Peaux-Rouges criards les avaient pris pour cibles,
//! # Les ayant cloués nus aux poteaux de couleurs.
//! #
//! # J'étais insoucieux de tous les équipages,
//! # Porteur de blés flamands ou de cotons anglais.
//! # Quand avec mes haleurs ont fini ces tapages,
//! # Les Fleuves m'ont laissé descendre où je voulais.
//! # "#);
//! # index_writer.add_document(doc.clone())?;
//! # index_writer.commit()?;
//! # let query_parser = QueryParser::for_index(&index, vec![text_field]);
//! // ...
//! let query = query_parser.parse_query("haleurs flamands").unwrap();
//! # let reader = index.reader()?;
//! # let searcher = reader.searcher();
//! let mut snippet_generator = SnippetGenerator::create(&searcher, &*query, text_field)?;
//! snippet_generator.set_max_num_chars(100);
//! let snippet = snippet_generator.snippet_from_doc(&doc);
//! let snippet_html: String = snippet.to_html();
//! assert_eq!(snippet_html, "Comme je descendais des Fleuves impassibles,\n Je ne me sentis plus guidé par les <b>haleurs</b> :\n Des");
//! # Ok(())
//! # }
//! ```
//!
//! You can also specify the maximum number of characters for the snippets generated with the
//! `set_max_num_chars` method. By default, this limit is set to 150.
//!
//! SnippetGenerator needs to be created from the `Searcher` and the query, and the field on which
//! the `SnippetGenerator` should generate the snippets.
use std::cmp::Ordering;
use std::collections::{BTreeMap, BTreeSet};
use std::ops::Range;
@@ -16,7 +72,7 @@ const DEFAULT_SNIPPET_PREFIX: &str = "<b>";
const DEFAULT_SNIPPET_POSTFIX: &str = "</b>";
#[derive(Debug)]
pub struct FragmentCandidate {
pub(crate) struct FragmentCandidate {
score: Score,
start_offset: usize,
stop_offset: usize,
@@ -256,7 +312,7 @@ fn is_sorted(mut it: impl Iterator<Item = usize>) -> bool {
/// # use tantivy::query::QueryParser;
/// # use tantivy::schema::{Schema, TEXT};
/// # use tantivy::{doc, Index};
/// use tantivy::SnippetGenerator;
/// use tantivy::snippet::SnippetGenerator;
///
/// # fn main() -> tantivy::Result<()> {
/// # let mut schema_builder = Schema::builder();
@@ -346,7 +402,7 @@ impl SnippetGenerator {
})
}
/// Sets a maximum number of chars.
/// Sets a maximum number of chars. Default is 150.
pub fn set_max_num_chars(&mut self, max_num_chars: usize) {
self.max_num_chars = max_num_chars;
}
@@ -398,8 +454,9 @@ mod tests {
use super::{collapse_overlapped_ranges, search_fragments, select_best_fragment_combination};
use crate::query::QueryParser;
use crate::schema::{IndexRecordOption, Schema, TextFieldIndexing, TextOptions, TEXT};
use crate::snippet::SnippetGenerator;
use crate::tokenizer::{NgramTokenizer, SimpleTokenizer};
use crate::{Index, SnippetGenerator};
use crate::Index;
const TEST_TEXT: &str = r#"Rust is a systems programming language sponsored by
Mozilla which describes it as a "safe, concurrent, practical language", supporting functional and

View File

@@ -1,5 +1,5 @@
//! The term dictionary main role is to associate the sorted [`Term`s](crate::Term) to
//! a [`TermInfo`](crate::postings::TermInfo) struct that contains some meta-information
//! a [`TermInfo`] struct that contains some meta-information
//! about the term.
//!
//! Internally, the term dictionary relies on the `fst` crate to store
@@ -16,8 +16,7 @@
//! `f64`-terms are transformed to `u64` using a mapping that preserve order, and are then treated
//! as `u64`.
//!
//! A second datastructure makes it possible to access a
//! [`TermInfo`](crate::postings::TermInfo).
//! A second datastructure makes it possible to access a [`TermInfo`].
#[cfg(not(feature = "quickwit"))]
mod fst_termdict;

View File

@@ -14,7 +14,6 @@ common = {version= "0.6", path="../common", package="tantivy-common"}
tantivy-fst = "0.4"
# experimental gives us access to Decompressor::upper_bound
zstd = { version = "0.13", features = ["experimental"] }
tracing = "0.1"
[dev-dependencies]
proptest = "1"

View File

@@ -89,15 +89,21 @@ Note: as the SSTable does not support redundant keys, there is no ambiguity betw
### SSTFooter
```
+-------+-------+-----+-------------+---------+---------+
| Block | Block | ... | IndexOffset | NumTerm | Version |
+-------+-------+-----+-------------+---------+---------+
|----( # of blocks)---|
+-------+-------+-----+------------------+------------+-------------+---------+---------+
| Block | Block | ... | FirstLayerOffset | LayerCount | IndexOffset | NumTerm | Version |
+-------+-------+-----+------------------+------------+-------------+---------+---------+
|----(# of blocks)----|
```
- Block(SSTBlock): uses IndexValue for its Values format
- FirstLayerOffset(u64): Offset between the start of the footer and the start of the top level index
- LayerCount(u32): Number of layers of index (min 1) ## TODO do we want to use 0 as a marker for no layers? It makes small sstables 12 bytes more compact (the 0u32 would alias with the "end of sstable marker")
- IndexOffset(u64): Offset to the start of the SSTFooter
- NumTerm(u64): number of terms in the sstable
- Version(u32): Currently equal to 2
- Version(u32): Currently equal to 3
Blocks referencing the main table and block referencing the index itself are encoded the same way and
are not directly differentiated. Offsets in blocks referencing the index are relative to the start of
the footer, blocks referencing the main table are relative to the start of that table.
### IndexValue
```

View File

@@ -20,6 +20,7 @@ where W: io::Write
// Only here to avoid allocations.
stateless_buffer: Vec<u8>,
block_len: usize,
compress: bool,
}
impl<W, TValueWriter> DeltaWriter<W, TValueWriter>
@@ -34,6 +35,18 @@ where
value_writer: TValueWriter::default(),
stateless_buffer: Vec::new(),
block_len: BLOCK_LEN,
compress: true,
}
}
pub fn new_no_compression(wrt: W) -> Self {
DeltaWriter {
block: Vec::with_capacity(BLOCK_LEN * 2),
write: CountingWriter::wrap(BufWriter::new(wrt)),
value_writer: TValueWriter::default(),
stateless_buffer: Vec::new(),
block_len: BLOCK_LEN,
compress: false,
}
}
@@ -53,7 +66,7 @@ where
let block_len = buffer.len() + self.block.len();
if block_len > 2048 {
if block_len > 2048 && self.compress {
buffer.extend_from_slice(&self.block);
self.block.clear();

View File

@@ -3,7 +3,6 @@ use std::io;
use std::marker::PhantomData;
use std::ops::{Bound, RangeBounds};
use std::sync::Arc;
use tracing::instrument;
use common::file_slice::FileSlice;
use common::{BinarySerializable, OwnedBytes};
@@ -95,7 +94,6 @@ impl<TSSTable: SSTable> Dictionary<TSSTable> {
Ok(TSSTable::delta_reader(data))
}
#[instrument(skip_all)]
pub(crate) async fn sstable_delta_reader_block_async(
&self,
block_addr: BlockAddr,
@@ -130,52 +128,86 @@ impl<TSSTable: SSTable> Dictionary<TSSTable> {
key_range: impl RangeBounds<[u8]>,
limit: Option<u64>,
) -> FileSlice {
let first_block_id = match key_range.start_bound() {
Bound::Included(key) | Bound::Excluded(key) => {
let Some(first_block_id) = self.sstable_index.locate_with_key(key) else {
return FileSlice::empty();
};
Some(first_block_id)
}
// we don't perform great when limit is set to a large value, and sometime we use u64::MAX
// as a marker for no limit, so we'd better capture that.
// (not great means we decode up to the whole bottom layer index, which can take dozens of
// ms on a 100m term dictionary)
let limit = limit.filter(|limit| *limit != u64::MAX);
// TODO replace unwraps with proper error handling
let start_key = match key_range.start_bound() {
Bound::Included(key) | Bound::Excluded(key) => key,
Bound::Unbounded => &[],
};
let end_key = match key_range.end_bound() {
Bound::Included(key) | Bound::Excluded(key) => Some(key),
Bound::Unbounded => None,
};
let last_block_id = match key_range.end_bound() {
Bound::Included(key) | Bound::Excluded(key) => self.sstable_index.locate_with_key(key),
Bound::Unbounded => None,
};
let start_bound = if let Some(first_block_id) = first_block_id {
let Some(block_addr) = self.sstable_index.get_block(first_block_id) else {
let bounds = if let Some(limit) = limit {
let mut sstable_iterator = self.sstable_index.iterate_from_key(start_key).unwrap();
let Some(start_block) = sstable_iterator.value() else {
// range_start is after end of table
return FileSlice::empty();
};
Bound::Included(block_addr.byte_range.start)
} else {
Bound::Unbounded
};
if let Some(end_key) = end_key {
if sstable_iterator.key().unwrap() >= end_key {
// the start and end keys are in the same block, return just that block
return self.sstable_slice.slice(start_block.byte_range.clone());
}
}
let start_bound = start_block.byte_range.start;
let last_block_id = if let Some(limit) = limit {
let second_block_id = first_block_id.map(|id| id + 1).unwrap_or(0);
if let Some(block_addr) = self.sstable_index.get_block(second_block_id) {
let ordinal_limit = block_addr.first_ordinal + limit;
let last_block_limit = self.sstable_index.locate_with_ord(ordinal_limit);
if let Some(last_block_id) = last_block_id {
Some(last_block_id.min(last_block_limit))
sstable_iterator.advance().unwrap();
let Some(second_block) = sstable_iterator.value() else {
// we reached the end of the sstable, return everything from start_bound
return self.sstable_slice.slice(start_bound..);
};
let mut end_bound = second_block.byte_range.end;
if let Some(end_key) = end_key {
if sstable_iterator.key().unwrap() >= end_key {
return self.sstable_slice.slice(start_bound..end_bound);
}
}
let target_ord = second_block.first_ordinal + limit;
while sstable_iterator.advance().unwrap() {
let block = sstable_iterator.value().unwrap();
if block.first_ordinal >= target_ord {
break;
}
end_bound = block.byte_range.end;
if let Some(end_key) = end_key {
if sstable_iterator.key().unwrap() >= end_key {
break;
}
}
}
let start_bound = Bound::Included(start_bound);
let end_bound = Bound::Excluded(end_bound);
(start_bound, end_bound)
} else {
let Some(start_block) = self.sstable_index.get_block_with_key(start_key).unwrap()
else {
// range_start is after end of table
return FileSlice::empty();
};
let start_bound = Bound::Included(start_block.byte_range.start);
let end_bound = if let Some(end_key) = end_key {
if let Some(end_block) = self.sstable_index.get_block_with_key(end_key).unwrap() {
Bound::Excluded(end_block.byte_range.end)
} else {
Some(last_block_limit)
Bound::Unbounded
}
} else {
last_block_id
}
} else {
last_block_id
Bound::Unbounded
};
(start_bound, end_bound)
};
let end_bound = last_block_id
.and_then(|block_id| self.sstable_index.get_block(block_id))
.map(|block_addr| Bound::Excluded(block_addr.byte_range.end))
.unwrap_or(Bound::Unbounded);
self.sstable_slice.slice((start_bound, end_bound))
self.sstable_slice.slice(bounds)
}
/// Opens a `TermDictionary`.
@@ -183,29 +215,57 @@ impl<TSSTable: SSTable> Dictionary<TSSTable> {
let (main_slice, footer_len_slice) = term_dictionary_file.split_from_end(20);
let mut footer_len_bytes: OwnedBytes = footer_len_slice.read_bytes()?;
// let layer_count = u32::deserialize(&mut footer_len_bytes)?;
let index_offset = u64::deserialize(&mut footer_len_bytes)?;
let num_terms = u64::deserialize(&mut footer_len_bytes)?;
let version = u32::deserialize(&mut footer_len_bytes)?;
if version != crate::SSTABLE_VERSION {
return Err(io::Error::new(
io::ErrorKind::Other,
format!(
"Unsuported sstable version, expected {version}, found {}",
crate::SSTABLE_VERSION,
),
));
}
let (sstable_slice, index_slice) = main_slice.split(index_offset as usize);
let sstable_index_bytes = index_slice.read_bytes()?;
let sstable_index = SSTableIndex::load(sstable_index_bytes)
.map_err(|_| io::Error::new(io::ErrorKind::InvalidData, "SSTable corruption"))?;
Ok(Dictionary {
sstable_slice,
sstable_index,
num_terms,
phantom_data: PhantomData,
})
match version {
2 => {
// previous format, kept for backward compatibility
let sstable_index_bytes = index_slice.read_bytes()?;
// on the old format, the 1st layer necessarily start immediately, and there is
// only a single layer
let sstable_index =
SSTableIndex::load(sstable_index_bytes, 1, 0).map_err(|_| {
io::Error::new(io::ErrorKind::InvalidData, "SSTable corruption")
})?;
Ok(Dictionary {
sstable_slice,
sstable_index,
num_terms,
phantom_data: PhantomData,
})
}
3 => {
let sstable_index_bytes = index_slice.read_bytes()?;
let (sstable_index_bytes, mut v3_footer_bytes) = sstable_index_bytes.rsplit(12);
let first_layer_offset = v3_footer_bytes.read_u64();
let layer_count = v3_footer_bytes.read_u32();
let sstable_index = SSTableIndex::load(
sstable_index_bytes,
layer_count,
first_layer_offset as usize,
)
.map_err(|_| io::Error::new(io::ErrorKind::InvalidData, "SSTable corruption"))?;
Ok(Dictionary {
sstable_slice,
sstable_index,
num_terms,
phantom_data: PhantomData,
})
}
_ => {
return Err(io::Error::new(
io::ErrorKind::Other,
format!(
"Unsuported sstable version, expected {}, found {version}",
crate::SSTABLE_VERSION,
),
))
}
}
}
/// Creates a term dictionary from the supplied bytes.
@@ -229,69 +289,17 @@ impl<TSSTable: SSTable> Dictionary<TSSTable> {
self.num_terms as usize
}
/// Decode a DeltaReader up to key, returning the number of terms traversed
///
/// If the key was not found, returns Ok(None).
/// After calling this function, it is possible to call `DeltaReader::value` to get the
/// associated value.
#[instrument(skip_all)]
fn decode_up_to_key<K: AsRef<[u8]>>(
&self,
key: K,
sstable_delta_reader: &mut DeltaReader<TSSTable::ValueReader>,
) -> io::Result<Option<TermOrdinal>> {
let mut term_ord = 0;
let key_bytes = key.as_ref();
let mut ok_bytes = 0;
while sstable_delta_reader.advance()? {
let prefix_len = sstable_delta_reader.common_prefix_len();
let suffix = sstable_delta_reader.suffix();
match prefix_len.cmp(&ok_bytes) {
Ordering::Less => return Ok(None), // popped bytes already matched => too far
Ordering::Equal => (),
Ordering::Greater => {
// the ok prefix is less than current entry prefix => continue to next elem
term_ord += 1;
continue;
}
}
// we have ok_bytes byte of common prefix, check if this key adds more
for (key_byte, suffix_byte) in key_bytes[ok_bytes..].iter().zip(suffix) {
match suffix_byte.cmp(key_byte) {
Ordering::Less => break, // byte too small
Ordering::Equal => ok_bytes += 1, // new matching byte
Ordering::Greater => return Ok(None), // too far
}
}
if ok_bytes == key_bytes.len() {
if prefix_len + suffix.len() == ok_bytes {
return Ok(Some(term_ord));
} else {
// current key is a prefix of current element, not a match
return Ok(None);
}
}
term_ord += 1;
}
Ok(None)
}
/// Returns the ordinal associated with a given term.
pub fn term_ord<K: AsRef<[u8]>>(&self, key: K) -> io::Result<Option<TermOrdinal>> {
let key_bytes = key.as_ref();
let Some(block_addr) = self.sstable_index.get_block_with_key(key_bytes) else {
let Some(block_addr) = self.sstable_index.get_block_with_key(key_bytes)? else {
return Ok(None);
};
let first_ordinal = block_addr.first_ordinal;
let mut sstable_delta_reader = self.sstable_delta_reader_block(block_addr)?;
self.decode_up_to_key(key_bytes, &mut sstable_delta_reader)
decode_up_to_key(key_bytes, &mut sstable_delta_reader)
.map(|opt| opt.map(|ord| ord + first_ordinal))
}
@@ -306,7 +314,7 @@ impl<TSSTable: SSTable> Dictionary<TSSTable> {
/// the buffer may be modified.
pub fn ord_to_term(&self, ord: TermOrdinal, bytes: &mut Vec<u8>) -> io::Result<bool> {
// find block in which the term would be
let block_addr = self.sstable_index.get_block_with_ord(ord);
let block_addr = self.sstable_index.get_block_with_ord(ord)?;
let first_ordinal = block_addr.first_ordinal;
// then search inside that block only
@@ -324,7 +332,7 @@ impl<TSSTable: SSTable> Dictionary<TSSTable> {
/// Returns the number of terms in the dictionary.
pub fn term_info_from_ord(&self, term_ord: TermOrdinal) -> io::Result<Option<TSSTable::Value>> {
// find block in which the term would be
let block_addr = self.sstable_index.get_block_with_ord(term_ord);
let block_addr = self.sstable_index.get_block_with_ord(term_ord)?;
let first_ordinal = block_addr.first_ordinal;
// then search inside that block only
@@ -339,7 +347,7 @@ impl<TSSTable: SSTable> Dictionary<TSSTable> {
/// Lookups the value corresponding to the key.
pub fn get<K: AsRef<[u8]>>(&self, key: K) -> io::Result<Option<TSSTable::Value>> {
if let Some(block_addr) = self.sstable_index.get_block_with_key(key.as_ref()) {
if let Some(block_addr) = self.sstable_index.get_block_with_key(key.as_ref())? {
let sstable_reader = self.sstable_delta_reader_block(block_addr)?;
return self.do_get(key, sstable_reader);
}
@@ -347,9 +355,8 @@ impl<TSSTable: SSTable> Dictionary<TSSTable> {
}
/// Lookups the value corresponding to the key.
#[instrument(skip_all)]
pub async fn get_async<K: AsRef<[u8]>>(&self, key: K) -> io::Result<Option<TSSTable::Value>> {
if let Some(block_addr) = self.sstable_index.get_block_with_key(key.as_ref()) {
if let Some(block_addr) = self.sstable_index.get_block_with_key(key.as_ref())? {
let sstable_reader = self.sstable_delta_reader_block_async(block_addr).await?;
return self.do_get(key, sstable_reader);
}
@@ -361,7 +368,7 @@ impl<TSSTable: SSTable> Dictionary<TSSTable> {
key: K,
mut reader: DeltaReader<TSSTable::ValueReader>,
) -> io::Result<Option<TSSTable::Value>> {
if let Some(_ord) = self.decode_up_to_key(key, &mut reader)? {
if let Some(_ord) = decode_up_to_key(key, &mut reader)? {
Ok(Some(reader.value().clone()))
} else {
Ok(None)
@@ -398,6 +405,56 @@ impl<TSSTable: SSTable> Dictionary<TSSTable> {
}
}
/// Decode a DeltaReader up to key, returning the number of terms traversed
///
/// If the key was not found, returns Ok(None).
/// After calling this function, it is possible to call `DeltaReader::value` to get the
/// associated value.
pub(crate) fn decode_up_to_key<K: AsRef<[u8]>, TValueReader: crate::ValueReader>(
key: K,
sstable_delta_reader: &mut DeltaReader<TValueReader>,
) -> io::Result<Option<TermOrdinal>> {
let mut term_ord = 0;
let key_bytes = key.as_ref();
let mut ok_bytes = 0;
while sstable_delta_reader.advance()? {
let prefix_len = sstable_delta_reader.common_prefix_len();
let suffix = sstable_delta_reader.suffix();
match prefix_len.cmp(&ok_bytes) {
Ordering::Less => return Ok(None), // popped bytes already matched => too far
Ordering::Equal => (),
Ordering::Greater => {
// the ok prefix is less than current entry prefix => continue to next elem
term_ord += 1;
continue;
}
}
// we have ok_bytes byte of common prefix, check if this key adds more
for (key_byte, suffix_byte) in key_bytes[ok_bytes..].iter().zip(suffix) {
match suffix_byte.cmp(key_byte) {
Ordering::Less => break, // byte too small
Ordering::Equal => ok_bytes += 1, // new matching byte
Ordering::Greater => return Ok(None), // too far
}
}
if ok_bytes == key_bytes.len() {
if prefix_len + suffix.len() == ok_bytes {
return Ok(Some(term_ord));
} else {
// current key is a prefix of current element, not a match
return Ok(None);
}
}
term_ord += 1;
}
Ok(None)
}
#[cfg(test)]
mod tests {
use std::ops::Range;
@@ -463,8 +520,6 @@ mod tests {
let dictionary = Dictionary::<MonotonicU64SSTable>::open(slice).unwrap();
// if the last block is id 0, tests are meaningless
assert_ne!(dictionary.sstable_index.locate_with_ord(u64::MAX), 0);
assert_eq!(dictionary.num_terms(), 0x3ffff);
(dictionary, table)
}
@@ -473,7 +528,7 @@ mod tests {
fn test_ord_term_conversion() {
let (dic, slice) = make_test_sstable();
let block = dic.sstable_index.get_block_with_ord(100_000);
let block = dic.sstable_index.get_block_with_ord(100_000).unwrap();
slice.restrict(block.byte_range);
let mut res = Vec::new();
@@ -499,7 +554,11 @@ mod tests {
// end of a block
let ordinal = block.first_ordinal - 1;
let new_range = dic.sstable_index.get_block_with_ord(ordinal).byte_range;
let new_range = dic
.sstable_index
.get_block_with_ord(ordinal)
.unwrap()
.byte_range;
slice.restrict(new_range);
assert!(dic.ord_to_term(ordinal, &mut res).unwrap());
assert_eq!(res, format!("{ordinal:05X}").into_bytes());
@@ -509,7 +568,7 @@ mod tests {
// before first block
// 1st block must be loaded for key-related operations
let block = dic.sstable_index.get_block_with_ord(0);
let block = dic.sstable_index.get_block_with_ord(0).unwrap();
slice.restrict(block.byte_range);
assert!(dic.get(b"$$$").unwrap().is_none());
@@ -518,7 +577,11 @@ mod tests {
// after last block
// last block must be loaded for ord related operations
let ordinal = 0x40000 + 10;
let new_range = dic.sstable_index.get_block_with_ord(ordinal).byte_range;
let new_range = dic
.sstable_index
.get_block_with_ord(ordinal)
.unwrap()
.byte_range;
slice.restrict(new_range);
assert!(!dic.ord_to_term(ordinal, &mut res).unwrap());
assert!(dic.term_info_from_ord(ordinal).unwrap().is_none());
@@ -543,11 +606,13 @@ mod tests {
.sstable_index
.get_block_with_key(b"10000")
.unwrap()
.unwrap()
.byte_range;
let end = dic
.sstable_index
.get_block_with_key(b"18000")
.unwrap()
.unwrap()
.byte_range;
slice.restrict(start.start..end.end);

View File

@@ -1,6 +1,6 @@
use std::io::{self, Write};
use std::num::NonZeroU64;
use std::ops::Range;
use std::usize;
use merge::ValueMerger;
@@ -28,7 +28,13 @@ use crate::value::{RangeValueReader, RangeValueWriter};
pub type TermOrdinal = u64;
const DEFAULT_KEY_CAPACITY: usize = 50;
const SSTABLE_VERSION: u32 = 2;
const SSTABLE_VERSION: u32 = 3;
// TODO tune that value. Maybe it's too little?
#[cfg(not(test))]
const DEFAULT_MAX_ROOT_BLOCKS: NonZeroU64 = unsafe { NonZeroU64::new_unchecked(32) };
#[cfg(test)]
const DEFAULT_MAX_ROOT_BLOCKS: NonZeroU64 = unsafe { NonZeroU64::new_unchecked(1) };
/// Given two byte string returns the length of
/// the longest common prefix.
@@ -55,7 +61,7 @@ pub trait SSTable: Sized {
}
fn writer<W: io::Write>(wrt: W) -> Writer<W, Self::ValueWriter> {
Writer::new(wrt)
Writer::new(wrt, DEFAULT_MAX_ROOT_BLOCKS)
}
fn delta_reader(reader: OwnedBytes) -> DeltaReader<Self::ValueReader> {
@@ -178,6 +184,7 @@ where W: io::Write
delta_writer: DeltaWriter<W, TValueWriter>,
num_terms: u64,
first_ordinal_of_the_block: u64,
index_max_root_blocks: NonZeroU64,
}
impl<W, TValueWriter> Writer<W, TValueWriter>
@@ -190,17 +197,18 @@ where
/// TODO remove this function. (See Issue #1727)
#[doc(hidden)]
pub fn create(wrt: W) -> io::Result<Self> {
Ok(Self::new(wrt))
Ok(Self::new(wrt, DEFAULT_MAX_ROOT_BLOCKS))
}
/// Creates a new `TermDictionaryBuilder`.
pub fn new(wrt: W) -> Self {
pub fn new(wrt: W, index_max_root_blocks: NonZeroU64) -> Self {
Writer {
previous_key: Vec::with_capacity(DEFAULT_KEY_CAPACITY),
num_terms: 0u64,
index_builder: SSTableIndexBuilder::default(),
delta_writer: DeltaWriter::new(wrt),
first_ordinal_of_the_block: 0u64,
index_max_root_blocks,
}
}
@@ -302,10 +310,15 @@ where
// add a final empty block as an end marker
wrt.write_all(&0u32.to_le_bytes())?;
let offset = wrt.written_bytes();
let index_offset = wrt.written_bytes();
self.index_builder.serialize(&mut wrt)?;
wrt.write_all(&offset.to_le_bytes())?;
let (layer_count, layer_offset): (u32, u64) = self
.index_builder
.serialize(&mut wrt, self.index_max_root_blocks)?;
wrt.write_all(&layer_offset.to_le_bytes())?;
wrt.write_all(&layer_count.to_le_bytes())?;
wrt.write_all(&index_offset.to_le_bytes())?;
wrt.write_all(&self.num_terms.to_le_bytes())?;
SSTABLE_VERSION.serialize(&mut wrt)?;
@@ -389,9 +402,11 @@ mod test {
0, // compression
1, 0, 12, 0, 32, 17, 20, // index block
0, 0, 0, 0, // no more index block
0, 0, 0, 0, 0, 0, 0, 0, // first layer offset
1, 0, 0, 0, // layer count
16, 0, 0, 0, 0, 0, 0, 0, // index start offset
3, 0, 0, 0, 0, 0, 0, 0, // num term
2, 0, 0, 0, // version
3, 0, 0, 0, // version
]
);
let buffer = OwnedBytes::new(buffer);

View File

@@ -1,84 +1,192 @@
use std::io::{self, Write};
use std::ops::Range;
use tracing::instrument;
use common::OwnedBytes;
use crate::{common_prefix_len, SSTable, SSTableDataCorruption, TermOrdinal};
#[derive(Default, Debug, Clone)]
#[derive(Debug, Clone)]
pub struct SSTableIndex {
blocks: Vec<BlockMeta>,
root_blocks: Vec<BlockMeta>,
layer_count: u32,
index_bytes: OwnedBytes,
}
impl Default for SSTableIndex {
fn default() -> Self {
SSTableIndex {
root_blocks: Vec::new(),
layer_count: 1,
index_bytes: OwnedBytes::empty(),
}
}
}
impl SSTableIndex {
/// Load an index from its binary representation
pub fn load(data: OwnedBytes) -> Result<SSTableIndex, SSTableDataCorruption> {
let mut reader = IndexSSTable::reader(data);
let mut blocks = Vec::new();
pub fn load(
data: OwnedBytes,
layer_count: u32,
first_layer_offset: usize,
) -> Result<SSTableIndex, SSTableDataCorruption> {
let (index_bytes, first_layer_slice) = data.split(first_layer_offset);
let mut reader = IndexSSTable::reader(first_layer_slice);
let mut root_blocks = Vec::new();
while reader.advance().map_err(|_| SSTableDataCorruption)? {
blocks.push(BlockMeta {
root_blocks.push(BlockMeta {
last_key_or_greater: reader.key().to_vec(),
block_addr: reader.value().clone(),
});
}
Ok(SSTableIndex { blocks })
}
/// Get the [`BlockAddr`] of the requested block.
#[instrument]
pub(crate) fn get_block(&self, block_id: usize) -> Option<BlockAddr> {
self.blocks
.get(block_id)
.map(|block_meta| block_meta.block_addr.clone())
}
/// Get the block id of the block that would contain `key`.
///
/// Returns None if `key` is lexicographically after the last key recorded.
pub(crate) fn locate_with_key(&self, key: &[u8]) -> Option<usize> {
let pos = self
.blocks
.binary_search_by_key(&key, |block| &block.last_key_or_greater);
match pos {
Ok(pos) => Some(pos),
Err(pos) => {
if pos < self.blocks.len() {
Some(pos)
} else {
// after end of last block: no block matches
None
}
}
}
Ok(SSTableIndex {
root_blocks,
layer_count,
index_bytes,
// index_bytes: OwnedBytes::empty(),
})
}
/// Get the [`BlockAddr`] of the block that would contain `key`.
///
/// Returns None if `key` is lexicographically after the last key recorded.
#[instrument]
pub fn get_block_with_key(&self, key: &[u8]) -> Option<BlockAddr> {
self.locate_with_key(key).and_then(|id| self.get_block(id))
}
pub(crate) fn locate_with_ord(&self, ord: TermOrdinal) -> usize {
let pos = self
.blocks
.binary_search_by_key(&ord, |block| block.block_addr.first_ordinal);
match pos {
Ok(pos) => pos,
// Err(0) can't happen as the sstable starts with ordinal zero
Err(pos) => pos - 1,
}
pub fn get_block_with_key(&self, key: &[u8]) -> io::Result<Option<BlockAddr>> {
self.iterate_from_key(key).map(|iter| iter.value().cloned())
}
/// Get the [`BlockAddr`] of the block containing the `ord`-th term.
pub(crate) fn get_block_with_ord(&self, ord: TermOrdinal) -> BlockAddr {
// locate_with_ord always returns an index within range
self.get_block(self.locate_with_ord(ord)).unwrap()
pub fn get_block_with_ord(&self, ord: TermOrdinal) -> io::Result<BlockAddr> {
let pos = self
.root_blocks
.binary_search_by_key(&ord, |block| block.block_addr.first_ordinal);
let root_pos = match pos {
Ok(pos) => pos,
// Err(0) can't happen as the sstable starts with ordinal zero
Err(pos) => pos - 1,
};
if self.layer_count == 1 {
return Ok(self.root_blocks[root_pos].block_addr.clone());
}
let mut next_layer_block_addr = self.root_blocks[root_pos].block_addr.clone();
for _ in 1..self.layer_count {
let mut sstable_delta_reader = IndexSSTable::delta_reader(
self.index_bytes
.slice(next_layer_block_addr.byte_range.clone()),
);
while sstable_delta_reader.advance()? {
if sstable_delta_reader.value().first_ordinal > ord {
break;
}
next_layer_block_addr = sstable_delta_reader.value().clone();
}
}
Ok(next_layer_block_addr)
}
pub(crate) fn iterate_from_key(&self, key: &[u8]) -> io::Result<ReaderOrSlice<'_>> {
let root_pos = self
.root_blocks
.binary_search_by_key(&key, |block| &block.last_key_or_greater);
let root_pos = match root_pos {
Ok(pos) => pos,
Err(pos) => {
if pos < self.root_blocks.len() {
pos
} else {
// after end of last block: no block matches
return Ok(ReaderOrSlice::End);
}
}
};
let mut next_layer_block_addr = self.root_blocks[root_pos].block_addr.clone();
let mut last_delta_reader = None;
for _ in 1..self.layer_count {
// we don't enter this loop for 1 layer index
let mut sstable_delta_reader = IndexSSTable::delta_reader(
self.index_bytes.slice(next_layer_block_addr.byte_range),
);
crate::dictionary::decode_up_to_key(key, &mut sstable_delta_reader)?;
next_layer_block_addr = sstable_delta_reader.value().clone();
last_delta_reader = Some(sstable_delta_reader);
}
if let Some(delta_reader) = last_delta_reader {
// reconstruct the current key. We stopped either on the exact key, or just after
// either way, common_prefix_len is something that did not change between the
// last-key-before-target and the current pos, so those bytes must match the prefix of
// `key`. The next bytes can be obtained from the delta reader
let mut result_key = Vec::with_capacity(crate::DEFAULT_KEY_CAPACITY);
let common_prefix_len = delta_reader.common_prefix_len();
let suffix = delta_reader.suffix();
let new_len = delta_reader.common_prefix_len() + suffix.len();
result_key.resize(new_len, 0u8);
result_key[..common_prefix_len].copy_from_slice(&key[..common_prefix_len]);
result_key[common_prefix_len..].copy_from_slice(suffix);
let reader = crate::Reader {
key: result_key,
delta_reader,
};
Ok(ReaderOrSlice::Reader(reader))
} else {
// self.layer_count == 1, there is no lvl2 sstable to decode.
Ok(ReaderOrSlice::Iter(&self.root_blocks, root_pos))
}
}
}
pub(crate) enum ReaderOrSlice<'a> {
Reader(crate::Reader<crate::value::index::IndexValueReader>),
Iter(&'a [BlockMeta], usize),
End,
}
impl<'a> ReaderOrSlice<'a> {
pub fn advance(&mut self) -> Result<bool, SSTableDataCorruption> {
match self {
ReaderOrSlice::Reader(reader) => {
let res = reader.advance().map_err(|_| SSTableDataCorruption);
if !matches!(res, Ok(true)) {
*self = ReaderOrSlice::End;
}
res
}
ReaderOrSlice::Iter(slice, index) => {
*index += 1;
if *index < slice.len() {
Ok(true)
} else {
*self = ReaderOrSlice::End;
Ok(false)
}
}
ReaderOrSlice::End => Ok(false),
}
}
/// Get current key. Always Some(_) unless last call to advance returned something else than
/// Ok(true)
pub fn key(&self) -> Option<&[u8]> {
match self {
ReaderOrSlice::Reader(reader) => Some(reader.key()),
ReaderOrSlice::Iter(slice, index) => Some(&slice[*index].last_key_or_greater),
ReaderOrSlice::End => None,
}
}
/// Get current value. Always Some(_) unless last call to advance returned something else than
/// Ok(true)
pub fn value(&self) -> Option<&BlockAddr> {
match self {
ReaderOrSlice::Reader(reader) => Some(reader.value()),
ReaderOrSlice::Iter(slice, index) => Some(&slice[*index].block_addr),
ReaderOrSlice::End => None,
}
}
}
@@ -127,13 +235,13 @@ impl SSTableIndexBuilder {
/// try to find a shorter alternative to the last key of the last block
/// that is still smaller than the next key.
pub(crate) fn shorten_last_block_key_given_next_key(&mut self, next_key: &[u8]) {
if let Some(last_block) = self.index.blocks.last_mut() {
if let Some(last_block) = self.index.root_blocks.last_mut() {
find_shorter_str_in_between(&mut last_block.last_key_or_greater, next_key);
}
}
pub fn add_block(&mut self, last_key: &[u8], byte_range: Range<usize>, first_ordinal: u64) {
self.index.blocks.push(BlockMeta {
self.index.root_blocks.push(BlockMeta {
last_key_or_greater: last_key.to_vec(),
block_addr: BlockAddr {
byte_range,
@@ -142,31 +250,90 @@ impl SSTableIndexBuilder {
})
}
pub fn serialize<W: std::io::Write>(&self, wrt: W) -> io::Result<()> {
// we can't use a plain writer as it would generate an index
let mut sstable_writer = IndexSSTable::delta_writer(wrt);
pub fn serialize<W: std::io::Write>(
&self,
wrt: W,
index_max_root_blocks: std::num::NonZeroU64,
) -> io::Result<(u32, u64)> {
let index_max_root_blocks = index_max_root_blocks.get();
// in tests, set a smaller block size to stress-test
#[cfg(test)]
sstable_writer.set_block_len(16);
let mut wrt = common::CountingWriter::wrap(wrt);
let mut next_layer = write_sstable_layer(&mut wrt, &self.index.root_blocks, 0)?;
let mut previous_key = Vec::with_capacity(crate::DEFAULT_KEY_CAPACITY);
for block in self.index.blocks.iter() {
let keep_len = common_prefix_len(&previous_key, &block.last_key_or_greater);
let mut layer_count = 1;
let mut offset = 0;
while next_layer.len() as u64 > index_max_root_blocks {
offset = wrt.written_bytes();
layer_count += 1;
sstable_writer.write_suffix(keep_len, &block.last_key_or_greater[keep_len..]);
sstable_writer.write_value(&block.block_addr);
sstable_writer.flush_block_if_required()?;
previous_key.clear();
previous_key.extend_from_slice(&block.last_key_or_greater);
next_layer = write_sstable_layer(&mut wrt, &next_layer, offset as usize)?;
}
sstable_writer.flush_block()?;
sstable_writer.finish().write_all(&0u32.to_le_bytes())?;
Ok(())
Ok((layer_count, offset))
}
}
fn write_sstable_layer<W: std::io::Write>(
wrt: W,
layer_content: &[BlockMeta],
offset: usize,
) -> io::Result<Vec<BlockMeta>> {
// we can't use a plain writer as it would generate an index
// also disable compression, the index is small anyway, and it's the most costly part of
// opening that kind of sstable
let mut sstable_writer =
crate::DeltaWriter::<_, crate::value::index::IndexValueWriter>::new_no_compression(wrt);
// in tests, set a smaller block size to stress-test
#[cfg(test)]
sstable_writer.set_block_len(16);
let mut next_layer = Vec::new();
let mut previous_key = Vec::with_capacity(crate::DEFAULT_KEY_CAPACITY);
let mut first_ordinal = None;
for block in layer_content.iter() {
if first_ordinal.is_none() {
first_ordinal = Some(block.block_addr.first_ordinal);
}
let keep_len = common_prefix_len(&previous_key, &block.last_key_or_greater);
sstable_writer.write_suffix(keep_len, &block.last_key_or_greater[keep_len..]);
sstable_writer.write_value(&block.block_addr);
if let Some(range) = sstable_writer.flush_block_if_required()? {
let real_range = (range.start + offset)..(range.end + offset);
let block_meta = BlockMeta {
last_key_or_greater: block.last_key_or_greater.clone(),
block_addr: BlockAddr {
byte_range: real_range,
first_ordinal: first_ordinal.take().unwrap(),
},
};
next_layer.push(block_meta);
previous_key.clear();
} else {
previous_key.extend_from_slice(&block.last_key_or_greater);
previous_key.resize(block.last_key_or_greater.len(), 0u8);
previous_key[keep_len..].copy_from_slice(&block.last_key_or_greater[keep_len..]);
}
}
if let Some(range) = sstable_writer.flush_block()? {
if let Some(last_block) = layer_content.last() {
// not going here means an empty table (?!)
let real_range = (range.start + offset)..(range.end + offset);
let block_meta = BlockMeta {
last_key_or_greater: last_block.last_key_or_greater.clone(),
block_addr: BlockAddr {
byte_range: real_range,
first_ordinal: first_ordinal.take().unwrap(),
},
};
next_layer.push(block_meta);
}
}
sstable_writer.finish().write_all(&0u32.to_le_bytes())?;
Ok(next_layer)
}
/// SSTable representing an index
///
/// `last_key_or_greater` is used as the key, the value contains the
@@ -197,28 +364,77 @@ mod tests {
sstable_builder.add_block(b"ccc", 30..40, 10u64);
sstable_builder.add_block(b"dddd", 40..50, 15u64);
let mut buffer: Vec<u8> = Vec::new();
sstable_builder.serialize(&mut buffer).unwrap();
sstable_builder
.serialize(&mut buffer, crate::DEFAULT_MAX_ROOT_BLOCKS)
.unwrap();
let buffer = OwnedBytes::new(buffer);
let sstable_index = SSTableIndex::load(buffer).unwrap();
let sstable_index = SSTableIndex::load(buffer, 1, 0).unwrap();
assert_eq!(
sstable_index.get_block_with_key(b"bbbde"),
sstable_index.get_block_with_key(b"bbbde").unwrap(),
Some(BlockAddr {
first_ordinal: 10u64,
byte_range: 30..40
})
);
assert_eq!(sstable_index.locate_with_key(b"aa").unwrap(), 0);
assert_eq!(sstable_index.locate_with_key(b"aaa").unwrap(), 0);
assert_eq!(sstable_index.locate_with_key(b"aab").unwrap(), 1);
assert_eq!(sstable_index.locate_with_key(b"ccc").unwrap(), 2);
assert!(sstable_index.locate_with_key(b"e").is_none());
assert_eq!(
sstable_index
.get_block_with_key(b"aa")
.unwrap()
.unwrap()
.first_ordinal,
0
);
assert_eq!(
sstable_index
.get_block_with_key(b"aaa")
.unwrap()
.unwrap()
.first_ordinal,
0
);
assert_eq!(
sstable_index
.get_block_with_key(b"aab")
.unwrap()
.unwrap()
.first_ordinal,
5
);
assert_eq!(
sstable_index
.get_block_with_key(b"ccc")
.unwrap()
.unwrap()
.first_ordinal,
10
);
assert!(sstable_index.get_block_with_key(b"e").unwrap().is_none());
assert_eq!(sstable_index.locate_with_ord(0), 0);
assert_eq!(sstable_index.locate_with_ord(1), 0);
assert_eq!(sstable_index.locate_with_ord(4), 0);
assert_eq!(sstable_index.locate_with_ord(5), 1);
assert_eq!(sstable_index.locate_with_ord(100), 3);
assert_eq!(
sstable_index.get_block_with_ord(0).unwrap().first_ordinal,
0
);
assert_eq!(
sstable_index.get_block_with_ord(1).unwrap().first_ordinal,
0
);
assert_eq!(
sstable_index.get_block_with_ord(4).unwrap().first_ordinal,
0
);
assert_eq!(
sstable_index.get_block_with_ord(5).unwrap().first_ordinal,
5
);
assert_eq!(
sstable_index.get_block_with_ord(6).unwrap().first_ordinal,
5
);
assert_eq!(
sstable_index.get_block_with_ord(100).unwrap().first_ordinal,
15
);
}
#[test]
@@ -229,10 +445,12 @@ mod tests {
sstable_builder.add_block(b"ccc", 30..40, 10u64);
sstable_builder.add_block(b"dddd", 40..50, 15u64);
let mut buffer: Vec<u8> = Vec::new();
sstable_builder.serialize(&mut buffer).unwrap();
sstable_builder
.serialize(&mut buffer, crate::DEFAULT_MAX_ROOT_BLOCKS)
.unwrap();
buffer[2] = 9u8;
let buffer = OwnedBytes::new(buffer);
let data_corruption_err = SSTableIndex::load(buffer).err().unwrap();
let data_corruption_err = SSTableIndex::load(buffer, 1, 0).err().unwrap();
assert!(matches!(data_corruption_err, SSTableDataCorruption));
}

View File

@@ -110,7 +110,7 @@ where
Bound::Included(key) | Bound::Excluded(key) => self
.term_dict
.sstable_index
.get_block_with_key(key)
.get_block_with_key(key)?
.map(|block| block.first_ordinal)
.unwrap_or(0),
Bound::Unbounded => 0,

View File

@@ -3,6 +3,10 @@ use std::io;
use crate::value::{deserialize_vint_u64, ValueReader, ValueWriter};
use crate::{vint, BlockAddr};
// TODO define a LazyIndexValueReader?
// one which keeps state could be useful for ord_to_block fns,
// one which doesn't at all woud be perfect for term_to_block fns
// pending bench to asses real impact
#[derive(Default)]
pub(crate) struct IndexValueReader {
vals: Vec<BlockAddr>,

View File

@@ -4,7 +4,6 @@ use std::mem;
use super::{Addr, MemoryArena};
use crate::fastcpy::fast_short_slice_copy;
use crate::memory_arena::store;
use crate::UnorderedId;
/// Returns the actual memory size in bytes
/// required to create a table with a given capacity.
@@ -24,9 +23,8 @@ type HashType = u64;
/// The `value_addr` also points to an address in the memory arena.
#[derive(Copy, Clone)]
struct KeyValue {
key_value_addr: Addr,
pub(crate) key_value_addr: Addr,
hash: HashType,
unordered_id: UnorderedId,
}
impl Default for KeyValue {
@@ -34,14 +32,13 @@ impl Default for KeyValue {
KeyValue {
key_value_addr: Addr::null_pointer(),
hash: 0,
unordered_id: UnorderedId::default(),
}
}
}
impl KeyValue {
#[inline]
fn is_empty(self) -> bool {
fn is_empty(&self) -> bool {
self.key_value_addr.is_null()
}
#[inline]
@@ -61,7 +58,7 @@ impl KeyValue {
/// or copying the key as long as there is no insert.
pub struct ArenaHashMap {
table: Vec<KeyValue>,
memory_arena: MemoryArena,
pub memory_arena: MemoryArena,
mask: usize,
len: usize,
}
@@ -96,12 +93,12 @@ pub struct Iter<'a> {
}
impl<'a> Iterator for Iter<'a> {
type Item = (&'a [u8], Addr, UnorderedId);
type Item = (&'a [u8], Addr);
fn next(&mut self) -> Option<Self::Item> {
self.inner.next().map(move |kv| {
let (key, offset): (&'a [u8], Addr) = self.hashmap.get_key_value(kv.key_value_addr);
(key, offset, kv.unordered_id)
(key, offset)
})
}
}
@@ -207,16 +204,13 @@ impl ArenaHashMap {
}
#[inline]
fn set_bucket(&mut self, hash: HashType, key_value_addr: Addr, bucket: usize) -> UnorderedId {
let unordered_id = self.len as UnorderedId;
fn set_bucket(&mut self, hash: HashType, key_value_addr: Addr, bucket: usize) {
self.len += 1;
self.table[bucket] = KeyValue {
key_value_addr,
hash,
unordered_id,
};
unordered_id
}
#[inline]
@@ -290,14 +284,8 @@ impl ArenaHashMap {
/// If the key already as an associated value, then it will be passed
/// `Some(previous_value)`.
#[inline]
pub fn mutate_or_create<V>(
&mut self,
key: &[u8],
mut updater: impl FnMut(Option<V>) -> V,
) -> UnorderedId
where
V: Copy + 'static,
{
pub fn mutate_or_create<V>(&mut self, key: &[u8], mut updater: impl FnMut(Option<V>) -> V)
where V: Copy + 'static {
if self.is_saturated() {
self.resize();
}
@@ -320,14 +308,15 @@ impl ArenaHashMap {
store(&mut data[stop..], val);
}
return self.set_bucket(hash, key_addr, bucket);
self.set_bucket(hash, key_addr, bucket);
return;
}
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);
let new_v = updater(Some(v));
self.memory_arena.write_at(val_addr, new_v);
return kv.unordered_id;
return;
}
}
// This allows fetching the next bucket before the loop jmp
@@ -361,7 +350,7 @@ mod tests {
});
let mut vanilla_hash_map = HashMap::new();
let iter_values = hash_map.iter();
for (key, addr, _) in iter_values {
for (key, addr) in iter_values {
let val: u32 = hash_map.memory_arena.read(addr);
vanilla_hash_map.insert(key.to_owned(), val);
}
@@ -390,7 +379,7 @@ mod tests {
}
let mut terms_back: Vec<String> = hash_map
.iter()
.map(|(bytes, _, _)| String::from_utf8(bytes.to_vec()).unwrap())
.map(|(bytes, _)| String::from_utf8(bytes.to_vec()).unwrap())
.collect();
terms_back.sort();
terms.sort();

View File

@@ -148,6 +148,11 @@ impl MemoryArena {
self.get_page(addr.page_id())
.slice_from(addr.page_local_addr())
}
#[inline]
pub fn slice_from_mut(&mut self, addr: Addr) -> &mut [u8] {
self.get_page_mut(addr.page_id())
.slice_from_mut(addr.page_local_addr())
}
#[inline]
pub fn slice_mut(&mut self, addr: Addr, len: usize) -> &mut [u8] {
@@ -206,6 +211,10 @@ impl Page {
fn slice_from(&self, local_addr: usize) -> &[u8] {
&self.data[local_addr..]
}
#[inline]
fn slice_from_mut(&mut self, local_addr: usize) -> &mut [u8] {
&mut self.data[local_addr..]
}
#[inline]
fn slice_mut(&mut self, local_addr: usize, len: usize) -> &mut [u8] {

View File

@@ -2,7 +2,7 @@
//! ready for indexing. This is an seperate crate from tantivy, so implementors don't need to update
//! for each new tantivy version.
//!
//! To add support for a tokenizer, implement the [`Tokenizer`](crate::Tokenizer) trait.
//! To add support for a tokenizer, implement the [`Tokenizer`] trait.
//! Checkout the [tantivy repo](https://github.com/quickwit-oss/tantivy/tree/main/src/tokenizer) for some examples.
use std::borrow::{Borrow, BorrowMut};