mirror of
https://github.com/quickwit-oss/tantivy.git
synced 2025-12-28 04:52:55 +00:00
Compare commits
33 Commits
missing-sp
...
remove_dyn
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
6037cdfe7e | ||
|
|
ca20bfa776 | ||
|
|
faa706d804 | ||
|
|
850a0d7ae2 | ||
|
|
7fae4d98d7 | ||
|
|
bc36458334 | ||
|
|
8a71e00da3 | ||
|
|
e510f699c8 | ||
|
|
d25fc155b2 | ||
|
|
8ea97e7d6b | ||
|
|
0a726a0897 | ||
|
|
66ff53b0f4 | ||
|
|
d002698008 | ||
|
|
c838aa808b | ||
|
|
06850719dc | ||
|
|
5f23bb7e65 | ||
|
|
533ad99cd5 | ||
|
|
c7278b3258 | ||
|
|
6b403e3281 | ||
|
|
789cc8703e | ||
|
|
e5098d9fe8 | ||
|
|
f537334e4f | ||
|
|
e2aa5af075 | ||
|
|
02bebf4ff5 | ||
|
|
0274c982d5 | ||
|
|
74bf60b4f7 | ||
|
|
bf1449b22d | ||
|
|
111f25a8f7 | ||
|
|
019db10e8e | ||
|
|
7423f99719 | ||
|
|
f2f38c43ce | ||
|
|
71f43ace1d | ||
|
|
347614c841 |
@@ -58,7 +58,7 @@ arc-swap = "1.5.0"
|
||||
columnar = { version="0.1", path="./columnar", package ="tantivy-columnar" }
|
||||
sstable = { version="0.1", path="./sstable", package ="tantivy-sstable", optional = true }
|
||||
stacker = { version="0.1", path="./stacker", package ="tantivy-stacker" }
|
||||
tantivy-query-grammar = { version= "0.19.0", path="./query-grammar" }
|
||||
query-grammar = { version= "0.19.0", path="./query-grammar", package = "tantivy-query-grammar" }
|
||||
tantivy-bitpacker = { version= "0.3", path="./bitpacker" }
|
||||
common = { version= "0.5", path = "./common/", package = "tantivy-common" }
|
||||
tokenizer-api = { version="0.1", path="./tokenizer-api", package="tantivy-tokenizer-api" }
|
||||
@@ -77,6 +77,7 @@ test-log = "0.2.10"
|
||||
env_logger = "0.10.0"
|
||||
pprof = { version = "0.11.0", features = ["flamegraph", "criterion"] }
|
||||
futures = "0.3.21"
|
||||
paste = "1.0.11"
|
||||
|
||||
[dev-dependencies.fail]
|
||||
version = "0.5.0"
|
||||
|
||||
@@ -17,6 +17,7 @@ stacker = { path = "../stacker", package="tantivy-stacker"}
|
||||
sstable = { path = "../sstable", package = "tantivy-sstable" }
|
||||
common = { path = "../common", package = "tantivy-common" }
|
||||
tantivy-bitpacker = { version= "0.3", path = "../bitpacker/" }
|
||||
serde = "1.0.152"
|
||||
|
||||
[dev-dependencies]
|
||||
proptest = "1"
|
||||
|
||||
@@ -1,7 +1,6 @@
|
||||
# zero to one
|
||||
|
||||
* revisit line codec
|
||||
* removal of all rows of a column in the schema due to deletes
|
||||
* add columns from schema on merge
|
||||
* Plugging JSON
|
||||
* replug examples
|
||||
|
||||
@@ -36,7 +36,7 @@ impl BytesColumn {
|
||||
}
|
||||
|
||||
pub fn term_ords(&self, row_id: RowId) -> impl Iterator<Item = u64> + '_ {
|
||||
self.term_ord_column.values(row_id)
|
||||
self.term_ord_column.values_for_doc(row_id)
|
||||
}
|
||||
|
||||
/// Returns the column of ordinals
|
||||
|
||||
@@ -38,13 +38,15 @@ impl<T: MonotonicallyMappableToU64> Column<T> {
|
||||
}
|
||||
|
||||
impl<T: PartialOrd + Copy + Debug + Send + Sync + 'static> Column<T> {
|
||||
#[inline]
|
||||
pub fn get_cardinality(&self) -> Cardinality {
|
||||
self.idx.get_cardinality()
|
||||
}
|
||||
|
||||
pub fn num_docs(&self) -> RowId {
|
||||
match &self.idx {
|
||||
ColumnIndex::Full => self.values.num_vals() as u32,
|
||||
ColumnIndex::Empty { num_docs } => *num_docs,
|
||||
ColumnIndex::Full => self.values.num_vals(),
|
||||
ColumnIndex::Optional(optional_index) => optional_index.num_docs(),
|
||||
ColumnIndex::Multivalued(col_index) => {
|
||||
// The multivalued index contains all value start row_id,
|
||||
@@ -63,10 +65,10 @@ impl<T: PartialOrd + Copy + Debug + Send + Sync + 'static> Column<T> {
|
||||
}
|
||||
|
||||
pub fn first(&self, row_id: RowId) -> Option<T> {
|
||||
self.values(row_id).next()
|
||||
self.values_for_doc(row_id).next()
|
||||
}
|
||||
|
||||
pub fn values(&self, row_id: RowId) -> impl Iterator<Item = T> + '_ {
|
||||
pub fn values_for_doc(&self, row_id: RowId) -> impl Iterator<Item = T> + '_ {
|
||||
self.value_row_ids(row_id)
|
||||
.map(|value_row_id: RowId| self.values.get_val(value_row_id))
|
||||
}
|
||||
@@ -77,17 +79,17 @@ impl<T: PartialOrd + Copy + Debug + Send + Sync + 'static> Column<T> {
|
||||
&self,
|
||||
value_range: RangeInclusive<T>,
|
||||
selected_docid_range: Range<u32>,
|
||||
docids: &mut Vec<u32>,
|
||||
doc_ids: &mut Vec<u32>,
|
||||
) {
|
||||
// convert passed docid range to row id range
|
||||
let rowid_range = self.idx.docid_range_to_rowids(selected_docid_range.clone());
|
||||
|
||||
// Load rows
|
||||
self.values
|
||||
.get_row_ids_for_value_range(value_range, rowid_range, docids);
|
||||
.get_row_ids_for_value_range(value_range, rowid_range, doc_ids);
|
||||
// Convert rows to docids
|
||||
self.idx
|
||||
.select_batch_in_place(docids, selected_docid_range.start);
|
||||
.select_batch_in_place(selected_docid_range.start, doc_ids);
|
||||
}
|
||||
|
||||
/// Fils the output vector with the (possibly multiple values that are associated_with
|
||||
@@ -96,7 +98,7 @@ impl<T: PartialOrd + Copy + Debug + Send + Sync + 'static> Column<T> {
|
||||
/// This method clears the `output` vector.
|
||||
pub fn fill_vals(&self, row_id: RowId, output: &mut Vec<T>) {
|
||||
output.clear();
|
||||
output.extend(self.values(row_id));
|
||||
output.extend(self.values_for_doc(row_id));
|
||||
}
|
||||
|
||||
pub fn first_or_default_col(self, default_value: T) -> Arc<dyn ColumnValues<T>> {
|
||||
@@ -150,6 +152,7 @@ impl<T: PartialOrd + Debug + Send + Sync + Copy + 'static> ColumnValues<T>
|
||||
|
||||
fn num_vals(&self) -> u32 {
|
||||
match &self.column.idx {
|
||||
ColumnIndex::Empty { .. } => 0u32,
|
||||
ColumnIndex::Full => self.column.values.num_vals(),
|
||||
ColumnIndex::Optional(optional_idx) => optional_idx.num_docs(),
|
||||
ColumnIndex::Multivalued(multivalue_idx) => multivalue_idx.num_docs(),
|
||||
|
||||
@@ -7,9 +7,10 @@ use sstable::Dictionary;
|
||||
|
||||
use crate::column::{BytesColumn, Column};
|
||||
use crate::column_index::{serialize_column_index, SerializableColumnIndex};
|
||||
use crate::column_values::serialize::serialize_column_values_u128;
|
||||
use crate::column_values::u64_based::{serialize_u64_based_column_values, CodecType};
|
||||
use crate::column_values::{MonotonicallyMappableToU128, MonotonicallyMappableToU64};
|
||||
use crate::column_values::{
|
||||
load_u64_based_column_values, serialize_column_values_u128, serialize_u64_based_column_values,
|
||||
CodecType, MonotonicallyMappableToU128, MonotonicallyMappableToU64,
|
||||
};
|
||||
use crate::iterable::Iterable;
|
||||
use crate::StrColumn;
|
||||
|
||||
@@ -49,8 +50,7 @@ pub fn open_column_u64<T: MonotonicallyMappableToU64>(bytes: OwnedBytes) -> io::
|
||||
);
|
||||
let (column_index_data, column_values_data) = body.split(column_index_num_bytes as usize);
|
||||
let column_index = crate::column_index::open_column_index(column_index_data)?;
|
||||
let column_values =
|
||||
crate::column_values::u64_based::load_u64_based_column_values(column_values_data)?;
|
||||
let column_values = load_u64_based_column_values(column_values_data)?;
|
||||
Ok(Column {
|
||||
idx: column_index,
|
||||
values: column_values,
|
||||
|
||||
@@ -91,6 +91,7 @@ fn iter_num_values<'a>(
|
||||
return 0u32;
|
||||
};
|
||||
match column_index {
|
||||
ColumnIndex::Empty { .. } => 0u32,
|
||||
ColumnIndex::Full => 1,
|
||||
ColumnIndex::Optional(optional_index) => {
|
||||
u32::from(optional_index.contains(row_addr.row_id))
|
||||
|
||||
@@ -55,7 +55,7 @@ impl<'a> Iterable<RowId> for StackedOptionalIndex<'a> {
|
||||
Some(ColumnIndex::Multivalued(_)) => {
|
||||
panic!("No multivalued index is allowed when stacking column index");
|
||||
}
|
||||
None => Box::new(std::iter::empty()),
|
||||
None | Some(ColumnIndex::Empty { .. }) => Box::new(std::iter::empty()),
|
||||
};
|
||||
rows_it
|
||||
}),
|
||||
@@ -74,7 +74,9 @@ fn convert_column_opt_to_multivalued_index<'a>(
|
||||
num_rows: RowId,
|
||||
) -> Box<dyn Iterator<Item = RowId> + 'a> {
|
||||
match column_index_opt {
|
||||
None => Box::new(iter::repeat(0u32).take(num_rows as usize + 1)),
|
||||
None | Some(ColumnIndex::Empty { .. }) => {
|
||||
Box::new(iter::repeat(0u32).take(num_rows as usize + 1))
|
||||
}
|
||||
Some(ColumnIndex::Full) => Box::new(0..num_rows + 1),
|
||||
Some(ColumnIndex::Optional(optional_index)) => {
|
||||
Box::new(
|
||||
|
||||
@@ -14,6 +14,9 @@ use crate::{Cardinality, DocId, RowId};
|
||||
|
||||
#[derive(Clone)]
|
||||
pub enum ColumnIndex {
|
||||
Empty {
|
||||
num_docs: u32,
|
||||
},
|
||||
Full,
|
||||
Optional(OptionalIndex),
|
||||
/// In addition, at index num_rows, an extra value is added
|
||||
@@ -34,8 +37,10 @@ impl From<MultiValueIndex> for ColumnIndex {
|
||||
}
|
||||
|
||||
impl ColumnIndex {
|
||||
#[inline]
|
||||
pub fn get_cardinality(&self) -> Cardinality {
|
||||
match self {
|
||||
ColumnIndex::Empty { .. } => Cardinality::Optional,
|
||||
ColumnIndex::Full => Cardinality::Full,
|
||||
ColumnIndex::Optional(_) => Cardinality::Optional,
|
||||
ColumnIndex::Multivalued(_) => Cardinality::Multivalued,
|
||||
@@ -45,6 +50,7 @@ impl ColumnIndex {
|
||||
/// Returns true if and only if there are at least one value associated to the row.
|
||||
pub fn has_value(&self, doc_id: DocId) -> bool {
|
||||
match self {
|
||||
ColumnIndex::Empty { .. } => false,
|
||||
ColumnIndex::Full => true,
|
||||
ColumnIndex::Optional(optional_index) => optional_index.contains(doc_id),
|
||||
ColumnIndex::Multivalued(multivalued_index) => {
|
||||
@@ -55,6 +61,7 @@ impl ColumnIndex {
|
||||
|
||||
pub fn value_row_ids(&self, doc_id: DocId) -> Range<RowId> {
|
||||
match self {
|
||||
ColumnIndex::Empty { .. } => 0..0,
|
||||
ColumnIndex::Full => doc_id..doc_id + 1,
|
||||
ColumnIndex::Optional(optional_index) => {
|
||||
if let Some(val) = optional_index.rank_if_exists(doc_id) {
|
||||
@@ -69,6 +76,7 @@ impl ColumnIndex {
|
||||
|
||||
pub fn docid_range_to_rowids(&self, doc_id: Range<DocId>) -> Range<RowId> {
|
||||
match self {
|
||||
ColumnIndex::Empty { .. } => 0..0,
|
||||
ColumnIndex::Full => doc_id,
|
||||
ColumnIndex::Optional(optional_index) => {
|
||||
let row_start = optional_index.rank(doc_id.start);
|
||||
@@ -87,8 +95,11 @@ impl ColumnIndex {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn select_batch_in_place(&self, rank_ids: &mut Vec<RowId>, doc_id_start: DocId) {
|
||||
pub fn select_batch_in_place(&self, doc_id_start: DocId, rank_ids: &mut Vec<RowId>) {
|
||||
match self {
|
||||
ColumnIndex::Empty { .. } => {
|
||||
rank_ids.clear();
|
||||
}
|
||||
ColumnIndex::Full => {
|
||||
// No need to do anything:
|
||||
// value_idx and row_idx are the same.
|
||||
|
||||
@@ -5,8 +5,9 @@ use std::sync::Arc;
|
||||
|
||||
use common::OwnedBytes;
|
||||
|
||||
use crate::column_values::u64_based::CodecType;
|
||||
use crate::column_values::ColumnValues;
|
||||
use crate::column_values::{
|
||||
load_u64_based_column_values, serialize_u64_based_column_values, CodecType, ColumnValues,
|
||||
};
|
||||
use crate::iterable::Iterable;
|
||||
use crate::{DocId, RowId};
|
||||
|
||||
@@ -14,7 +15,7 @@ pub fn serialize_multivalued_index(
|
||||
multivalued_index: &dyn Iterable<RowId>,
|
||||
output: &mut impl Write,
|
||||
) -> io::Result<()> {
|
||||
crate::column_values::u64_based::serialize_u64_based_column_values(
|
||||
serialize_u64_based_column_values(
|
||||
multivalued_index,
|
||||
&[CodecType::Bitpacked, CodecType::Linear],
|
||||
output,
|
||||
@@ -23,8 +24,7 @@ pub fn serialize_multivalued_index(
|
||||
}
|
||||
|
||||
pub fn open_multivalued_index(bytes: OwnedBytes) -> io::Result<MultiValueIndex> {
|
||||
let start_index_column: Arc<dyn ColumnValues<RowId>> =
|
||||
crate::column_values::u64_based::load_u64_based_column_values(bytes)?;
|
||||
let start_index_column: Arc<dyn ColumnValues<RowId>> = load_u64_based_column_values(bytes)?;
|
||||
Ok(MultiValueIndex { start_index_column })
|
||||
}
|
||||
|
||||
@@ -83,13 +83,13 @@ impl MultiValueIndex {
|
||||
let mut cur_doc = docid_start;
|
||||
let mut last_doc = None;
|
||||
|
||||
assert!(self.start_index_column.get_val(docid_start) as u32 <= ranks[0]);
|
||||
assert!(self.start_index_column.get_val(docid_start) <= ranks[0]);
|
||||
|
||||
let mut write_doc_pos = 0;
|
||||
for i in 0..ranks.len() {
|
||||
let pos = ranks[i];
|
||||
loop {
|
||||
let end = self.start_index_column.get_val(cur_doc + 1) as u32;
|
||||
let end = self.start_index_column.get_val(cur_doc + 1);
|
||||
if end > pos {
|
||||
ranks[write_doc_pos] = cur_doc;
|
||||
write_doc_pos += if last_doc == Some(cur_doc) { 0 } else { 1 };
|
||||
|
||||
@@ -440,7 +440,7 @@ impl SerializedBlockMeta {
|
||||
|
||||
#[inline]
|
||||
fn is_sparse(num_rows_in_block: u32) -> bool {
|
||||
num_rows_in_block < DENSE_BLOCK_THRESHOLD as u32
|
||||
num_rows_in_block < DENSE_BLOCK_THRESHOLD
|
||||
}
|
||||
|
||||
fn deserialize_optional_index_block_metadatas(
|
||||
@@ -448,7 +448,7 @@ fn deserialize_optional_index_block_metadatas(
|
||||
num_rows: u32,
|
||||
) -> (Box<[BlockMeta]>, u32) {
|
||||
let num_blocks = data.len() / SERIALIZED_BLOCK_META_NUM_BYTES;
|
||||
let mut block_metas = Vec::with_capacity(num_blocks as usize + 1);
|
||||
let mut block_metas = Vec::with_capacity(num_blocks + 1);
|
||||
let mut start_byte_offset = 0;
|
||||
let mut non_null_rows_before_block = 0;
|
||||
for block_meta_bytes in data.chunks_exact(SERIALIZED_BLOCK_META_NUM_BYTES) {
|
||||
@@ -479,7 +479,7 @@ fn deserialize_optional_index_block_metadatas(
|
||||
block_variant,
|
||||
});
|
||||
start_byte_offset += block_variant.num_bytes_in_block();
|
||||
non_null_rows_before_block += num_non_null_rows as u32;
|
||||
non_null_rows_before_block += num_non_null_rows;
|
||||
}
|
||||
block_metas.resize(
|
||||
((num_rows + BLOCK_SIZE - 1) / BLOCK_SIZE) as usize,
|
||||
|
||||
@@ -32,7 +32,7 @@ pub const MINI_BLOCK_NUM_BYTES: usize = MINI_BLOCK_BITVEC_NUM_BYTES + MINI_BLOCK
|
||||
|
||||
/// Number of bytes in a dense block.
|
||||
pub const DENSE_BLOCK_NUM_BYTES: u32 =
|
||||
(ELEMENTS_PER_BLOCK as u32 / ELEMENTS_PER_MINI_BLOCK as u32) * MINI_BLOCK_NUM_BYTES as u32;
|
||||
(ELEMENTS_PER_BLOCK / ELEMENTS_PER_MINI_BLOCK as u32) * MINI_BLOCK_NUM_BYTES as u32;
|
||||
|
||||
pub struct DenseBlockCodec;
|
||||
|
||||
@@ -229,7 +229,7 @@ pub fn serialize_dense_codec(
|
||||
while block_id > current_block_id {
|
||||
let dense_mini_block = DenseMiniBlock {
|
||||
bitvec: block,
|
||||
rank: non_null_rows_before as u16,
|
||||
rank: non_null_rows_before,
|
||||
};
|
||||
output.write_all(&dense_mini_block.to_bytes())?;
|
||||
non_null_rows_before += block.count_ones() as u16;
|
||||
|
||||
@@ -37,7 +37,7 @@ proptest! {
|
||||
fn test_with_random_sets_simple() {
|
||||
let vals = 10..BLOCK_SIZE * 2;
|
||||
let mut out: Vec<u8> = Vec::new();
|
||||
serialize_optional_index(&vals.clone(), 100, &mut out).unwrap();
|
||||
serialize_optional_index(&vals, 100, &mut out).unwrap();
|
||||
let null_index = open_optional_index(OwnedBytes::new(out)).unwrap();
|
||||
let ranks: Vec<u32> = (65_472u32..65_473u32).collect();
|
||||
let els: Vec<u32> = ranks.iter().copied().map(|rank| rank + 10).collect();
|
||||
|
||||
135
columnar/src/column_values/bench.rs
Normal file
135
columnar/src/column_values/bench.rs
Normal file
@@ -0,0 +1,135 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use common::OwnedBytes;
|
||||
use rand::rngs::StdRng;
|
||||
use rand::{Rng, SeedableRng};
|
||||
use test::{self, Bencher};
|
||||
|
||||
use super::*;
|
||||
use crate::column_values::u64_based::*;
|
||||
|
||||
fn get_data() -> Vec<u64> {
|
||||
let mut rng = StdRng::seed_from_u64(2u64);
|
||||
let mut data: Vec<_> = (100..55000_u64)
|
||||
.map(|num| num + rng.gen::<u8>() as u64)
|
||||
.collect();
|
||||
data.push(99_000);
|
||||
data.insert(1000, 2000);
|
||||
data.insert(2000, 100);
|
||||
data.insert(3000, 4100);
|
||||
data.insert(4000, 100);
|
||||
data.insert(5000, 800);
|
||||
data
|
||||
}
|
||||
|
||||
fn compute_stats(vals: impl Iterator<Item = u64>) -> ColumnStats {
|
||||
let mut stats_collector = StatsCollector::default();
|
||||
for val in vals {
|
||||
stats_collector.collect(val);
|
||||
}
|
||||
stats_collector.stats()
|
||||
}
|
||||
|
||||
#[inline(never)]
|
||||
fn value_iter() -> impl Iterator<Item = u64> {
|
||||
0..20_000
|
||||
}
|
||||
fn get_reader_for_bench<Codec: ColumnCodec>(data: &[u64]) -> Codec::ColumnValues {
|
||||
let mut bytes = Vec::new();
|
||||
let stats = compute_stats(data.iter().cloned());
|
||||
let mut codec_serializer = Codec::estimator();
|
||||
for val in data {
|
||||
codec_serializer.collect(*val);
|
||||
}
|
||||
codec_serializer.serialize(&stats, Box::new(data.iter().copied()).as_mut(), &mut bytes);
|
||||
|
||||
Codec::load(OwnedBytes::new(bytes)).unwrap()
|
||||
}
|
||||
fn bench_get<Codec: ColumnCodec>(b: &mut Bencher, data: &[u64]) {
|
||||
let col = get_reader_for_bench::<Codec>(data);
|
||||
b.iter(|| {
|
||||
let mut sum = 0u64;
|
||||
for pos in value_iter() {
|
||||
let val = col.get_val(pos as u32);
|
||||
sum = sum.wrapping_add(val);
|
||||
}
|
||||
sum
|
||||
});
|
||||
}
|
||||
|
||||
#[inline(never)]
|
||||
fn bench_get_dynamic_helper(b: &mut Bencher, col: Arc<dyn ColumnValues>) {
|
||||
b.iter(|| {
|
||||
let mut sum = 0u64;
|
||||
for pos in value_iter() {
|
||||
let val = col.get_val(pos as u32);
|
||||
sum = sum.wrapping_add(val);
|
||||
}
|
||||
sum
|
||||
});
|
||||
}
|
||||
|
||||
fn bench_get_dynamic<Codec: ColumnCodec>(b: &mut Bencher, data: &[u64]) {
|
||||
let col = Arc::new(get_reader_for_bench::<Codec>(data));
|
||||
bench_get_dynamic_helper(b, col);
|
||||
}
|
||||
fn bench_create<Codec: ColumnCodec>(b: &mut Bencher, data: &[u64]) {
|
||||
let stats = compute_stats(data.iter().cloned());
|
||||
|
||||
let mut bytes = Vec::new();
|
||||
b.iter(|| {
|
||||
bytes.clear();
|
||||
let mut codec_serializer = Codec::estimator();
|
||||
for val in data.iter().take(1024) {
|
||||
codec_serializer.collect(*val);
|
||||
}
|
||||
|
||||
codec_serializer.serialize(&stats, Box::new(data.iter().copied()).as_mut(), &mut bytes)
|
||||
});
|
||||
}
|
||||
|
||||
#[bench]
|
||||
fn bench_fastfield_bitpack_create(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_create::<BitpackedCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_linearinterpol_create(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_create::<LinearCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_multilinearinterpol_create(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_create::<BlockwiseLinearCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_bitpack_get(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_get::<BitpackedCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_bitpack_get_dynamic(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_get_dynamic::<BitpackedCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_linearinterpol_get(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_get::<LinearCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_linearinterpol_get_dynamic(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_get_dynamic::<LinearCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_multilinearinterpol_get(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_get::<BlockwiseLinearCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_multilinearinterpol_get_dynamic(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_get_dynamic::<BlockwiseLinearCodec>(b, &data);
|
||||
}
|
||||
@@ -1,367 +0,0 @@
|
||||
use std::fmt::Debug;
|
||||
use std::marker::PhantomData;
|
||||
use std::ops::{Range, RangeInclusive};
|
||||
use std::sync::Arc;
|
||||
|
||||
use tantivy_bitpacker::minmax;
|
||||
|
||||
use crate::column_values::monotonic_mapping::StrictlyMonotonicFn;
|
||||
use crate::RowId;
|
||||
|
||||
/// `ColumnValues` provides access to a dense field column.
|
||||
///
|
||||
/// `Column` are just a wrapper over `ColumnValues` and a `ColumnIndex`.
|
||||
pub trait ColumnValues<T: PartialOrd = u64>: Send + Sync {
|
||||
/// Return the value associated with the given idx.
|
||||
///
|
||||
/// This accessor should return as fast as possible.
|
||||
///
|
||||
/// # Panics
|
||||
///
|
||||
/// May panic if `idx` is greater than the column length.
|
||||
fn get_val(&self, idx: u32) -> T;
|
||||
|
||||
/// Fills an output buffer with the fast field values
|
||||
/// associated with the `DocId` going from
|
||||
/// `start` to `start + output.len()`.
|
||||
///
|
||||
/// # Panics
|
||||
///
|
||||
/// Must panic if `start + output.len()` is greater than
|
||||
/// the segment's `maxdoc`.
|
||||
#[inline(always)]
|
||||
fn get_range(&self, start: u64, output: &mut [T]) {
|
||||
for (out, idx) in output.iter_mut().zip(start..) {
|
||||
*out = self.get_val(idx as u32);
|
||||
}
|
||||
}
|
||||
|
||||
/// Get the row ids of values which are in the provided value range.
|
||||
///
|
||||
/// Note that position == docid for single value fast fields
|
||||
#[inline(always)]
|
||||
fn get_row_ids_for_value_range(
|
||||
&self,
|
||||
value_range: RangeInclusive<T>,
|
||||
row_id_range: Range<RowId>,
|
||||
row_id_hits: &mut Vec<RowId>,
|
||||
) {
|
||||
let row_id_range = row_id_range.start..row_id_range.end.min(self.num_vals());
|
||||
for idx in row_id_range.start..row_id_range.end {
|
||||
let val = self.get_val(idx);
|
||||
if value_range.contains(&val) {
|
||||
row_id_hits.push(idx);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the minimum value for this fast field.
|
||||
///
|
||||
/// This min_value may not be exact.
|
||||
/// For instance, the min value does not take in account of possible
|
||||
/// deleted document. All values are however guaranteed to be higher than
|
||||
/// `.min_value()`.
|
||||
fn min_value(&self) -> T;
|
||||
|
||||
/// Returns the maximum value for this fast field.
|
||||
///
|
||||
/// This max_value may not be exact.
|
||||
/// For instance, the max value does not take in account of possible
|
||||
/// deleted document. All values are however guaranteed to be higher than
|
||||
/// `.max_value()`.
|
||||
fn max_value(&self) -> T;
|
||||
|
||||
/// The number of values in the column.
|
||||
fn num_vals(&self) -> u32;
|
||||
|
||||
/// Returns a iterator over the data
|
||||
fn iter<'a>(&'a self) -> Box<dyn Iterator<Item = T> + 'a> {
|
||||
Box::new((0..self.num_vals()).map(|idx| self.get_val(idx)))
|
||||
}
|
||||
}
|
||||
|
||||
impl<T: Copy + PartialOrd + Debug> ColumnValues<T> for Arc<dyn ColumnValues<T>> {
|
||||
#[inline(always)]
|
||||
fn get_val(&self, idx: u32) -> T {
|
||||
self.as_ref().get_val(idx)
|
||||
}
|
||||
|
||||
#[inline(always)]
|
||||
fn min_value(&self) -> T {
|
||||
self.as_ref().min_value()
|
||||
}
|
||||
|
||||
#[inline(always)]
|
||||
fn max_value(&self) -> T {
|
||||
self.as_ref().max_value()
|
||||
}
|
||||
|
||||
#[inline(always)]
|
||||
fn num_vals(&self) -> u32 {
|
||||
self.as_ref().num_vals()
|
||||
}
|
||||
|
||||
#[inline(always)]
|
||||
fn iter<'b>(&'b self) -> Box<dyn Iterator<Item = T> + 'b> {
|
||||
self.as_ref().iter()
|
||||
}
|
||||
|
||||
#[inline(always)]
|
||||
fn get_range(&self, start: u64, output: &mut [T]) {
|
||||
self.as_ref().get_range(start, output)
|
||||
}
|
||||
|
||||
fn get_row_ids_for_value_range(
|
||||
&self,
|
||||
value_range: RangeInclusive<T>,
|
||||
row_id_range: Range<RowId>,
|
||||
row_id_hits: &mut Vec<RowId>,
|
||||
) {
|
||||
self.as_ref().get_row_ids_for_value_range(value_range, row_id_range, row_id_hits)
|
||||
}
|
||||
}
|
||||
|
||||
/// VecColumn provides `Column` over a slice.
|
||||
pub struct VecColumn<'a, T = u64> {
|
||||
pub(crate) values: &'a [T],
|
||||
pub(crate) min_value: T,
|
||||
pub(crate) max_value: T,
|
||||
}
|
||||
|
||||
impl<'a, T: Copy + PartialOrd + Send + Sync + Debug> ColumnValues<T> for VecColumn<'a, T> {
|
||||
fn get_val(&self, position: u32) -> T {
|
||||
self.values[position as usize]
|
||||
}
|
||||
|
||||
fn iter(&self) -> Box<dyn Iterator<Item = T> + '_> {
|
||||
Box::new(self.values.iter().copied())
|
||||
}
|
||||
|
||||
fn min_value(&self) -> T {
|
||||
self.min_value
|
||||
}
|
||||
|
||||
fn max_value(&self) -> T {
|
||||
self.max_value
|
||||
}
|
||||
|
||||
fn num_vals(&self) -> u32 {
|
||||
self.values.len() as u32
|
||||
}
|
||||
|
||||
fn get_range(&self, start: u64, output: &mut [T]) {
|
||||
output.copy_from_slice(&self.values[start as usize..][..output.len()])
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, T: Copy + PartialOrd + Default, V> From<&'a V> for VecColumn<'a, T>
|
||||
where V: AsRef<[T]> + ?Sized
|
||||
{
|
||||
fn from(values: &'a V) -> Self {
|
||||
let values = values.as_ref();
|
||||
let (min_value, max_value) = minmax(values.iter().copied()).unwrap_or_default();
|
||||
Self {
|
||||
values,
|
||||
min_value,
|
||||
max_value,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
struct MonotonicMappingColumn<C, T, Input> {
|
||||
from_column: C,
|
||||
monotonic_mapping: T,
|
||||
_phantom: PhantomData<Input>,
|
||||
}
|
||||
|
||||
/// Creates a view of a column transformed by a strictly monotonic mapping. See
|
||||
/// [`StrictlyMonotonicFn`].
|
||||
///
|
||||
/// E.g. apply a gcd monotonic_mapping([100, 200, 300]) == [1, 2, 3]
|
||||
/// monotonic_mapping.mapping() is expected to be injective, and we should always have
|
||||
/// monotonic_mapping.inverse(monotonic_mapping.mapping(el)) == el
|
||||
///
|
||||
/// The inverse of the mapping is required for:
|
||||
/// `fn get_positions_for_value_range(&self, range: RangeInclusive<T>) -> Vec<u64> `
|
||||
/// The user provides the original value range and we need to monotonic map them in the same way the
|
||||
/// serialization does before calling the underlying column.
|
||||
///
|
||||
/// Note that when opening a codec, the monotonic_mapping should be the inverse of the mapping
|
||||
/// during serialization. And therefore the monotonic_mapping_inv when opening is the same as
|
||||
/// monotonic_mapping during serialization.
|
||||
pub fn monotonic_map_column<C, T, Input, Output>(
|
||||
from_column: C,
|
||||
monotonic_mapping: T,
|
||||
) -> impl ColumnValues<Output>
|
||||
where
|
||||
C: ColumnValues<Input>,
|
||||
T: StrictlyMonotonicFn<Input, Output> + Send + Sync,
|
||||
Input: PartialOrd + Debug + Send + Sync + Clone,
|
||||
Output: PartialOrd + Debug + Send + Sync + Clone,
|
||||
{
|
||||
MonotonicMappingColumn {
|
||||
from_column,
|
||||
monotonic_mapping,
|
||||
_phantom: PhantomData,
|
||||
}
|
||||
}
|
||||
|
||||
impl<C, T, Input, Output> ColumnValues<Output> for MonotonicMappingColumn<C, T, Input>
|
||||
where
|
||||
C: ColumnValues<Input>,
|
||||
T: StrictlyMonotonicFn<Input, Output> + Send + Sync,
|
||||
Input: PartialOrd + Send + Debug + Sync + Clone,
|
||||
Output: PartialOrd + Send + Debug + Sync + Clone,
|
||||
{
|
||||
#[inline]
|
||||
fn get_val(&self, idx: u32) -> Output {
|
||||
let from_val = self.from_column.get_val(idx);
|
||||
self.monotonic_mapping.mapping(from_val)
|
||||
}
|
||||
|
||||
fn min_value(&self) -> Output {
|
||||
let from_min_value = self.from_column.min_value();
|
||||
self.monotonic_mapping.mapping(from_min_value)
|
||||
}
|
||||
|
||||
fn max_value(&self) -> Output {
|
||||
let from_max_value = self.from_column.max_value();
|
||||
self.monotonic_mapping.mapping(from_max_value)
|
||||
}
|
||||
|
||||
fn num_vals(&self) -> u32 {
|
||||
self.from_column.num_vals()
|
||||
}
|
||||
|
||||
fn iter(&self) -> Box<dyn Iterator<Item = Output> + '_> {
|
||||
Box::new(
|
||||
self.from_column
|
||||
.iter()
|
||||
.map(|el| self.monotonic_mapping.mapping(el)),
|
||||
)
|
||||
}
|
||||
|
||||
fn get_row_ids_for_value_range(
|
||||
&self,
|
||||
range: RangeInclusive<Output>,
|
||||
doc_id_range: Range<u32>,
|
||||
positions: &mut Vec<u32>,
|
||||
) {
|
||||
self.from_column.get_row_ids_for_value_range(
|
||||
self.monotonic_mapping.inverse(range.start().clone())
|
||||
..=self.monotonic_mapping.inverse(range.end().clone()),
|
||||
doc_id_range,
|
||||
positions,
|
||||
)
|
||||
}
|
||||
|
||||
// We voluntarily do not implement get_range as it yields a regression,
|
||||
// and we do not have any specialized implementation anyway.
|
||||
}
|
||||
|
||||
/// Wraps an iterator into a `Column`.
|
||||
pub struct IterColumn<T>(T);
|
||||
|
||||
impl<T> From<T> for IterColumn<T>
|
||||
where T: Iterator + Clone + ExactSizeIterator
|
||||
{
|
||||
fn from(iter: T) -> Self {
|
||||
IterColumn(iter)
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> ColumnValues<T::Item> for IterColumn<T>
|
||||
where
|
||||
T: Iterator + Clone + ExactSizeIterator + Send + Sync,
|
||||
T::Item: PartialOrd + Debug,
|
||||
{
|
||||
fn get_val(&self, idx: u32) -> T::Item {
|
||||
self.0.clone().nth(idx as usize).unwrap()
|
||||
}
|
||||
|
||||
fn min_value(&self) -> T::Item {
|
||||
self.0.clone().next().unwrap()
|
||||
}
|
||||
|
||||
fn max_value(&self) -> T::Item {
|
||||
self.0.clone().last().unwrap()
|
||||
}
|
||||
|
||||
fn num_vals(&self) -> u32 {
|
||||
self.0.len() as u32
|
||||
}
|
||||
|
||||
fn iter(&self) -> Box<dyn Iterator<Item = T::Item> + '_> {
|
||||
Box::new(self.0.clone())
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::column_values::monotonic_mapping::{
|
||||
StrictlyMonotonicMappingInverter, StrictlyMonotonicMappingToInternalBaseval,
|
||||
StrictlyMonotonicMappingToInternalGCDBaseval,
|
||||
};
|
||||
|
||||
#[test]
|
||||
fn test_monotonic_mapping() {
|
||||
let vals = &[3u64, 5u64][..];
|
||||
let col = VecColumn::from(vals);
|
||||
let mapped = monotonic_map_column(col, StrictlyMonotonicMappingToInternalBaseval::new(2));
|
||||
assert_eq!(mapped.min_value(), 1u64);
|
||||
assert_eq!(mapped.max_value(), 3u64);
|
||||
assert_eq!(mapped.num_vals(), 2);
|
||||
assert_eq!(mapped.num_vals(), 2);
|
||||
assert_eq!(mapped.get_val(0), 1);
|
||||
assert_eq!(mapped.get_val(1), 3);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_range_as_col() {
|
||||
let col = IterColumn::from(10..100);
|
||||
assert_eq!(col.num_vals(), 90);
|
||||
assert_eq!(col.max_value(), 99);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_monotonic_mapping_iter() {
|
||||
let vals: Vec<u64> = (10..110u64).map(|el| el * 10).collect();
|
||||
let col = VecColumn::from(&vals);
|
||||
let mapped = monotonic_map_column(
|
||||
col,
|
||||
StrictlyMonotonicMappingInverter::from(
|
||||
StrictlyMonotonicMappingToInternalGCDBaseval::new(10, 100),
|
||||
),
|
||||
);
|
||||
let val_i64s: Vec<u64> = mapped.iter().collect();
|
||||
for i in 0..100 {
|
||||
assert_eq!(val_i64s[i as usize], mapped.get_val(i));
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_monotonic_mapping_get_range() {
|
||||
let vals: Vec<u64> = (0..100u64).map(|el| el * 10).collect();
|
||||
let col = VecColumn::from(&vals);
|
||||
let mapped = monotonic_map_column(
|
||||
col,
|
||||
StrictlyMonotonicMappingInverter::from(
|
||||
StrictlyMonotonicMappingToInternalGCDBaseval::new(10, 0),
|
||||
),
|
||||
);
|
||||
|
||||
assert_eq!(mapped.min_value(), 0u64);
|
||||
assert_eq!(mapped.max_value(), 9900u64);
|
||||
assert_eq!(mapped.num_vals(), 100);
|
||||
let val_u64s: Vec<u64> = mapped.iter().collect();
|
||||
assert_eq!(val_u64s.len(), 100);
|
||||
for i in 0..100 {
|
||||
assert_eq!(val_u64s[i as usize], mapped.get_val(i));
|
||||
assert_eq!(val_u64s[i as usize], vals[i as usize] * 10);
|
||||
}
|
||||
let mut buf = [0u64; 20];
|
||||
mapped.get_range(7, &mut buf[..]);
|
||||
assert_eq!(&val_u64s[7..][..20], &buf);
|
||||
}
|
||||
}
|
||||
41
columnar/src/column_values/merge.rs
Normal file
41
columnar/src/column_values/merge.rs
Normal file
@@ -0,0 +1,41 @@
|
||||
use std::fmt::Debug;
|
||||
use std::sync::Arc;
|
||||
|
||||
use crate::iterable::Iterable;
|
||||
use crate::{ColumnIndex, ColumnValues, MergeRowOrder};
|
||||
|
||||
pub(crate) struct MergedColumnValues<'a, T> {
|
||||
pub(crate) column_indexes: &'a [Option<ColumnIndex>],
|
||||
pub(crate) column_values: &'a [Option<Arc<dyn ColumnValues<T>>>],
|
||||
pub(crate) merge_row_order: &'a MergeRowOrder,
|
||||
}
|
||||
|
||||
impl<'a, T: Copy + PartialOrd + Debug> Iterable<T> for MergedColumnValues<'a, T> {
|
||||
fn boxed_iter(&self) -> Box<dyn Iterator<Item = T> + '_> {
|
||||
match self.merge_row_order {
|
||||
MergeRowOrder::Stack(_) => Box::new(
|
||||
self.column_values
|
||||
.iter()
|
||||
.flatten()
|
||||
.flat_map(|column_value| column_value.iter()),
|
||||
),
|
||||
MergeRowOrder::Shuffled(shuffle_merge_order) => Box::new(
|
||||
shuffle_merge_order
|
||||
.iter_new_to_old_row_addrs()
|
||||
.flat_map(|row_addr| {
|
||||
let column_index =
|
||||
self.column_indexes[row_addr.segment_ord as usize].as_ref()?;
|
||||
let column_values =
|
||||
self.column_values[row_addr.segment_ord as usize].as_ref()?;
|
||||
let value_range = column_index.value_row_ids(row_addr.row_id);
|
||||
Some((value_range, column_values))
|
||||
})
|
||||
.flat_map(|(value_range, column_values)| {
|
||||
value_range
|
||||
.into_iter()
|
||||
.map(|val| column_values.get_val(val))
|
||||
}),
|
||||
),
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -7,260 +7,214 @@
|
||||
//! - Monotonically map values to u64/u128
|
||||
|
||||
use std::fmt::Debug;
|
||||
use std::io;
|
||||
use std::io::Write;
|
||||
use std::ops::{Range, RangeInclusive};
|
||||
use std::sync::Arc;
|
||||
|
||||
use common::{BinarySerializable, OwnedBytes};
|
||||
use compact_space::CompactSpaceDecompressor;
|
||||
pub use monotonic_mapping::{MonotonicallyMappableToU64, StrictlyMonotonicFn};
|
||||
use monotonic_mapping::{StrictlyMonotonicMappingInverter, StrictlyMonotonicMappingToInternal};
|
||||
pub use monotonic_mapping_u128::MonotonicallyMappableToU128;
|
||||
use serialize::U128Header;
|
||||
|
||||
mod compact_space;
|
||||
mod merge;
|
||||
pub(crate) mod monotonic_mapping;
|
||||
pub(crate) mod monotonic_mapping_u128;
|
||||
mod stats;
|
||||
pub(crate) mod u64_based;
|
||||
mod u128_based;
|
||||
mod u64_based;
|
||||
mod vec_column;
|
||||
|
||||
mod column;
|
||||
pub(crate) mod serialize;
|
||||
mod monotonic_column;
|
||||
|
||||
pub use serialize::serialize_column_values_u128;
|
||||
pub(crate) use merge::MergedColumnValues;
|
||||
pub use stats::ColumnStats;
|
||||
pub use u128_based::{open_u128_mapped, serialize_column_values_u128};
|
||||
pub use u64_based::{
|
||||
load_u64_based_column_values, serialize_and_load_u64_based_column_values,
|
||||
serialize_u64_based_column_values, CodecType, ALL_U64_CODEC_TYPES,
|
||||
};
|
||||
pub use vec_column::VecColumn;
|
||||
|
||||
pub use self::column::{monotonic_map_column, ColumnValues, IterColumn, VecColumn};
|
||||
use crate::iterable::Iterable;
|
||||
use crate::{ColumnIndex, MergeRowOrder};
|
||||
pub use self::monotonic_column::monotonic_map_column;
|
||||
use crate::RowId;
|
||||
|
||||
pub(crate) struct MergedColumnValues<'a, T> {
|
||||
pub(crate) column_indexes: &'a [Option<ColumnIndex>],
|
||||
pub(crate) column_values: &'a [Option<Arc<dyn ColumnValues<T>>>],
|
||||
pub(crate) merge_row_order: &'a MergeRowOrder,
|
||||
}
|
||||
/// `ColumnValues` provides access to a dense field column.
|
||||
///
|
||||
/// `Column` are just a wrapper over `ColumnValues` and a `ColumnIndex`.
|
||||
///
|
||||
/// Any methods with a default and specialized implementation need to be called in the
|
||||
/// wrappers that implement the trait: Arc and MonotonicMappingColumn
|
||||
pub trait ColumnValues<T: PartialOrd = u64>: Send + Sync {
|
||||
/// Return the value associated with the given idx.
|
||||
///
|
||||
/// This accessor should return as fast as possible.
|
||||
///
|
||||
/// # Panics
|
||||
///
|
||||
/// May panic if `idx` is greater than the column length.
|
||||
fn get_val(&self, idx: u32) -> T;
|
||||
|
||||
impl<'a, T: Copy + PartialOrd + Debug> Iterable<T> for MergedColumnValues<'a, T> {
|
||||
fn boxed_iter(&self) -> Box<dyn Iterator<Item = T> + '_> {
|
||||
match self.merge_row_order {
|
||||
MergeRowOrder::Stack(_) => {
|
||||
Box::new(self
|
||||
.column_values
|
||||
.iter()
|
||||
.flatten()
|
||||
.flat_map(|column_value| column_value.iter()))
|
||||
},
|
||||
MergeRowOrder::Shuffled(shuffle_merge_order) => {
|
||||
Box::new(shuffle_merge_order
|
||||
.iter_new_to_old_row_addrs()
|
||||
.flat_map(|row_addr| {
|
||||
let Some(column_index) = self.column_indexes[row_addr.segment_ord as usize].as_ref() else {
|
||||
return None;
|
||||
};
|
||||
let Some(column_values) = self.column_values[row_addr.segment_ord as usize].as_ref() else {
|
||||
return None;
|
||||
};
|
||||
let value_range = column_index.value_row_ids(row_addr.row_id);
|
||||
Some((value_range, column_values))
|
||||
})
|
||||
.flat_map(|(value_range, column_values)| {
|
||||
value_range
|
||||
.into_iter()
|
||||
.map(|val| column_values.get_val(val))
|
||||
})
|
||||
)
|
||||
},
|
||||
/// Allows to push down multiple fetch calls, to avoid dynamic dispatch overhead.
|
||||
///
|
||||
/// idx and output should have the same length
|
||||
///
|
||||
/// # Panics
|
||||
///
|
||||
/// May panic if `idx` is greater than the column length.
|
||||
fn get_vals(&self, idx: &[u32], output: &mut [T]) {
|
||||
assert!(idx.len() == output.len());
|
||||
for (out, idx) in output.iter_mut().zip(idx.iter()) {
|
||||
*out = self.get_val(*idx as u32);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(PartialEq, Eq, PartialOrd, Ord, Debug, Clone, Copy)]
|
||||
#[repr(u8)]
|
||||
/// Available codecs to use to encode the u128 (via [`MonotonicallyMappableToU128`]) converted data.
|
||||
pub enum U128FastFieldCodecType {
|
||||
/// This codec takes a large number space (u128) and reduces it to a compact number space, by
|
||||
/// removing the holes.
|
||||
CompactSpace = 1,
|
||||
}
|
||||
|
||||
impl BinarySerializable for U128FastFieldCodecType {
|
||||
fn serialize<W: Write + ?Sized>(&self, wrt: &mut W) -> io::Result<()> {
|
||||
self.to_code().serialize(wrt)
|
||||
}
|
||||
|
||||
fn deserialize<R: io::Read>(reader: &mut R) -> io::Result<Self> {
|
||||
let code = u8::deserialize(reader)?;
|
||||
let codec_type: Self = Self::from_code(code)
|
||||
.ok_or_else(|| io::Error::new(io::ErrorKind::InvalidData, "Unknown code `{code}.`"))?;
|
||||
Ok(codec_type)
|
||||
}
|
||||
}
|
||||
|
||||
impl U128FastFieldCodecType {
|
||||
pub(crate) fn to_code(self) -> u8 {
|
||||
self as u8
|
||||
}
|
||||
|
||||
pub(crate) fn from_code(code: u8) -> Option<Self> {
|
||||
match code {
|
||||
1 => Some(Self::CompactSpace),
|
||||
_ => None,
|
||||
/// Fills an output buffer with the fast field values
|
||||
/// associated with the `DocId` going from
|
||||
/// `start` to `start + output.len()`.
|
||||
///
|
||||
/// # Panics
|
||||
///
|
||||
/// Must panic if `start + output.len()` is greater than
|
||||
/// the segment's `maxdoc`.
|
||||
#[inline(always)]
|
||||
fn get_range(&self, start: u64, output: &mut [T]) {
|
||||
for (out, idx) in output.iter_mut().zip(start..) {
|
||||
*out = self.get_val(idx as u32);
|
||||
}
|
||||
}
|
||||
|
||||
/// Get the row ids of values which are in the provided value range.
|
||||
///
|
||||
/// Note that position == docid for single value fast fields
|
||||
#[inline(always)]
|
||||
fn get_row_ids_for_value_range(
|
||||
&self,
|
||||
value_range: RangeInclusive<T>,
|
||||
row_id_range: Range<RowId>,
|
||||
row_id_hits: &mut Vec<RowId>,
|
||||
) {
|
||||
let row_id_range = row_id_range.start..row_id_range.end.min(self.num_vals());
|
||||
for idx in row_id_range.start..row_id_range.end {
|
||||
let val = self.get_val(idx);
|
||||
if value_range.contains(&val) {
|
||||
row_id_hits.push(idx);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the minimum value for this fast field.
|
||||
///
|
||||
/// This min_value may not be exact.
|
||||
/// For instance, the min value does not take in account of possible
|
||||
/// deleted document. All values are however guaranteed to be higher than
|
||||
/// `.min_value()`.
|
||||
fn min_value(&self) -> T;
|
||||
|
||||
/// Returns the maximum value for this fast field.
|
||||
///
|
||||
/// This max_value may not be exact.
|
||||
/// For instance, the max value does not take in account of possible
|
||||
/// deleted document. All values are however guaranteed to be higher than
|
||||
/// `.max_value()`.
|
||||
fn max_value(&self) -> T;
|
||||
|
||||
/// The number of values in the column.
|
||||
fn num_vals(&self) -> u32;
|
||||
|
||||
/// Returns a iterator over the data
|
||||
fn iter<'a>(&'a self) -> Box<dyn Iterator<Item = T> + 'a> {
|
||||
Box::new((0..self.num_vals()).map(|idx| self.get_val(idx)))
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the correct codec reader wrapped in the `Arc` for the data.
|
||||
pub fn open_u128_mapped<T: MonotonicallyMappableToU128 + Debug>(
|
||||
mut bytes: OwnedBytes,
|
||||
) -> io::Result<Arc<dyn ColumnValues<T>>> {
|
||||
let header = U128Header::deserialize(&mut bytes)?;
|
||||
assert_eq!(header.codec_type, U128FastFieldCodecType::CompactSpace);
|
||||
let reader = CompactSpaceDecompressor::open(bytes)?;
|
||||
impl<T: Copy + PartialOrd + Debug> ColumnValues<T> for Arc<dyn ColumnValues<T>> {
|
||||
#[inline(always)]
|
||||
fn get_val(&self, idx: u32) -> T {
|
||||
self.as_ref().get_val(idx)
|
||||
}
|
||||
|
||||
let inverted: StrictlyMonotonicMappingInverter<StrictlyMonotonicMappingToInternal<T>> =
|
||||
StrictlyMonotonicMappingToInternal::<T>::new().into();
|
||||
Ok(Arc::new(monotonic_map_column(reader, inverted)))
|
||||
#[inline(always)]
|
||||
fn min_value(&self) -> T {
|
||||
self.as_ref().min_value()
|
||||
}
|
||||
|
||||
#[inline(always)]
|
||||
fn max_value(&self) -> T {
|
||||
self.as_ref().max_value()
|
||||
}
|
||||
|
||||
#[inline(always)]
|
||||
fn num_vals(&self) -> u32 {
|
||||
self.as_ref().num_vals()
|
||||
}
|
||||
|
||||
#[inline(always)]
|
||||
fn iter<'b>(&'b self) -> Box<dyn Iterator<Item = T> + 'b> {
|
||||
self.as_ref().iter()
|
||||
}
|
||||
|
||||
#[inline(always)]
|
||||
fn get_range(&self, start: u64, output: &mut [T]) {
|
||||
self.as_ref().get_range(start, output)
|
||||
}
|
||||
|
||||
#[inline(always)]
|
||||
fn get_row_ids_for_value_range(
|
||||
&self,
|
||||
range: RangeInclusive<T>,
|
||||
doc_id_range: Range<u32>,
|
||||
positions: &mut Vec<u32>,
|
||||
) {
|
||||
self.as_ref()
|
||||
.get_row_ids_for_value_range(range, doc_id_range, positions)
|
||||
}
|
||||
}
|
||||
|
||||
/// Wraps an cloneable iterator into a `Column`.
|
||||
pub struct IterColumn<T>(T);
|
||||
|
||||
impl<T> From<T> for IterColumn<T>
|
||||
where T: Iterator + Clone + ExactSizeIterator
|
||||
{
|
||||
fn from(iter: T) -> Self {
|
||||
IterColumn(iter)
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> ColumnValues<T::Item> for IterColumn<T>
|
||||
where
|
||||
T: Iterator + Clone + ExactSizeIterator + Send + Sync,
|
||||
T::Item: PartialOrd + Debug,
|
||||
{
|
||||
fn get_val(&self, idx: u32) -> T::Item {
|
||||
self.0.clone().nth(idx as usize).unwrap()
|
||||
}
|
||||
|
||||
fn min_value(&self) -> T::Item {
|
||||
self.0.clone().next().unwrap()
|
||||
}
|
||||
|
||||
fn max_value(&self) -> T::Item {
|
||||
self.0.clone().last().unwrap()
|
||||
}
|
||||
|
||||
fn num_vals(&self) -> u32 {
|
||||
self.0.len() as u32
|
||||
}
|
||||
|
||||
fn iter(&self) -> Box<dyn Iterator<Item = T::Item> + '_> {
|
||||
Box::new(self.0.clone())
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(all(test, feature = "unstable"))]
|
||||
mod bench {
|
||||
use std::sync::Arc;
|
||||
|
||||
use common::OwnedBytes;
|
||||
use rand::rngs::StdRng;
|
||||
use rand::{Rng, SeedableRng};
|
||||
use test::{self, Bencher};
|
||||
mod bench;
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::column_values::u64_based::*;
|
||||
|
||||
fn get_data() -> Vec<u64> {
|
||||
let mut rng = StdRng::seed_from_u64(2u64);
|
||||
let mut data: Vec<_> = (100..55000_u64)
|
||||
.map(|num| num + rng.gen::<u8>() as u64)
|
||||
.collect();
|
||||
data.push(99_000);
|
||||
data.insert(1000, 2000);
|
||||
data.insert(2000, 100);
|
||||
data.insert(3000, 4100);
|
||||
data.insert(4000, 100);
|
||||
data.insert(5000, 800);
|
||||
data
|
||||
}
|
||||
|
||||
fn compute_stats(vals: impl Iterator<Item = u64>) -> ColumnStats {
|
||||
let mut stats_collector = StatsCollector::default();
|
||||
for val in vals {
|
||||
stats_collector.collect(val);
|
||||
}
|
||||
stats_collector.stats()
|
||||
}
|
||||
|
||||
#[inline(never)]
|
||||
fn value_iter() -> impl Iterator<Item = u64> {
|
||||
0..20_000
|
||||
}
|
||||
fn get_reader_for_bench<Codec: ColumnCodec>(data: &[u64]) -> Codec::ColumnValues {
|
||||
let mut bytes = Vec::new();
|
||||
let stats = compute_stats(data.iter().cloned());
|
||||
let mut codec_serializer = Codec::estimator();
|
||||
for val in data {
|
||||
codec_serializer.collect(*val);
|
||||
}
|
||||
codec_serializer.serialize(&stats, Box::new(data.iter().copied()).as_mut(), &mut bytes);
|
||||
|
||||
Codec::load(OwnedBytes::new(bytes)).unwrap()
|
||||
}
|
||||
fn bench_get<Codec: ColumnCodec>(b: &mut Bencher, data: &[u64]) {
|
||||
let col = get_reader_for_bench::<Codec>(data);
|
||||
b.iter(|| {
|
||||
let mut sum = 0u64;
|
||||
for pos in value_iter() {
|
||||
let val = col.get_val(pos as u32);
|
||||
sum = sum.wrapping_add(val);
|
||||
}
|
||||
sum
|
||||
});
|
||||
}
|
||||
|
||||
#[inline(never)]
|
||||
fn bench_get_dynamic_helper(b: &mut Bencher, col: Arc<dyn ColumnValues>) {
|
||||
b.iter(|| {
|
||||
let mut sum = 0u64;
|
||||
for pos in value_iter() {
|
||||
let val = col.get_val(pos as u32);
|
||||
sum = sum.wrapping_add(val);
|
||||
}
|
||||
sum
|
||||
});
|
||||
}
|
||||
|
||||
fn bench_get_dynamic<Codec: ColumnCodec>(b: &mut Bencher, data: &[u64]) {
|
||||
let col = Arc::new(get_reader_for_bench::<Codec>(data));
|
||||
bench_get_dynamic_helper(b, col);
|
||||
}
|
||||
fn bench_create<Codec: ColumnCodec>(b: &mut Bencher, data: &[u64]) {
|
||||
let stats = compute_stats(data.iter().cloned());
|
||||
|
||||
let mut bytes = Vec::new();
|
||||
b.iter(|| {
|
||||
bytes.clear();
|
||||
let mut codec_serializer = Codec::estimator();
|
||||
for val in data.iter().take(1024) {
|
||||
codec_serializer.collect(*val);
|
||||
}
|
||||
|
||||
codec_serializer.serialize(&stats, Box::new(data.iter().copied()).as_mut(), &mut bytes)
|
||||
});
|
||||
}
|
||||
|
||||
#[bench]
|
||||
fn bench_fastfield_bitpack_create(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_create::<BitpackedCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_linearinterpol_create(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_create::<LinearCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_multilinearinterpol_create(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_create::<BlockwiseLinearCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_bitpack_get(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_get::<BitpackedCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_bitpack_get_dynamic(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_get_dynamic::<BitpackedCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_linearinterpol_get(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_get::<LinearCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_linearinterpol_get_dynamic(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_get_dynamic::<LinearCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_multilinearinterpol_get(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_get::<BlockwiseLinearCodec>(b, &data);
|
||||
}
|
||||
#[bench]
|
||||
fn bench_fastfield_multilinearinterpol_get_dynamic(b: &mut Bencher) {
|
||||
let data: Vec<_> = get_data();
|
||||
bench_get_dynamic::<BlockwiseLinearCodec>(b, &data);
|
||||
#[test]
|
||||
fn test_range_as_col() {
|
||||
let col = IterColumn::from(10..100);
|
||||
assert_eq!(col.num_vals(), 90);
|
||||
assert_eq!(col.max_value(), 99);
|
||||
}
|
||||
}
|
||||
|
||||
120
columnar/src/column_values/monotonic_column.rs
Normal file
120
columnar/src/column_values/monotonic_column.rs
Normal file
@@ -0,0 +1,120 @@
|
||||
use std::fmt::Debug;
|
||||
use std::marker::PhantomData;
|
||||
use std::ops::{Range, RangeInclusive};
|
||||
|
||||
use crate::column_values::monotonic_mapping::StrictlyMonotonicFn;
|
||||
use crate::ColumnValues;
|
||||
|
||||
struct MonotonicMappingColumn<C, T, Input> {
|
||||
from_column: C,
|
||||
monotonic_mapping: T,
|
||||
_phantom: PhantomData<Input>,
|
||||
}
|
||||
|
||||
/// Creates a view of a column transformed by a strictly monotonic mapping. See
|
||||
/// [`StrictlyMonotonicFn`].
|
||||
///
|
||||
/// E.g. apply a gcd monotonic_mapping([100, 200, 300]) == [1, 2, 3]
|
||||
/// monotonic_mapping.mapping() is expected to be injective, and we should always have
|
||||
/// monotonic_mapping.inverse(monotonic_mapping.mapping(el)) == el
|
||||
///
|
||||
/// The inverse of the mapping is required for:
|
||||
/// `fn get_positions_for_value_range(&self, range: RangeInclusive<T>) -> Vec<u64> `
|
||||
/// The user provides the original value range and we need to monotonic map them in the same way the
|
||||
/// serialization does before calling the underlying column.
|
||||
///
|
||||
/// Note that when opening a codec, the monotonic_mapping should be the inverse of the mapping
|
||||
/// during serialization. And therefore the monotonic_mapping_inv when opening is the same as
|
||||
/// monotonic_mapping during serialization.
|
||||
pub fn monotonic_map_column<C, T, Input, Output>(
|
||||
from_column: C,
|
||||
monotonic_mapping: T,
|
||||
) -> impl ColumnValues<Output>
|
||||
where
|
||||
C: ColumnValues<Input>,
|
||||
T: StrictlyMonotonicFn<Input, Output> + Send + Sync,
|
||||
Input: PartialOrd + Debug + Send + Sync + Clone,
|
||||
Output: PartialOrd + Debug + Send + Sync + Clone,
|
||||
{
|
||||
MonotonicMappingColumn {
|
||||
from_column,
|
||||
monotonic_mapping,
|
||||
_phantom: PhantomData,
|
||||
}
|
||||
}
|
||||
|
||||
impl<C, T, Input, Output> ColumnValues<Output> for MonotonicMappingColumn<C, T, Input>
|
||||
where
|
||||
C: ColumnValues<Input>,
|
||||
T: StrictlyMonotonicFn<Input, Output> + Send + Sync,
|
||||
Input: PartialOrd + Send + Debug + Sync + Clone,
|
||||
Output: PartialOrd + Send + Debug + Sync + Clone,
|
||||
{
|
||||
#[inline]
|
||||
fn get_val(&self, idx: u32) -> Output {
|
||||
let from_val = self.from_column.get_val(idx);
|
||||
self.monotonic_mapping.mapping(from_val)
|
||||
}
|
||||
|
||||
fn min_value(&self) -> Output {
|
||||
let from_min_value = self.from_column.min_value();
|
||||
self.monotonic_mapping.mapping(from_min_value)
|
||||
}
|
||||
|
||||
fn max_value(&self) -> Output {
|
||||
let from_max_value = self.from_column.max_value();
|
||||
self.monotonic_mapping.mapping(from_max_value)
|
||||
}
|
||||
|
||||
fn num_vals(&self) -> u32 {
|
||||
self.from_column.num_vals()
|
||||
}
|
||||
|
||||
fn iter(&self) -> Box<dyn Iterator<Item = Output> + '_> {
|
||||
Box::new(
|
||||
self.from_column
|
||||
.iter()
|
||||
.map(|el| self.monotonic_mapping.mapping(el)),
|
||||
)
|
||||
}
|
||||
|
||||
fn get_row_ids_for_value_range(
|
||||
&self,
|
||||
range: RangeInclusive<Output>,
|
||||
doc_id_range: Range<u32>,
|
||||
positions: &mut Vec<u32>,
|
||||
) {
|
||||
self.from_column.get_row_ids_for_value_range(
|
||||
self.monotonic_mapping.inverse(range.start().clone())
|
||||
..=self.monotonic_mapping.inverse(range.end().clone()),
|
||||
doc_id_range,
|
||||
positions,
|
||||
)
|
||||
}
|
||||
|
||||
// We voluntarily do not implement get_range as it yields a regression,
|
||||
// and we do not have any specialized implementation anyway.
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::column_values::monotonic_mapping::{
|
||||
StrictlyMonotonicMappingInverter, StrictlyMonotonicMappingToInternal,
|
||||
};
|
||||
use crate::column_values::VecColumn;
|
||||
|
||||
#[test]
|
||||
fn test_monotonic_mapping_iter() {
|
||||
let vals: Vec<u64> = (0..100u64).map(|el| el * 10).collect();
|
||||
let col = VecColumn::from(&vals);
|
||||
let mapped = monotonic_map_column(
|
||||
col,
|
||||
StrictlyMonotonicMappingInverter::from(StrictlyMonotonicMappingToInternal::<i64>::new()),
|
||||
);
|
||||
let val_i64s: Vec<u64> = mapped.iter().collect();
|
||||
for i in 0..100 {
|
||||
assert_eq!(val_i64s[i as usize], mapped.get_val(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -2,7 +2,6 @@ use std::fmt::Debug;
|
||||
use std::marker::PhantomData;
|
||||
|
||||
use common::DateTime;
|
||||
use fastdivide::DividerU64;
|
||||
|
||||
use super::MonotonicallyMappableToU128;
|
||||
use crate::RowId;
|
||||
@@ -113,68 +112,6 @@ where T: MonotonicallyMappableToU64
|
||||
}
|
||||
}
|
||||
|
||||
/// Mapping dividing by gcd and a base value.
|
||||
///
|
||||
/// The function is assumed to be only called on values divided by passed
|
||||
/// gcd value. (It is necessary for the function to be monotonic.)
|
||||
pub(crate) struct StrictlyMonotonicMappingToInternalGCDBaseval {
|
||||
gcd_divider: DividerU64,
|
||||
gcd: u64,
|
||||
min_value: u64,
|
||||
}
|
||||
impl StrictlyMonotonicMappingToInternalGCDBaseval {
|
||||
/// Creates a linear mapping `x -> gcd*x + min_value`.
|
||||
pub(crate) fn new(gcd: u64, min_value: u64) -> Self {
|
||||
let gcd_divider = DividerU64::divide_by(gcd);
|
||||
Self {
|
||||
gcd_divider,
|
||||
gcd,
|
||||
min_value,
|
||||
}
|
||||
}
|
||||
}
|
||||
impl<External: MonotonicallyMappableToU64> StrictlyMonotonicFn<External, u64>
|
||||
for StrictlyMonotonicMappingToInternalGCDBaseval
|
||||
{
|
||||
#[inline(always)]
|
||||
fn mapping(&self, inp: External) -> u64 {
|
||||
self.gcd_divider
|
||||
.divide(External::to_u64(inp) - self.min_value)
|
||||
}
|
||||
|
||||
#[inline(always)]
|
||||
fn inverse(&self, out: u64) -> External {
|
||||
External::from_u64(self.min_value + out * self.gcd)
|
||||
}
|
||||
}
|
||||
|
||||
/// Strictly monotonic mapping with a base value.
|
||||
pub(crate) struct StrictlyMonotonicMappingToInternalBaseval {
|
||||
min_value: u64,
|
||||
}
|
||||
|
||||
impl StrictlyMonotonicMappingToInternalBaseval {
|
||||
/// Creates a linear mapping `x -> x + min_value`.
|
||||
#[inline(always)]
|
||||
pub(crate) fn new(min_value: u64) -> Self {
|
||||
Self { min_value }
|
||||
}
|
||||
}
|
||||
|
||||
impl<External: MonotonicallyMappableToU64> StrictlyMonotonicFn<External, u64>
|
||||
for StrictlyMonotonicMappingToInternalBaseval
|
||||
{
|
||||
#[inline(always)]
|
||||
fn mapping(&self, val: External) -> u64 {
|
||||
External::to_u64(val) - self.min_value
|
||||
}
|
||||
|
||||
#[inline(always)]
|
||||
fn inverse(&self, val: u64) -> External {
|
||||
External::from_u64(self.min_value + val)
|
||||
}
|
||||
}
|
||||
|
||||
impl MonotonicallyMappableToU64 for u64 {
|
||||
#[inline(always)]
|
||||
fn to_u64(self) -> u64 {
|
||||
@@ -263,13 +200,6 @@ mod tests {
|
||||
// TODO
|
||||
// identity mapping
|
||||
// test_round_trip(&StrictlyMonotonicMappingToInternal::<u128>::new(), 100u128);
|
||||
|
||||
// base value to i64 round trip
|
||||
let mapping = StrictlyMonotonicMappingToInternalBaseval::new(100);
|
||||
test_round_trip::<_, _, u64>(&mapping, 100i64);
|
||||
// base value and gcd to u64 round trip
|
||||
let mapping = StrictlyMonotonicMappingToInternalGCDBaseval::new(10, 100);
|
||||
test_round_trip::<_, _, u64>(&mapping, 100u64);
|
||||
}
|
||||
|
||||
fn test_round_trip<T: StrictlyMonotonicFn<K, L>, K: std::fmt::Debug + Eq + Copy, L>(
|
||||
|
||||
@@ -17,16 +17,16 @@ use std::{
|
||||
ops::{Range, RangeInclusive},
|
||||
};
|
||||
|
||||
mod blank_range;
|
||||
mod build_compact_space;
|
||||
|
||||
use build_compact_space::get_compact_space;
|
||||
use common::{BinarySerializable, CountingWriter, OwnedBytes, VInt, VIntU128};
|
||||
use tantivy_bitpacker::{self, BitPacker, BitUnpacker};
|
||||
|
||||
use crate::column_values::compact_space::build_compact_space::get_compact_space;
|
||||
use crate::column_values::ColumnValues;
|
||||
use crate::RowId;
|
||||
|
||||
mod blank_range;
|
||||
mod build_compact_space;
|
||||
|
||||
/// The cost per blank is quite hard actually, since blanks are delta encoded, the actual cost of
|
||||
/// blanks depends on the number of blanks.
|
||||
///
|
||||
@@ -464,7 +464,7 @@ mod tests {
|
||||
use itertools::Itertools;
|
||||
|
||||
use super::*;
|
||||
use crate::column_values::serialize::U128Header;
|
||||
use crate::column_values::u128_based::U128Header;
|
||||
use crate::column_values::{open_u128_mapped, serialize_column_values_u128};
|
||||
|
||||
#[test]
|
||||
@@ -1,12 +1,19 @@
|
||||
use std::fmt::Debug;
|
||||
use std::io;
|
||||
use std::io::Write;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common::{BinarySerializable, VInt};
|
||||
mod compact_space;
|
||||
|
||||
use crate::column_values::compact_space::CompactSpaceCompressor;
|
||||
use crate::column_values::U128FastFieldCodecType;
|
||||
use common::{BinarySerializable, OwnedBytes, VInt};
|
||||
use compact_space::{CompactSpaceCompressor, CompactSpaceDecompressor};
|
||||
|
||||
use crate::column_values::monotonic_map_column;
|
||||
use crate::column_values::monotonic_mapping::{
|
||||
StrictlyMonotonicMappingInverter, StrictlyMonotonicMappingToInternal,
|
||||
};
|
||||
use crate::iterable::Iterable;
|
||||
use crate::MonotonicallyMappableToU128;
|
||||
use crate::{ColumnValues, MonotonicallyMappableToU128};
|
||||
|
||||
#[derive(Debug, Copy, Clone, PartialEq, Eq)]
|
||||
pub(crate) struct U128Header {
|
||||
@@ -55,6 +62,52 @@ pub fn serialize_column_values_u128<T: MonotonicallyMappableToU128>(
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[derive(PartialEq, Eq, PartialOrd, Ord, Debug, Clone, Copy)]
|
||||
#[repr(u8)]
|
||||
/// Available codecs to use to encode the u128 (via [`MonotonicallyMappableToU128`]) converted data.
|
||||
pub(crate) enum U128FastFieldCodecType {
|
||||
/// This codec takes a large number space (u128) and reduces it to a compact number space, by
|
||||
/// removing the holes.
|
||||
CompactSpace = 1,
|
||||
}
|
||||
|
||||
impl BinarySerializable for U128FastFieldCodecType {
|
||||
fn serialize<W: Write + ?Sized>(&self, wrt: &mut W) -> io::Result<()> {
|
||||
self.to_code().serialize(wrt)
|
||||
}
|
||||
|
||||
fn deserialize<R: io::Read>(reader: &mut R) -> io::Result<Self> {
|
||||
let code = u8::deserialize(reader)?;
|
||||
let codec_type: Self = Self::from_code(code)
|
||||
.ok_or_else(|| io::Error::new(io::ErrorKind::InvalidData, "Unknown code `{code}.`"))?;
|
||||
Ok(codec_type)
|
||||
}
|
||||
}
|
||||
|
||||
impl U128FastFieldCodecType {
|
||||
pub(crate) fn to_code(self) -> u8 {
|
||||
self as u8
|
||||
}
|
||||
|
||||
pub(crate) fn from_code(code: u8) -> Option<Self> {
|
||||
match code {
|
||||
1 => Some(Self::CompactSpace),
|
||||
_ => None,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the correct codec reader wrapped in the `Arc` for the data.
|
||||
pub fn open_u128_mapped<T: MonotonicallyMappableToU128 + Debug>(
|
||||
mut bytes: OwnedBytes,
|
||||
) -> io::Result<Arc<dyn ColumnValues<T>>> {
|
||||
let header = U128Header::deserialize(&mut bytes)?;
|
||||
assert_eq!(header.codec_type, U128FastFieldCodecType::CompactSpace);
|
||||
let reader = CompactSpaceDecompressor::open(bytes)?;
|
||||
let inverted: StrictlyMonotonicMappingInverter<StrictlyMonotonicMappingToInternal<T>> =
|
||||
StrictlyMonotonicMappingToInternal::<T>::new().into();
|
||||
Ok(Arc::new(monotonic_map_column(reader, inverted)))
|
||||
}
|
||||
#[cfg(test)]
|
||||
pub mod tests {
|
||||
use super::*;
|
||||
@@ -201,8 +201,8 @@ pub struct BlockwiseLinearReader {
|
||||
impl ColumnValues for BlockwiseLinearReader {
|
||||
#[inline(always)]
|
||||
fn get_val(&self, idx: u32) -> u64 {
|
||||
let block_id = (idx / BLOCK_SIZE as u32) as usize;
|
||||
let idx_within_block = idx % (BLOCK_SIZE as u32);
|
||||
let block_id = (idx / BLOCK_SIZE) as usize;
|
||||
let idx_within_block = idx % BLOCK_SIZE;
|
||||
let block = &self.blocks[block_id];
|
||||
let interpoled_val: u64 = block.line.eval(idx_within_block);
|
||||
let block_bytes = &self.data[block.data_start_offset..];
|
||||
|
||||
@@ -1,6 +1,6 @@
|
||||
use proptest::prelude::*;
|
||||
use proptest::strategy::Strategy;
|
||||
use proptest::{prop_oneof, proptest};
|
||||
use proptest::{num, prop_oneof, proptest};
|
||||
|
||||
#[test]
|
||||
fn test_serialize_and_load_simple() {
|
||||
@@ -19,6 +19,62 @@ fn test_serialize_and_load_simple() {
|
||||
assert_eq!(col.get_val(1), 2);
|
||||
assert_eq!(col.get_val(2), 5);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_empty_column_i64() {
|
||||
let vals: [i64; 0] = [];
|
||||
let mut num_acceptable_codecs = 0;
|
||||
for codec in ALL_U64_CODEC_TYPES {
|
||||
let mut buffer = Vec::new();
|
||||
if serialize_u64_based_column_values(&&vals[..], &[codec], &mut buffer).is_err() {
|
||||
continue;
|
||||
}
|
||||
num_acceptable_codecs += 1;
|
||||
let col = load_u64_based_column_values::<i64>(OwnedBytes::new(buffer)).unwrap();
|
||||
assert_eq!(col.num_vals(), 0);
|
||||
assert_eq!(col.min_value(), i64::MIN);
|
||||
assert_eq!(col.max_value(), i64::MIN);
|
||||
}
|
||||
assert!(num_acceptable_codecs > 0);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_empty_column_u64() {
|
||||
let vals: [u64; 0] = [];
|
||||
let mut num_acceptable_codecs = 0;
|
||||
for codec in ALL_U64_CODEC_TYPES {
|
||||
let mut buffer = Vec::new();
|
||||
if serialize_u64_based_column_values(&&vals[..], &[codec], &mut buffer).is_err() {
|
||||
continue;
|
||||
}
|
||||
num_acceptable_codecs += 1;
|
||||
let col = load_u64_based_column_values::<u64>(OwnedBytes::new(buffer)).unwrap();
|
||||
assert_eq!(col.num_vals(), 0);
|
||||
assert_eq!(col.min_value(), u64::MIN);
|
||||
assert_eq!(col.max_value(), u64::MIN);
|
||||
}
|
||||
assert!(num_acceptable_codecs > 0);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_empty_column_f64() {
|
||||
let vals: [f64; 0] = [];
|
||||
let mut num_acceptable_codecs = 0;
|
||||
for codec in ALL_U64_CODEC_TYPES {
|
||||
let mut buffer = Vec::new();
|
||||
if serialize_u64_based_column_values(&&vals[..], &[codec], &mut buffer).is_err() {
|
||||
continue;
|
||||
}
|
||||
num_acceptable_codecs += 1;
|
||||
let col = load_u64_based_column_values::<f64>(OwnedBytes::new(buffer)).unwrap();
|
||||
assert_eq!(col.num_vals(), 0);
|
||||
// FIXME. f64::MIN would be better!
|
||||
assert!(col.min_value().is_nan());
|
||||
assert!(col.max_value().is_nan());
|
||||
}
|
||||
assert!(num_acceptable_codecs > 0);
|
||||
}
|
||||
|
||||
pub(crate) fn create_and_validate<TColumnCodec: ColumnCodec>(
|
||||
vals: &[u64],
|
||||
name: &str,
|
||||
|
||||
52
columnar/src/column_values/vec_column.rs
Normal file
52
columnar/src/column_values/vec_column.rs
Normal file
@@ -0,0 +1,52 @@
|
||||
use std::fmt::Debug;
|
||||
|
||||
use tantivy_bitpacker::minmax;
|
||||
|
||||
use crate::ColumnValues;
|
||||
|
||||
/// VecColumn provides `Column` over a slice.
|
||||
pub struct VecColumn<'a, T = u64> {
|
||||
pub(crate) values: &'a [T],
|
||||
pub(crate) min_value: T,
|
||||
pub(crate) max_value: T,
|
||||
}
|
||||
|
||||
impl<'a, T: Copy + PartialOrd + Send + Sync + Debug> ColumnValues<T> for VecColumn<'a, T> {
|
||||
fn get_val(&self, position: u32) -> T {
|
||||
self.values[position as usize]
|
||||
}
|
||||
|
||||
fn iter(&self) -> Box<dyn Iterator<Item = T> + '_> {
|
||||
Box::new(self.values.iter().copied())
|
||||
}
|
||||
|
||||
fn min_value(&self) -> T {
|
||||
self.min_value
|
||||
}
|
||||
|
||||
fn max_value(&self) -> T {
|
||||
self.max_value
|
||||
}
|
||||
|
||||
fn num_vals(&self) -> u32 {
|
||||
self.values.len() as u32
|
||||
}
|
||||
|
||||
fn get_range(&self, start: u64, output: &mut [T]) {
|
||||
output.copy_from_slice(&self.values[start as usize..][..output.len()])
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a, T: Copy + PartialOrd + Default, V> From<&'a V> for VecColumn<'a, T>
|
||||
where V: AsRef<[T]> + ?Sized
|
||||
{
|
||||
fn from(values: &'a V) -> Self {
|
||||
let values = values.as_ref();
|
||||
let (min_value, max_value) = minmax(values.iter().copied()).unwrap_or_default();
|
||||
Self {
|
||||
values,
|
||||
min_value,
|
||||
max_value,
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,12 +1,14 @@
|
||||
use std::fmt::Debug;
|
||||
use std::net::Ipv6Addr;
|
||||
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
use crate::value::NumericalType;
|
||||
use crate::InvalidData;
|
||||
|
||||
/// The column type represents the column type.
|
||||
/// Any changes need to be propagated to `COLUMN_TYPES`.
|
||||
#[derive(Hash, Eq, PartialEq, Debug, Clone, Copy, Ord, PartialOrd)]
|
||||
#[derive(Hash, Eq, PartialEq, Debug, Clone, Copy, Ord, PartialOrd, Serialize, Deserialize)]
|
||||
#[repr(u8)]
|
||||
pub enum ColumnType {
|
||||
I64 = 0u8,
|
||||
@@ -143,7 +145,7 @@ mod tests {
|
||||
}
|
||||
}
|
||||
for code in COLUMN_TYPES.len() as u8..=u8::MAX {
|
||||
assert!(ColumnType::try_from_code(code as u8).is_err());
|
||||
assert!(ColumnType::try_from_code(code).is_err());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -96,7 +96,7 @@ fn compute_term_bitset(column: &BytesColumn, row_bitset: &ReadOnlyBitSet) -> Bit
|
||||
let num_terms = column.dictionary().num_terms();
|
||||
let mut term_bitset = BitSet::with_max_value(num_terms as u32);
|
||||
for row_id in row_bitset.iter() {
|
||||
for term_ord in column.term_ord_column.values(row_id) {
|
||||
for term_ord in column.term_ord_column.values_for_doc(row_id) {
|
||||
term_bitset.insert(term_ord as u32);
|
||||
}
|
||||
}
|
||||
@@ -191,7 +191,7 @@ struct TermOrdinalMapping {
|
||||
impl TermOrdinalMapping {
|
||||
fn add_segment(&mut self, max_term_ord: usize) {
|
||||
self.per_segment_new_term_ordinals
|
||||
.push(vec![TermOrdinal::default(); max_term_ord as usize]);
|
||||
.push(vec![TermOrdinal::default(); max_term_ord]);
|
||||
}
|
||||
|
||||
fn register_from_to(&mut self, segment_ord: usize, from_ord: TermOrdinal, to_ord: TermOrdinal) {
|
||||
|
||||
@@ -2,8 +2,6 @@ mod merge_dict_column;
|
||||
mod merge_mapping;
|
||||
mod term_merger;
|
||||
|
||||
// mod sorted_doc_id_column;
|
||||
|
||||
use std::collections::{BTreeMap, HashMap, HashSet};
|
||||
use std::io;
|
||||
use std::net::Ipv6Addr;
|
||||
@@ -54,14 +52,34 @@ impl From<ColumnType> for ColumnTypeCategory {
|
||||
}
|
||||
}
|
||||
|
||||
/// Merge several columnar table together.
|
||||
///
|
||||
/// If several columns with the same name are conflicting with the numerical types in the
|
||||
/// input columnars, the first type compatible out of i64, u64, f64 in that order will be used.
|
||||
///
|
||||
/// `require_columns` makes it possible to ensure that some columns will be present in the
|
||||
/// resulting columnar. When a required column is a numerical column type, one of two things can
|
||||
/// happen:
|
||||
/// - If the required column type is compatible with all of the input columnar, the resulsting
|
||||
/// merged
|
||||
/// columnar will simply coerce the input column and use the required column type.
|
||||
/// - If the required column type is incompatible with one of the input columnar, the merged
|
||||
/// will fail with an InvalidData error.
|
||||
///
|
||||
/// `merge_row_order` makes it possible to remove or reorder row in the resulting
|
||||
/// `Columnar` table.
|
||||
///
|
||||
/// Reminder: a string and a numerical column may bare the same column name. This is not
|
||||
/// considered a conflict.
|
||||
pub fn merge_columnar(
|
||||
columnar_readers: &[&ColumnarReader],
|
||||
required_columns: &[(String, ColumnType)],
|
||||
merge_row_order: MergeRowOrder,
|
||||
output: &mut impl io::Write,
|
||||
) -> io::Result<()> {
|
||||
let mut serializer = ColumnarSerializer::new(output);
|
||||
|
||||
let columns_to_merge = group_columns_for_merge(columnar_readers)?;
|
||||
let columns_to_merge = group_columns_for_merge(columnar_readers, required_columns)?;
|
||||
for ((column_name, column_type), columns) in columns_to_merge {
|
||||
let mut column_serializer =
|
||||
serializer.serialize_column(column_name.as_bytes(), column_type);
|
||||
@@ -174,98 +192,183 @@ fn merge_column(
|
||||
Ok(())
|
||||
}
|
||||
|
||||
struct GroupedColumns {
|
||||
required_column_type: Option<ColumnType>,
|
||||
columns: Vec<Option<DynamicColumn>>,
|
||||
column_category: ColumnTypeCategory,
|
||||
}
|
||||
|
||||
impl GroupedColumns {
|
||||
fn for_category(column_category: ColumnTypeCategory, num_columnars: usize) -> Self {
|
||||
GroupedColumns {
|
||||
required_column_type: None,
|
||||
columns: vec![None; num_columnars],
|
||||
column_category,
|
||||
}
|
||||
}
|
||||
|
||||
/// Set the dynamic column for a given columnar.
|
||||
fn set_column(&mut self, columnar_id: usize, column: DynamicColumn) {
|
||||
self.columns[columnar_id] = Some(column);
|
||||
}
|
||||
|
||||
/// Force the existence of a column, as well as its type.
|
||||
fn require_type(&mut self, required_type: ColumnType) -> io::Result<()> {
|
||||
if let Some(existing_required_type) = self.required_column_type {
|
||||
if existing_required_type == required_type {
|
||||
// This was just a duplicate in the `required_columns`.
|
||||
// Nothing to do.
|
||||
return Ok(());
|
||||
} else {
|
||||
return Err(io::Error::new(
|
||||
io::ErrorKind::InvalidInput,
|
||||
"Required column conflicts with another required column of the same type \
|
||||
category.",
|
||||
));
|
||||
}
|
||||
}
|
||||
self.required_column_type = Some(required_type);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Returns the column type after merge.
|
||||
///
|
||||
/// This method does not check if the column types can actually be coerced to
|
||||
/// this type.
|
||||
fn column_type_after_merge(&self) -> ColumnType {
|
||||
if let Some(required_type) = self.required_column_type {
|
||||
return required_type;
|
||||
}
|
||||
let column_type: HashSet<ColumnType> = self
|
||||
.columns
|
||||
.iter()
|
||||
.flatten()
|
||||
.map(|column| column.column_type())
|
||||
.collect();
|
||||
if column_type.len() == 1 {
|
||||
return column_type.into_iter().next().unwrap();
|
||||
}
|
||||
// At the moment, only the numerical categorical column type has more than one possible
|
||||
// column type.
|
||||
assert_eq!(self.column_category, ColumnTypeCategory::Numerical);
|
||||
merged_numerical_columns_type(self.columns.iter().flatten()).into()
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the type of the merged numerical column.
|
||||
///
|
||||
/// This function picks the first numerical type out of i64, u64, f64 (order matters
|
||||
/// here), that is compatible with all the `columns`.
|
||||
///
|
||||
/// # Panics
|
||||
/// Panics if one of the column is not numerical.
|
||||
fn merged_numerical_columns_type<'a>(
|
||||
columns: impl Iterator<Item = &'a DynamicColumn>,
|
||||
) -> NumericalType {
|
||||
let mut compatible_numerical_types = CompatibleNumericalTypes::default();
|
||||
for column in columns {
|
||||
let (min_value, max_value) =
|
||||
min_max_if_numerical(column).expect("All columns re required to be numerical");
|
||||
compatible_numerical_types.accept_value(min_value);
|
||||
compatible_numerical_types.accept_value(max_value);
|
||||
}
|
||||
compatible_numerical_types.to_numerical_type()
|
||||
}
|
||||
|
||||
#[allow(clippy::type_complexity)]
|
||||
fn group_columns_for_merge(
|
||||
columnar_readers: &[&ColumnarReader],
|
||||
required_columns: &[(String, ColumnType)],
|
||||
) -> io::Result<BTreeMap<(String, ColumnType), Vec<Option<DynamicColumn>>>> {
|
||||
// Each column name may have multiple types of column associated.
|
||||
// For merging we are interested in the same column type category since they can be merged.
|
||||
let mut columns_grouped: HashMap<(String, ColumnTypeCategory), Vec<Option<DynamicColumn>>> =
|
||||
HashMap::new();
|
||||
let mut columns_grouped: HashMap<(String, ColumnTypeCategory), GroupedColumns> = HashMap::new();
|
||||
|
||||
let num_columnars = columnar_readers.len();
|
||||
for &(ref column_name, column_type) in required_columns {
|
||||
columns_grouped
|
||||
.entry((column_name.clone(), column_type.into()))
|
||||
.or_insert_with(|| {
|
||||
GroupedColumns::for_category(column_type.into(), columnar_readers.len())
|
||||
})
|
||||
.require_type(column_type)?;
|
||||
}
|
||||
|
||||
for (columnar_id, columnar_reader) in columnar_readers.iter().enumerate() {
|
||||
let column_name_and_handle = columnar_reader.list_columns()?;
|
||||
for (column_name, handle) in column_name_and_handle {
|
||||
let column_type_category: ColumnTypeCategory = handle.column_type().into();
|
||||
let columns = columns_grouped
|
||||
.entry((column_name, column_type_category))
|
||||
.or_insert_with(|| vec![None; num_columnars]);
|
||||
let column_category: ColumnTypeCategory = handle.column_type().into();
|
||||
let column = handle.open()?;
|
||||
columns[columnar_id] = Some(column);
|
||||
columns_grouped
|
||||
.entry((column_name, column_category))
|
||||
.or_insert_with(|| {
|
||||
GroupedColumns::for_category(column_category, columnar_readers.len())
|
||||
})
|
||||
.set_column(columnar_id, column);
|
||||
}
|
||||
}
|
||||
|
||||
let mut merge_columns: BTreeMap<(String, ColumnType), Vec<Option<DynamicColumn>>> =
|
||||
BTreeMap::default();
|
||||
Default::default();
|
||||
|
||||
for ((column_name, col_category), mut columns) in columns_grouped {
|
||||
if col_category == ColumnTypeCategory::Numerical {
|
||||
coerce_numerical_columns_to_same_type(&mut columns);
|
||||
}
|
||||
let column_type = columns
|
||||
.iter()
|
||||
.flatten()
|
||||
.map(|col| col.column_type())
|
||||
.next()
|
||||
.unwrap();
|
||||
merge_columns.insert((column_name, column_type), columns);
|
||||
for ((column_name, _), mut grouped_columns) in columns_grouped {
|
||||
let column_type = grouped_columns.column_type_after_merge();
|
||||
coerce_columns(column_type, &mut grouped_columns.columns)?;
|
||||
merge_columns.insert((column_name, column_type), grouped_columns.columns);
|
||||
}
|
||||
|
||||
Ok(merge_columns)
|
||||
}
|
||||
|
||||
/// Coerce a set of numerical columns to the same type.
|
||||
///
|
||||
/// If all columns are already from the same type, keep this type
|
||||
/// (even if they could all be coerced to i64).
|
||||
fn coerce_numerical_columns_to_same_type(columns: &mut [Option<DynamicColumn>]) {
|
||||
let mut column_types: HashSet<NumericalType> = HashSet::default();
|
||||
let mut compatible_numerical_types = CompatibleNumericalTypes::default();
|
||||
for column in columns.iter().flatten() {
|
||||
let min_value: NumericalValue;
|
||||
let max_value: NumericalValue;
|
||||
match column {
|
||||
DynamicColumn::I64(column) => {
|
||||
min_value = column.min_value().into();
|
||||
max_value = column.max_value().into();
|
||||
}
|
||||
DynamicColumn::U64(column) => {
|
||||
min_value = column.min_value().into();
|
||||
max_value = column.min_value().into();
|
||||
}
|
||||
DynamicColumn::F64(column) => {
|
||||
min_value = column.min_value().into();
|
||||
max_value = column.min_value().into();
|
||||
}
|
||||
DynamicColumn::Bool(_)
|
||||
| DynamicColumn::IpAddr(_)
|
||||
| DynamicColumn::DateTime(_)
|
||||
| DynamicColumn::Bytes(_)
|
||||
| DynamicColumn::Str(_) => {
|
||||
panic!("We expected only numerical columns.");
|
||||
}
|
||||
}
|
||||
column_types.insert(column.column_type().numerical_type().unwrap());
|
||||
compatible_numerical_types.accept_value(min_value);
|
||||
compatible_numerical_types.accept_value(max_value);
|
||||
}
|
||||
if column_types.len() <= 1 {
|
||||
// No need to do anything. The columns are already all from the same type.
|
||||
// This is necessary to let use force a given type.
|
||||
|
||||
// TODO This works in a world where we do not allow a change of schema,
|
||||
// but in the future, we will have to pass some kind of schema to enforce
|
||||
// the logic.
|
||||
return;
|
||||
}
|
||||
let coerce_type = compatible_numerical_types.to_numerical_type();
|
||||
fn coerce_columns(
|
||||
column_type: ColumnType,
|
||||
columns: &mut [Option<DynamicColumn>],
|
||||
) -> io::Result<()> {
|
||||
for column_opt in columns.iter_mut() {
|
||||
if let Some(column) = column_opt.take() {
|
||||
*column_opt = column.coerce_numerical(coerce_type);
|
||||
*column_opt = Some(coerce_column(column_type, column)?);
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn coerce_column(column_type: ColumnType, column: DynamicColumn) -> io::Result<DynamicColumn> {
|
||||
if let Some(numerical_type) = column_type.numerical_type() {
|
||||
column
|
||||
.coerce_numerical(numerical_type)
|
||||
.ok_or_else(|| io::Error::new(io::ErrorKind::InvalidInput, ""))
|
||||
} else {
|
||||
if column.column_type() != column_type {
|
||||
return Err(io::Error::new(
|
||||
io::ErrorKind::InvalidInput,
|
||||
format!(
|
||||
"Cannot coerce column of type `{:?}` to `{column_type:?}`",
|
||||
column.column_type()
|
||||
),
|
||||
));
|
||||
}
|
||||
Ok(column)
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the (min, max) of a column provided it is numerical (i64, u64. f64).
|
||||
///
|
||||
/// The min and the max are simply the numerical value as defined by `ColumnValue::min_value()`,
|
||||
/// and `ColumnValue::max_value()`.
|
||||
///
|
||||
/// It is important to note that these values are only guaranteed to be lower/upper bound
|
||||
/// (as opposed to min/max value).
|
||||
/// If a column is empty, the min and max values are currently set to 0.
|
||||
fn min_max_if_numerical(column: &DynamicColumn) -> Option<(NumericalValue, NumericalValue)> {
|
||||
match column {
|
||||
DynamicColumn::I64(column) => Some((column.min_value().into(), column.max_value().into())),
|
||||
DynamicColumn::U64(column) => Some((column.min_value().into(), column.min_value().into())),
|
||||
DynamicColumn::F64(column) => Some((column.min_value().into(), column.min_value().into())),
|
||||
DynamicColumn::Bool(_)
|
||||
| DynamicColumn::IpAddr(_)
|
||||
| DynamicColumn::DateTime(_)
|
||||
| DynamicColumn::Bytes(_)
|
||||
| DynamicColumn::Str(_) => None,
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
|
||||
@@ -1,107 +0,0 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use fastfield_codecs::Column;
|
||||
use itertools::Itertools;
|
||||
|
||||
use crate::indexer::doc_id_mapping::SegmentDocIdMapping;
|
||||
use crate::SegmentReader;
|
||||
|
||||
pub(crate) struct RemappedDocIdColumn<'a> {
|
||||
doc_id_mapping: &'a SegmentDocIdMapping,
|
||||
fast_field_readers: Vec<Arc<dyn Column<u64>>>,
|
||||
min_value: u64,
|
||||
max_value: u64,
|
||||
num_vals: u32,
|
||||
}
|
||||
|
||||
fn compute_min_max_val(
|
||||
u64_reader: &dyn Column<u64>,
|
||||
segment_reader: &SegmentReader,
|
||||
) -> Option<(u64, u64)> {
|
||||
if segment_reader.max_doc() == 0 {
|
||||
return None;
|
||||
}
|
||||
|
||||
if segment_reader.alive_bitset().is_none() {
|
||||
// no deleted documents,
|
||||
// we can use the previous min_val, max_val.
|
||||
return Some((u64_reader.min_value(), u64_reader.max_value()));
|
||||
}
|
||||
// some deleted documents,
|
||||
// we need to recompute the max / min
|
||||
segment_reader
|
||||
.doc_ids_alive()
|
||||
.map(|doc_id| u64_reader.get_val(doc_id))
|
||||
.minmax()
|
||||
.into_option()
|
||||
}
|
||||
|
||||
impl<'a> RemappedDocIdColumn<'a> {
|
||||
pub(crate) fn new(
|
||||
readers: &'a [SegmentReader],
|
||||
doc_id_mapping: &'a SegmentDocIdMapping,
|
||||
field: &str,
|
||||
) -> Self {
|
||||
let (min_value, max_value) = readers
|
||||
.iter()
|
||||
.filter_map(|reader| {
|
||||
let u64_reader: Arc<dyn Column<u64>> =
|
||||
reader.fast_fields().typed_fast_field_reader(field).expect(
|
||||
"Failed to find a reader for single fast field. This is a tantivy bug and \
|
||||
it should never happen.",
|
||||
);
|
||||
compute_min_max_val(&*u64_reader, reader)
|
||||
})
|
||||
.reduce(|a, b| (a.0.min(b.0), a.1.max(b.1)))
|
||||
.expect("Unexpected error, empty readers in IndexMerger");
|
||||
|
||||
let fast_field_readers = readers
|
||||
.iter()
|
||||
.map(|reader| {
|
||||
let u64_reader: Arc<dyn Column<u64>> =
|
||||
reader.fast_fields().typed_fast_field_reader(field).expect(
|
||||
"Failed to find a reader for single fast field. This is a tantivy bug and \
|
||||
it should never happen.",
|
||||
);
|
||||
u64_reader
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
RemappedDocIdColumn {
|
||||
doc_id_mapping,
|
||||
fast_field_readers,
|
||||
min_value,
|
||||
max_value,
|
||||
num_vals: doc_id_mapping.len() as u32,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<'a> Column for RemappedDocIdColumn<'a> {
|
||||
fn get_val(&self, _doc: u32) -> u64 {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
fn iter(&self) -> Box<dyn Iterator<Item = u64> + '_> {
|
||||
Box::new(
|
||||
self.doc_id_mapping
|
||||
.iter_old_doc_addrs()
|
||||
.map(|old_doc_addr| {
|
||||
let fast_field_reader =
|
||||
&self.fast_field_readers[old_doc_addr.segment_ord as usize];
|
||||
fast_field_reader.get_val(old_doc_addr.doc_id)
|
||||
}),
|
||||
)
|
||||
}
|
||||
fn min_value(&self) -> u64 {
|
||||
self.min_value
|
||||
}
|
||||
|
||||
fn max_value(&self) -> u64 {
|
||||
self.max_value
|
||||
}
|
||||
|
||||
fn num_vals(&self) -> u32 {
|
||||
self.num_vals
|
||||
}
|
||||
}
|
||||
@@ -24,7 +24,7 @@ fn test_column_coercion_to_u64() {
|
||||
// u64 type
|
||||
let columnar2 = make_columnar("numbers", &[u64::MAX]);
|
||||
let column_map: BTreeMap<(String, ColumnType), Vec<Option<DynamicColumn>>> =
|
||||
group_columns_for_merge(&[&columnar1, &columnar2]).unwrap();
|
||||
group_columns_for_merge(&[&columnar1, &columnar2], &[]).unwrap();
|
||||
assert_eq!(column_map.len(), 1);
|
||||
assert!(column_map.contains_key(&("numbers".to_string(), ColumnType::U64)));
|
||||
}
|
||||
@@ -34,7 +34,7 @@ fn test_column_no_coercion_if_all_the_same() {
|
||||
let columnar1 = make_columnar("numbers", &[1u64]);
|
||||
let columnar2 = make_columnar("numbers", &[2u64]);
|
||||
let column_map: BTreeMap<(String, ColumnType), Vec<Option<DynamicColumn>>> =
|
||||
group_columns_for_merge(&[&columnar1, &columnar2]).unwrap();
|
||||
group_columns_for_merge(&[&columnar1, &columnar2], &[]).unwrap();
|
||||
assert_eq!(column_map.len(), 1);
|
||||
assert!(column_map.contains_key(&("numbers".to_string(), ColumnType::U64)));
|
||||
}
|
||||
@@ -44,17 +44,74 @@ fn test_column_coercion_to_i64() {
|
||||
let columnar1 = make_columnar("numbers", &[-1i64]);
|
||||
let columnar2 = make_columnar("numbers", &[2u64]);
|
||||
let column_map: BTreeMap<(String, ColumnType), Vec<Option<DynamicColumn>>> =
|
||||
group_columns_for_merge(&[&columnar1, &columnar2]).unwrap();
|
||||
group_columns_for_merge(&[&columnar1, &columnar2], &[]).unwrap();
|
||||
assert_eq!(column_map.len(), 1);
|
||||
assert!(column_map.contains_key(&("numbers".to_string(), ColumnType::I64)));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_impossible_coercion_returns_an_error() {
|
||||
let columnar1 = make_columnar("numbers", &[u64::MAX]);
|
||||
let group_error =
|
||||
group_columns_for_merge(&[&columnar1], &[("numbers".to_string(), ColumnType::I64)])
|
||||
.map(|_| ())
|
||||
.unwrap_err();
|
||||
assert_eq!(group_error.kind(), io::ErrorKind::InvalidInput);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_group_columns_with_required_column() {
|
||||
let columnar1 = make_columnar("numbers", &[1i64]);
|
||||
let columnar2 = make_columnar("numbers", &[2u64]);
|
||||
let column_map: BTreeMap<(String, ColumnType), Vec<Option<DynamicColumn>>> =
|
||||
group_columns_for_merge(
|
||||
&[&columnar1, &columnar2],
|
||||
&[("numbers".to_string(), ColumnType::U64)],
|
||||
)
|
||||
.unwrap();
|
||||
assert_eq!(column_map.len(), 1);
|
||||
assert!(column_map.contains_key(&("numbers".to_string(), ColumnType::U64)));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_group_columns_required_column_with_no_existing_columns() {
|
||||
let columnar1 = make_columnar("numbers", &[2u64]);
|
||||
let columnar2 = make_columnar("numbers", &[2u64]);
|
||||
let column_map: BTreeMap<(String, ColumnType), Vec<Option<DynamicColumn>>> =
|
||||
group_columns_for_merge(
|
||||
&[&columnar1, &columnar2],
|
||||
&[("required_col".to_string(), ColumnType::Str)],
|
||||
)
|
||||
.unwrap();
|
||||
assert_eq!(column_map.len(), 2);
|
||||
let columns = column_map
|
||||
.get(&("required_col".to_string(), ColumnType::Str))
|
||||
.unwrap();
|
||||
assert_eq!(columns.len(), 2);
|
||||
assert!(columns[0].is_none());
|
||||
assert!(columns[1].is_none());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_group_columns_required_column_is_above_all_columns_have_the_same_type_rule() {
|
||||
let columnar1 = make_columnar("numbers", &[2i64]);
|
||||
let columnar2 = make_columnar("numbers", &[2i64]);
|
||||
let column_map: BTreeMap<(String, ColumnType), Vec<Option<DynamicColumn>>> =
|
||||
group_columns_for_merge(
|
||||
&[&columnar1, &columnar2],
|
||||
&[("numbers".to_string(), ColumnType::U64)],
|
||||
)
|
||||
.unwrap();
|
||||
assert_eq!(column_map.len(), 1);
|
||||
assert!(column_map.contains_key(&("numbers".to_string(), ColumnType::U64)));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_missing_column() {
|
||||
let columnar1 = make_columnar("numbers", &[-1i64]);
|
||||
let columnar2 = make_columnar("numbers2", &[2u64]);
|
||||
let column_map: BTreeMap<(String, ColumnType), Vec<Option<DynamicColumn>>> =
|
||||
group_columns_for_merge(&[&columnar1, &columnar2]).unwrap();
|
||||
group_columns_for_merge(&[&columnar1, &columnar2], &[]).unwrap();
|
||||
assert_eq!(column_map.len(), 2);
|
||||
assert!(column_map.contains_key(&("numbers".to_string(), ColumnType::I64)));
|
||||
{
|
||||
@@ -101,7 +158,7 @@ fn make_byte_columnar_multiple_columns(columns: &[(&str, &[&[&[u8]]])]) -> Colum
|
||||
for (column_name, column_values) in columns {
|
||||
for (row_id, vals) in column_values.iter().enumerate() {
|
||||
for val in vals.iter() {
|
||||
dataframe_writer.record_bytes(row_id as u32, column_name, *val);
|
||||
dataframe_writer.record_bytes(row_id as u32, column_name, val);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -122,7 +179,7 @@ fn make_text_columnar_multiple_columns(columns: &[(&str, &[&[&str]])]) -> Column
|
||||
for (column_name, column_values) in columns {
|
||||
for (row_id, vals) in column_values.iter().enumerate() {
|
||||
for val in vals.iter() {
|
||||
dataframe_writer.record_str(row_id as u32, column_name, *val);
|
||||
dataframe_writer.record_str(row_id as u32, column_name, val);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -151,6 +208,7 @@ fn test_merge_columnar_numbers() {
|
||||
let stack_merge_order = StackMergeOrder::stack(columnars);
|
||||
crate::columnar::merge_columnar(
|
||||
columnars,
|
||||
&[],
|
||||
MergeRowOrder::Stack(stack_merge_order),
|
||||
&mut buffer,
|
||||
)
|
||||
@@ -176,6 +234,7 @@ fn test_merge_columnar_texts() {
|
||||
let stack_merge_order = StackMergeOrder::stack(columnars);
|
||||
crate::columnar::merge_columnar(
|
||||
columnars,
|
||||
&[],
|
||||
MergeRowOrder::Stack(stack_merge_order),
|
||||
&mut buffer,
|
||||
)
|
||||
@@ -220,6 +279,7 @@ fn test_merge_columnar_byte() {
|
||||
let stack_merge_order = StackMergeOrder::stack(columnars);
|
||||
crate::columnar::merge_columnar(
|
||||
columnars,
|
||||
&[],
|
||||
MergeRowOrder::Stack(stack_merge_order),
|
||||
&mut buffer,
|
||||
)
|
||||
|
||||
@@ -1 +0,0 @@
|
||||
|
||||
@@ -1,7 +1,6 @@
|
||||
mod column_type;
|
||||
mod format_version;
|
||||
mod merge;
|
||||
mod merge_index;
|
||||
mod reader;
|
||||
mod writer;
|
||||
|
||||
|
||||
@@ -21,6 +21,32 @@ pub struct ColumnarReader {
|
||||
num_rows: RowId,
|
||||
}
|
||||
|
||||
/// Functions by both the async/sync code listing columns.
|
||||
/// It takes a stream from the column sstable and return the list of
|
||||
/// `DynamicColumn` available in it.
|
||||
fn read_all_columns_in_stream(
|
||||
mut stream: sstable::Streamer<'_, RangeSSTable>,
|
||||
column_data: &FileSlice,
|
||||
) -> io::Result<Vec<DynamicColumnHandle>> {
|
||||
let mut results = Vec::new();
|
||||
while stream.advance() {
|
||||
let key_bytes: &[u8] = stream.key();
|
||||
let Some(column_code) = key_bytes.last().copied() else {
|
||||
return Err(io_invalid_data("Empty column name.".to_string()));
|
||||
};
|
||||
let column_type = ColumnType::try_from_code(column_code)
|
||||
.map_err(|_| io_invalid_data(format!("Unknown column code `{column_code}`")))?;
|
||||
let range = stream.value();
|
||||
let file_slice = column_data.slice(range.start as usize..range.end as usize);
|
||||
let dynamic_column_handle = DynamicColumnHandle {
|
||||
file_slice,
|
||||
column_type,
|
||||
};
|
||||
results.push(dynamic_column_handle);
|
||||
}
|
||||
Ok(results)
|
||||
}
|
||||
|
||||
impl ColumnarReader {
|
||||
/// Opens a new Columnar file.
|
||||
pub fn open<F>(file_slice: F) -> io::Result<ColumnarReader>
|
||||
@@ -76,11 +102,7 @@ impl ColumnarReader {
|
||||
Ok(results)
|
||||
}
|
||||
|
||||
/// Get all columns for the given column name.
|
||||
///
|
||||
/// There can be more than one column associated to a given column name, provided they have
|
||||
/// different types.
|
||||
pub fn read_columns(&self, column_name: &str) -> io::Result<Vec<DynamicColumnHandle>> {
|
||||
fn stream_for_column_range(&self, column_name: &str) -> sstable::StreamerBuilder<RangeSSTable> {
|
||||
// Each column is a associated to a given `column_key`,
|
||||
// that starts by `column_name\0column_header`.
|
||||
//
|
||||
@@ -89,36 +111,35 @@ impl ColumnarReader {
|
||||
//
|
||||
// This is in turn equivalent to searching for the range
|
||||
// `[column_name,\0`..column_name\1)`.
|
||||
|
||||
// TODO can we get some more generic `prefix(..)` logic in the dictioanry.
|
||||
// TODO can we get some more generic `prefix(..)` logic in the dictionary.
|
||||
let mut start_key = column_name.to_string();
|
||||
start_key.push('\0');
|
||||
let mut end_key = column_name.to_string();
|
||||
end_key.push(1u8 as char);
|
||||
let mut stream = self
|
||||
.column_dictionary
|
||||
self.column_dictionary
|
||||
.range()
|
||||
.ge(start_key.as_bytes())
|
||||
.lt(end_key.as_bytes())
|
||||
.into_stream()?;
|
||||
let mut results = Vec::new();
|
||||
while stream.advance() {
|
||||
let key_bytes: &[u8] = stream.key();
|
||||
assert!(key_bytes.starts_with(start_key.as_bytes()));
|
||||
let column_code: u8 = key_bytes.last().cloned().unwrap();
|
||||
let column_type = ColumnType::try_from_code(column_code)
|
||||
.map_err(|_| io_invalid_data(format!("Unknown column code `{column_code}`")))?;
|
||||
let range = stream.value().clone();
|
||||
let file_slice = self
|
||||
.column_data
|
||||
.slice(range.start as usize..range.end as usize);
|
||||
let dynamic_column_handle = DynamicColumnHandle {
|
||||
file_slice,
|
||||
column_type,
|
||||
};
|
||||
results.push(dynamic_column_handle);
|
||||
}
|
||||
Ok(results)
|
||||
}
|
||||
|
||||
pub async fn read_columns_async(
|
||||
&self,
|
||||
column_name: &str,
|
||||
) -> io::Result<Vec<DynamicColumnHandle>> {
|
||||
let stream = self
|
||||
.stream_for_column_range(column_name)
|
||||
.into_stream_async()
|
||||
.await?;
|
||||
read_all_columns_in_stream(stream, &self.column_data)
|
||||
}
|
||||
|
||||
/// Get all columns for the given column name.
|
||||
///
|
||||
/// There can be more than one column associated to a given column name, provided they have
|
||||
/// different types.
|
||||
pub fn read_columns(&self, column_name: &str) -> io::Result<Vec<DynamicColumnHandle>> {
|
||||
let stream = self.stream_for_column_range(column_name).into_stream()?;
|
||||
read_all_columns_in_stream(stream, &self.column_data)
|
||||
}
|
||||
|
||||
/// Return the number of columns in the columnar.
|
||||
|
||||
@@ -310,7 +310,7 @@ mod tests {
|
||||
buffer.extend_from_slice(b"234234");
|
||||
let mut bytes = &buffer[..];
|
||||
let serdeser_symbol = ColumnOperation::deserialize(&mut bytes).unwrap();
|
||||
assert_eq!(bytes.len() + buf.as_ref().len() as usize, buffer.len());
|
||||
assert_eq!(bytes.len() + buf.as_ref().len(), buffer.len());
|
||||
assert_eq!(column_op, serdeser_symbol);
|
||||
}
|
||||
|
||||
@@ -341,7 +341,7 @@ mod tests {
|
||||
fn test_column_operation_unordered_aux(val: u32, expected_len: usize) {
|
||||
let column_op = ColumnOperation::Value(UnorderedId(val));
|
||||
let minibuf = column_op.serialize();
|
||||
assert_eq!(minibuf.as_ref().len() as usize, expected_len);
|
||||
assert_eq!({ minibuf.as_ref().len() }, expected_len);
|
||||
let mut buf = minibuf.as_ref().to_vec();
|
||||
buf.extend_from_slice(&[2, 2, 2, 2, 2, 2]);
|
||||
let mut cursor = &buf[..];
|
||||
|
||||
@@ -761,7 +761,7 @@ mod tests {
|
||||
assert_eq!(column_writer.get_cardinality(3), Cardinality::Full);
|
||||
let mut buffer = Vec::new();
|
||||
let symbols: Vec<ColumnOperation<NumericalValue>> = column_writer
|
||||
.operation_iterator(&mut arena, None, &mut buffer)
|
||||
.operation_iterator(&arena, None, &mut buffer)
|
||||
.collect();
|
||||
assert_eq!(symbols.len(), 6);
|
||||
assert!(matches!(symbols[0], ColumnOperation::NewDoc(0u32)));
|
||||
@@ -790,7 +790,7 @@ mod tests {
|
||||
assert_eq!(column_writer.get_cardinality(3), Cardinality::Optional);
|
||||
let mut buffer = Vec::new();
|
||||
let symbols: Vec<ColumnOperation<NumericalValue>> = column_writer
|
||||
.operation_iterator(&mut arena, None, &mut buffer)
|
||||
.operation_iterator(&arena, None, &mut buffer)
|
||||
.collect();
|
||||
assert_eq!(symbols.len(), 4);
|
||||
assert!(matches!(symbols[0], ColumnOperation::NewDoc(1u32)));
|
||||
@@ -813,7 +813,7 @@ mod tests {
|
||||
assert_eq!(column_writer.get_cardinality(2), Cardinality::Optional);
|
||||
let mut buffer = Vec::new();
|
||||
let symbols: Vec<ColumnOperation<NumericalValue>> = column_writer
|
||||
.operation_iterator(&mut arena, None, &mut buffer)
|
||||
.operation_iterator(&arena, None, &mut buffer)
|
||||
.collect();
|
||||
assert_eq!(symbols.len(), 2);
|
||||
assert!(matches!(symbols[0], ColumnOperation::NewDoc(0u32)));
|
||||
@@ -832,7 +832,7 @@ mod tests {
|
||||
assert_eq!(column_writer.get_cardinality(1), Cardinality::Multivalued);
|
||||
let mut buffer = Vec::new();
|
||||
let symbols: Vec<ColumnOperation<NumericalValue>> = column_writer
|
||||
.operation_iterator(&mut arena, None, &mut buffer)
|
||||
.operation_iterator(&arena, None, &mut buffer)
|
||||
.collect();
|
||||
assert_eq!(symbols.len(), 3);
|
||||
assert!(matches!(symbols[0], ColumnOperation::NewDoc(0u32)));
|
||||
|
||||
@@ -150,11 +150,7 @@ mod tests {
|
||||
multivalued_value_index_builder.record_row(2u32);
|
||||
multivalued_value_index_builder.record_value();
|
||||
assert_eq!(
|
||||
multivalued_value_index_builder
|
||||
.finish(4u32)
|
||||
.iter()
|
||||
.copied()
|
||||
.collect::<Vec<u32>>(),
|
||||
multivalued_value_index_builder.finish(4u32).to_vec(),
|
||||
vec![0, 0, 2, 3, 3]
|
||||
);
|
||||
multivalued_value_index_builder.reset();
|
||||
@@ -162,11 +158,7 @@ mod tests {
|
||||
multivalued_value_index_builder.record_value();
|
||||
multivalued_value_index_builder.record_value();
|
||||
assert_eq!(
|
||||
multivalued_value_index_builder
|
||||
.finish(4u32)
|
||||
.iter()
|
||||
.copied()
|
||||
.collect::<Vec<u32>>(),
|
||||
multivalued_value_index_builder.finish(4u32).to_vec(),
|
||||
vec![0, 0, 0, 2, 2]
|
||||
);
|
||||
}
|
||||
|
||||
@@ -206,10 +206,9 @@ impl DynamicColumnHandle {
|
||||
self.open_internal(column_bytes)
|
||||
}
|
||||
|
||||
// TODO rename load_async
|
||||
pub async fn open_async(&self) -> io::Result<DynamicColumn> {
|
||||
let column_bytes: OwnedBytes = self.file_slice.read_bytes_async().await?;
|
||||
self.open_internal(column_bytes)
|
||||
#[doc(hidden)]
|
||||
pub fn file_slice(&self) -> &FileSlice {
|
||||
&self.file_slice
|
||||
}
|
||||
|
||||
/// Returns the `u64` fast field reader reader associated with `fields` of types
|
||||
|
||||
@@ -29,7 +29,7 @@ pub enum DatePrecision {
|
||||
/// All constructors and conversions are provided as explicit
|
||||
/// functions and not by implementing any `From`/`Into` traits
|
||||
/// to prevent unintended usage.
|
||||
#[derive(Clone, Default, Copy, PartialEq, Eq, PartialOrd, Ord)]
|
||||
#[derive(Clone, Default, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)]
|
||||
pub struct DateTime {
|
||||
// Timestamp in microseconds.
|
||||
pub(crate) timestamp_micros: i64,
|
||||
|
||||
@@ -109,6 +109,21 @@ pub fn u64_to_f64(val: u64) -> f64 {
|
||||
})
|
||||
}
|
||||
|
||||
/// Replaces a given byte in the `bytes` slice of bytes.
|
||||
///
|
||||
/// This function assumes that the needle is rarely contained in the bytes string
|
||||
/// and offers a fast path if the needle is not present.
|
||||
pub fn replace_in_place(needle: u8, replacement: u8, bytes: &mut [u8]) {
|
||||
if !bytes.contains(&needle) {
|
||||
return;
|
||||
}
|
||||
for b in bytes {
|
||||
if *b == needle {
|
||||
*b = replacement;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
pub mod test {
|
||||
|
||||
@@ -173,4 +188,20 @@ pub mod test {
|
||||
assert!(f64_to_u64(-2.0) < f64_to_u64(1.0));
|
||||
assert!(f64_to_u64(-2.0) < f64_to_u64(-1.5));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_replace_in_place() {
|
||||
let test_aux = |before_replacement: &[u8], expected: &[u8]| {
|
||||
let mut bytes: Vec<u8> = before_replacement.to_vec();
|
||||
super::replace_in_place(b'b', b'c', &mut bytes);
|
||||
assert_eq!(&bytes[..], expected);
|
||||
};
|
||||
test_aux(b"", b"");
|
||||
test_aux(b"b", b"c");
|
||||
test_aux(b"baaa", b"caaa");
|
||||
test_aux(b"aaab", b"aaac");
|
||||
test_aux(b"aaabaa", b"aaacaa");
|
||||
test_aux(b"aaaaaa", b"aaaaaa");
|
||||
test_aux(b"bbbb", b"cccc");
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,129 +1,319 @@
|
||||
// # Aggregation example
|
||||
//
|
||||
// This example shows how you can use built-in aggregations.
|
||||
// We will use range buckets and compute the average in each bucket.
|
||||
//
|
||||
// We will use nested aggregations with buckets and metrics:
|
||||
// - Range buckets and compute the average in each bucket.
|
||||
// - Term aggregation and compute the min price in each bucket
|
||||
// ---
|
||||
|
||||
use serde_json::Value;
|
||||
use serde_json::{Deserializer, Value};
|
||||
use tantivy::aggregation::agg_req::{
|
||||
Aggregation, Aggregations, BucketAggregation, BucketAggregationType, MetricAggregation,
|
||||
RangeAggregation,
|
||||
};
|
||||
use tantivy::aggregation::agg_result::AggregationResults;
|
||||
use tantivy::aggregation::bucket::RangeAggregationRange;
|
||||
use tantivy::aggregation::metric::AverageAggregation;
|
||||
use tantivy::aggregation::AggregationCollector;
|
||||
use tantivy::query::TermQuery;
|
||||
use tantivy::schema::{self, IndexRecordOption, Schema, TextFieldIndexing};
|
||||
use tantivy::{doc, Index, Term};
|
||||
use tantivy::query::AllQuery;
|
||||
use tantivy::schema::{self, IndexRecordOption, Schema, TextFieldIndexing, FAST};
|
||||
use tantivy::Index;
|
||||
|
||||
fn main() -> tantivy::Result<()> {
|
||||
// # Create Schema
|
||||
//
|
||||
// Lets create a schema for a footwear shop, with 4 fields: name, category, stock and price.
|
||||
// category, stock and price will be fast fields as that's the requirement
|
||||
// for aggregation queries.
|
||||
//
|
||||
|
||||
let mut schema_builder = Schema::builder();
|
||||
// In preparation of the `TermsAggregation`, the category field is configured with:
|
||||
// - `set_fast`
|
||||
// - `raw` tokenizer
|
||||
//
|
||||
// The tokenizer is set to "raw", because the fast field uses the same dictionary as the
|
||||
// inverted index. (This behaviour will change in tantivy 0.20, where the fast field will
|
||||
// always be raw tokenized independent from the regular tokenizing)
|
||||
//
|
||||
let text_fieldtype = schema::TextOptions::default()
|
||||
.set_indexing_options(
|
||||
TextFieldIndexing::default().set_index_option(IndexRecordOption::WithFreqs),
|
||||
TextFieldIndexing::default()
|
||||
.set_index_option(IndexRecordOption::WithFreqs)
|
||||
.set_tokenizer("raw"),
|
||||
)
|
||||
.set_fast()
|
||||
.set_stored();
|
||||
let text_field = schema_builder.add_text_field("text", text_fieldtype);
|
||||
let score_fieldtype = crate::schema::NumericOptions::default().set_fast();
|
||||
let highscore_field = schema_builder.add_f64_field("highscore", score_fieldtype.clone());
|
||||
let price_field = schema_builder.add_f64_field("price", score_fieldtype);
|
||||
schema_builder.add_text_field("category", text_fieldtype);
|
||||
schema_builder.add_f64_field("stock", FAST);
|
||||
schema_builder.add_f64_field("price", FAST);
|
||||
|
||||
let schema = schema_builder.build();
|
||||
|
||||
// # Indexing documents
|
||||
//
|
||||
// Lets index a bunch of documents for this example.
|
||||
let index = Index::create_in_ram(schema);
|
||||
let index = Index::create_in_ram(schema.clone());
|
||||
|
||||
let data = r#"{
|
||||
"name": "Almond Toe Court Shoes, Patent Black",
|
||||
"category": "Womens Footwear",
|
||||
"price": 99.00,
|
||||
"stock": 5
|
||||
}
|
||||
{
|
||||
"name": "Suede Shoes, Blue",
|
||||
"category": "Womens Footwear",
|
||||
"price": 42.00,
|
||||
"stock": 4
|
||||
}
|
||||
{
|
||||
"name": "Leather Driver Saddle Loafers, Tan",
|
||||
"category": "Mens Footwear",
|
||||
"price": 34.00,
|
||||
"stock": 12
|
||||
}
|
||||
{
|
||||
"name": "Flip Flops, Red",
|
||||
"category": "Mens Footwear",
|
||||
"price": 19.00,
|
||||
"stock": 6
|
||||
}
|
||||
{
|
||||
"name": "Flip Flops, Blue",
|
||||
"category": "Mens Footwear",
|
||||
"price": 19.00,
|
||||
"stock": 0
|
||||
}
|
||||
{
|
||||
"name": "Gold Button Cardigan, Black",
|
||||
"category": "Womens Casualwear",
|
||||
"price": 167.00,
|
||||
"stock": 6
|
||||
}
|
||||
{
|
||||
"name": "Cotton Shorts, Medium Red",
|
||||
"category": "Womens Casualwear",
|
||||
"price": 30.00,
|
||||
"stock": 5
|
||||
}
|
||||
{
|
||||
"name": "Fine Stripe Short SleeveShirt, Grey",
|
||||
"category": "Mens Casualwear",
|
||||
"price": 49.99,
|
||||
"stock": 9
|
||||
}
|
||||
{
|
||||
"name": "Fine Stripe Short SleeveShirt, Green",
|
||||
"category": "Mens Casualwear",
|
||||
"price": 49.99,
|
||||
"offer": 39.99,
|
||||
"stock": 9
|
||||
}
|
||||
{
|
||||
"name": "Sharkskin Waistcoat, Charcoal",
|
||||
"category": "Mens Formalwear",
|
||||
"price": 75.00,
|
||||
"stock": 2
|
||||
}
|
||||
{
|
||||
"name": "Lightweight Patch PocketBlazer, Deer",
|
||||
"category": "Mens Formalwear",
|
||||
"price": 175.50,
|
||||
"stock": 1
|
||||
}
|
||||
{
|
||||
"name": "Bird Print Dress, Black",
|
||||
"category": "Womens Formalwear",
|
||||
"price": 270.00,
|
||||
"stock": 10
|
||||
}
|
||||
{
|
||||
"name": "Mid Twist Cut-Out Dress, Pink",
|
||||
"category": "Womens Formalwear",
|
||||
"price": 540.00,
|
||||
"stock": 5
|
||||
}"#;
|
||||
|
||||
let stream = Deserializer::from_str(data).into_iter::<Value>();
|
||||
|
||||
let mut index_writer = index.writer(50_000_000)?;
|
||||
// writing the segment
|
||||
index_writer.add_document(doc!(
|
||||
text_field => "cool",
|
||||
highscore_field => 1f64,
|
||||
price_field => 0f64,
|
||||
))?;
|
||||
index_writer.add_document(doc!(
|
||||
text_field => "cool",
|
||||
highscore_field => 3f64,
|
||||
price_field => 1f64,
|
||||
))?;
|
||||
index_writer.add_document(doc!(
|
||||
text_field => "cool",
|
||||
highscore_field => 5f64,
|
||||
price_field => 1f64,
|
||||
))?;
|
||||
index_writer.add_document(doc!(
|
||||
text_field => "nohit",
|
||||
highscore_field => 6f64,
|
||||
price_field => 2f64,
|
||||
))?;
|
||||
index_writer.add_document(doc!(
|
||||
text_field => "cool",
|
||||
highscore_field => 7f64,
|
||||
price_field => 2f64,
|
||||
))?;
|
||||
index_writer.commit()?;
|
||||
index_writer.add_document(doc!(
|
||||
text_field => "cool",
|
||||
highscore_field => 11f64,
|
||||
price_field => 10f64,
|
||||
))?;
|
||||
index_writer.add_document(doc!(
|
||||
text_field => "cool",
|
||||
highscore_field => 14f64,
|
||||
price_field => 15f64,
|
||||
))?;
|
||||
|
||||
index_writer.add_document(doc!(
|
||||
text_field => "cool",
|
||||
highscore_field => 15f64,
|
||||
price_field => 20f64,
|
||||
))?;
|
||||
let mut num_indexed = 0;
|
||||
for value in stream {
|
||||
let doc = schema.parse_document(&serde_json::to_string(&value.unwrap())?)?;
|
||||
index_writer.add_document(doc)?;
|
||||
num_indexed += 1;
|
||||
if num_indexed > 4 {
|
||||
// Writing the first segment
|
||||
index_writer.commit()?;
|
||||
}
|
||||
}
|
||||
|
||||
// Writing the second segment
|
||||
index_writer.commit()?;
|
||||
|
||||
// We have two segments now. The `AggregationCollector` will run the aggregation on each
|
||||
// segment and then merge the results into an `IntermediateAggregationResult`.
|
||||
|
||||
let reader = index.reader()?;
|
||||
let text_field = reader.searcher().schema().get_field("text").unwrap();
|
||||
let searcher = reader.searcher();
|
||||
// ---
|
||||
// # Aggregation Query
|
||||
//
|
||||
//
|
||||
// We can construct the query by building the request structure or by deserializing from JSON.
|
||||
// The JSON API is more stable and therefore recommended.
|
||||
//
|
||||
// ## Request 1
|
||||
|
||||
let term_query = TermQuery::new(
|
||||
Term::from_field_text(text_field, "cool"),
|
||||
IndexRecordOption::Basic,
|
||||
);
|
||||
let agg_req_str = r#"
|
||||
{
|
||||
"group_by_stock": {
|
||||
"aggs": {
|
||||
"average_price": { "avg": { "field": "price" } }
|
||||
},
|
||||
"range": {
|
||||
"field": "stock",
|
||||
"ranges": [
|
||||
{ "key": "few", "to": 1.0 },
|
||||
{ "key": "some", "from": 1.0, "to": 10.0 },
|
||||
{ "key": "many", "from": 10.0 }
|
||||
]
|
||||
}
|
||||
}
|
||||
} "#;
|
||||
|
||||
let sub_agg_req_1: Aggregations = vec![(
|
||||
"average_price".to_string(),
|
||||
Aggregation::Metric(MetricAggregation::Average(
|
||||
AverageAggregation::from_field_name("price".to_string()),
|
||||
)),
|
||||
)]
|
||||
.into_iter()
|
||||
.collect();
|
||||
// In this Aggregation we want to get the average price for different groups, depending on how
|
||||
// many items are in stock. We define custom ranges `few`, `some`, `many` via the
|
||||
// range aggregation.
|
||||
// For every bucket we want the average price, so we create a nested metric aggregation on the
|
||||
// range bucket aggregation. Only buckets support nested aggregations.
|
||||
// ### Request JSON API
|
||||
//
|
||||
|
||||
let agg_req_1: Aggregations = vec![(
|
||||
"score_ranges".to_string(),
|
||||
let agg_req: Aggregations = serde_json::from_str(agg_req_str)?;
|
||||
let collector = AggregationCollector::from_aggs(agg_req, None);
|
||||
|
||||
let agg_res: AggregationResults = searcher.search(&AllQuery, &collector).unwrap();
|
||||
let res2: Value = serde_json::to_value(agg_res)?;
|
||||
|
||||
// ### Request Rust API
|
||||
//
|
||||
// This is exactly the same request as above, but via the rust structures.
|
||||
//
|
||||
|
||||
let agg_req: Aggregations = vec![(
|
||||
"group_by_stock".to_string(),
|
||||
Aggregation::Bucket(BucketAggregation {
|
||||
bucket_agg: BucketAggregationType::Range(RangeAggregation {
|
||||
field: "highscore".to_string(),
|
||||
field: "stock".to_string(),
|
||||
ranges: vec![
|
||||
(-1f64..9f64).into(),
|
||||
(9f64..14f64).into(),
|
||||
(14f64..20f64).into(),
|
||||
RangeAggregationRange {
|
||||
key: Some("few".into()),
|
||||
from: None,
|
||||
to: Some(1f64),
|
||||
},
|
||||
RangeAggregationRange {
|
||||
key: Some("some".into()),
|
||||
from: Some(1f64),
|
||||
to: Some(10f64),
|
||||
},
|
||||
RangeAggregationRange {
|
||||
key: Some("many".into()),
|
||||
from: Some(10f64),
|
||||
to: None,
|
||||
},
|
||||
],
|
||||
..Default::default()
|
||||
}),
|
||||
sub_aggregation: sub_agg_req_1,
|
||||
sub_aggregation: vec![(
|
||||
"average_price".to_string(),
|
||||
Aggregation::Metric(MetricAggregation::Average(
|
||||
AverageAggregation::from_field_name("price".to_string()),
|
||||
)),
|
||||
)]
|
||||
.into_iter()
|
||||
.collect(),
|
||||
}),
|
||||
)]
|
||||
.into_iter()
|
||||
.collect();
|
||||
|
||||
let collector = AggregationCollector::from_aggs(agg_req_1, None, index.schema());
|
||||
let collector = AggregationCollector::from_aggs(agg_req, None);
|
||||
// We use the `AllQuery` which will pass all documents to the AggregationCollector.
|
||||
let agg_res: AggregationResults = searcher.search(&AllQuery, &collector).unwrap();
|
||||
|
||||
let searcher = reader.searcher();
|
||||
let agg_res: AggregationResults = searcher.search(&term_query, &collector).unwrap();
|
||||
let res1: Value = serde_json::to_value(agg_res)?;
|
||||
|
||||
// ### Aggregation Result
|
||||
//
|
||||
// The resulting structure deserializes in the same JSON format as elastic search.
|
||||
//
|
||||
let expected_res = r#"
|
||||
{
|
||||
"group_by_stock":{
|
||||
"buckets":[
|
||||
{"average_price":{"value":19.0},"doc_count":1,"key":"few","to":1.0},
|
||||
{"average_price":{"value":124.748},"doc_count":10,"from":1.0,"key":"some","to":10.0},
|
||||
{"average_price":{"value":152.0},"doc_count":2,"from":10.0,"key":"many"}
|
||||
]
|
||||
}
|
||||
}
|
||||
"#;
|
||||
let expected_json: Value = serde_json::from_str(expected_res)?;
|
||||
assert_eq!(expected_json, res1);
|
||||
assert_eq!(expected_json, res2);
|
||||
|
||||
// ### Request 2
|
||||
//
|
||||
// Now we are interested in the minimum price per category, so we create a bucket per
|
||||
// category via `TermsAggregation`. We are interested in the highest minimum prices, and set the
|
||||
// order of the buckets `"order": { "min_price": "desc" }` to be sorted by the the metric of
|
||||
// the sub aggregation. (awesome)
|
||||
//
|
||||
let agg_req_str = r#"
|
||||
{
|
||||
"min_price_per_category": {
|
||||
"aggs": {
|
||||
"min_price": { "min": { "field": "price" } }
|
||||
},
|
||||
"terms": {
|
||||
"field": "category",
|
||||
"min_doc_count": 1,
|
||||
"order": { "min_price": "desc" }
|
||||
}
|
||||
}
|
||||
} "#;
|
||||
|
||||
let agg_req: Aggregations = serde_json::from_str(agg_req_str)?;
|
||||
|
||||
let collector = AggregationCollector::from_aggs(agg_req, None);
|
||||
|
||||
let agg_res: AggregationResults = searcher.search(&AllQuery, &collector).unwrap();
|
||||
let res: Value = serde_json::to_value(agg_res)?;
|
||||
println!("{}", serde_json::to_string_pretty(&res)?);
|
||||
|
||||
// Minimum price per category, sorted by minimum price descending
|
||||
//
|
||||
// As you can see, the starting prices for `Formalwear` are higher than `Casualwear`.
|
||||
//
|
||||
let expected_res = r#"
|
||||
{
|
||||
"min_price_per_category": {
|
||||
"buckets": [
|
||||
{ "doc_count": 2, "key": "Womens Formalwear", "min_price": { "value": 270.0 } },
|
||||
{ "doc_count": 2, "key": "Mens Formalwear", "min_price": { "value": 75.0 } },
|
||||
{ "doc_count": 2, "key": "Mens Casualwear", "min_price": { "value": 49.99 } },
|
||||
{ "doc_count": 2, "key": "Womens Footwear", "min_price": { "value": 42.0 } },
|
||||
{ "doc_count": 2, "key": "Womens Casualwear", "min_price": { "value": 30.0 } },
|
||||
{ "doc_count": 3, "key": "Mens Footwear", "min_price": { "value": 19.0 } }
|
||||
],
|
||||
"sum_other_doc_count": 0
|
||||
}
|
||||
}
|
||||
"#;
|
||||
let expected_json: Value = serde_json::from_str(expected_res)?;
|
||||
|
||||
assert_eq!(expected_json, res);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@@ -105,7 +105,7 @@ impl SegmentCollector for StatsSegmentCollector {
|
||||
fn collect(&mut self, doc: u32, _score: Score) {
|
||||
// Since we know the values are single value, we could call `first_or_default_col` on the
|
||||
// column and fetch single values.
|
||||
for value in self.fast_field_reader.values(doc) {
|
||||
for value in self.fast_field_reader.values_for_doc(doc) {
|
||||
let value = value as f64;
|
||||
self.stats.count += 1;
|
||||
self.stats.sum += value;
|
||||
@@ -171,7 +171,7 @@ fn main() -> tantivy::Result<()> {
|
||||
let searcher = reader.searcher();
|
||||
let query_parser = QueryParser::for_index(&index, vec![product_name, product_description]);
|
||||
|
||||
// here we want to get a hit on the 'ken' in Frankenstein
|
||||
// here we want to search for `broom` and use `StatsCollector` on the hits.
|
||||
let query = query_parser.parse_query("broom")?;
|
||||
if let Some(stats) =
|
||||
searcher.search(&query, &StatsCollector::with_field("price".to_string()))?
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
// # Defining a tokenizer pipeline
|
||||
//
|
||||
// In this example, we'll see how to define a tokenizer pipeline
|
||||
// by aligning a bunch of `TokenFilter`.
|
||||
// In this example, we'll see how to define a tokenizer
|
||||
// by creating a custom `NgramTokenizer`.
|
||||
use tantivy::collector::TopDocs;
|
||||
use tantivy::query::QueryParser;
|
||||
use tantivy::schema::*;
|
||||
|
||||
@@ -14,6 +14,7 @@ fn main() -> tantivy::Result<()> {
|
||||
.set_stored()
|
||||
.set_fast()
|
||||
.set_precision(tantivy::DatePrecision::Seconds);
|
||||
// Add `occurred_at` date field type
|
||||
let occurred_at = schema_builder.add_date_field("occurred_at", opts);
|
||||
let event_type = schema_builder.add_text_field("event", STRING | STORED);
|
||||
let schema = schema_builder.build();
|
||||
@@ -22,6 +23,7 @@ fn main() -> tantivy::Result<()> {
|
||||
let index = Index::create_in_ram(schema.clone());
|
||||
|
||||
let mut index_writer = index.writer(50_000_000)?;
|
||||
// The dates are passed as string in the RFC3339 format
|
||||
let doc = schema.parse_document(
|
||||
r#"{
|
||||
"occurred_at": "2022-06-22T12:53:50.53Z",
|
||||
@@ -41,14 +43,16 @@ fn main() -> tantivy::Result<()> {
|
||||
let reader = index.reader()?;
|
||||
let searcher = reader.searcher();
|
||||
|
||||
// # Default fields: event_type
|
||||
// # Search
|
||||
let query_parser = QueryParser::for_index(&index, vec![event_type]);
|
||||
{
|
||||
let query = query_parser.parse_query("event:comment")?;
|
||||
// Simple exact search on the date
|
||||
let query = query_parser.parse_query("occurred_at:\"2022-06-22T12:53:50.53Z\"")?;
|
||||
let count_docs = searcher.search(&*query, &TopDocs::with_limit(5))?;
|
||||
assert_eq!(count_docs.len(), 1);
|
||||
}
|
||||
{
|
||||
// Range query on the date field
|
||||
let query = query_parser
|
||||
.parse_query(r#"occurred_at:[2022-06-22T12:58:00Z TO 2022-06-23T00:00:00Z}"#)?;
|
||||
let count_docs = searcher.search(&*query, &TopDocs::with_limit(4))?;
|
||||
|
||||
@@ -1,3 +1,12 @@
|
||||
// # Faceted Search With Tweak Score
|
||||
//
|
||||
// This example covers the faceted search functionalities of
|
||||
// tantivy.
|
||||
//
|
||||
// We will :
|
||||
// - define a text field "name" in our schema
|
||||
// - define a facet field "classification" in our schema
|
||||
|
||||
use std::collections::HashSet;
|
||||
|
||||
use tantivy::collector::TopDocs;
|
||||
@@ -55,6 +64,7 @@ fn main() -> tantivy::Result<()> {
|
||||
.collect(),
|
||||
);
|
||||
let top_docs_by_custom_score =
|
||||
// Call TopDocs with a custom tweak score
|
||||
TopDocs::with_limit(2).tweak_score(move |segment_reader: &SegmentReader| {
|
||||
let ingredient_reader = segment_reader.facet_reader("ingredient").unwrap();
|
||||
let facet_dict = ingredient_reader.facet_dict();
|
||||
@@ -65,6 +75,7 @@ fn main() -> tantivy::Result<()> {
|
||||
.collect();
|
||||
|
||||
move |doc: DocId, original_score: Score| {
|
||||
// Update the original score with a tweaked score
|
||||
let missing_ingredients = ingredient_reader
|
||||
.facet_ords(doc)
|
||||
.filter(|ord| !query_ords.contains(ord))
|
||||
|
||||
167
examples/fuzzy_search.rs
Normal file
167
examples/fuzzy_search.rs
Normal file
@@ -0,0 +1,167 @@
|
||||
// # Basic Example
|
||||
//
|
||||
// This example covers the basic functionalities of
|
||||
// tantivy.
|
||||
//
|
||||
// We will :
|
||||
// - define our schema
|
||||
// - create an index in a directory
|
||||
// - index a few documents into our index
|
||||
// - search for the best document matching a basic query
|
||||
// - retrieve the best document's original content.
|
||||
// ---
|
||||
// Importing tantivy...
|
||||
use tantivy::collector::{Count, TopDocs};
|
||||
use tantivy::query::FuzzyTermQuery;
|
||||
use tantivy::schema::*;
|
||||
use tantivy::{doc, Index, ReloadPolicy};
|
||||
use tempfile::TempDir;
|
||||
|
||||
fn main() -> tantivy::Result<()> {
|
||||
// Let's create a temporary directory for the
|
||||
// sake of this example
|
||||
let index_path = TempDir::new()?;
|
||||
|
||||
// # Defining the schema
|
||||
//
|
||||
// The Tantivy index requires a very strict schema.
|
||||
// The schema declares which fields are in the index,
|
||||
// and for each field, its type and "the way it should
|
||||
// be indexed".
|
||||
|
||||
// First we need to define a schema ...
|
||||
let mut schema_builder = Schema::builder();
|
||||
|
||||
// Our first field is title.
|
||||
// We want full-text search for it, and we also want
|
||||
// to be able to retrieve the document after the search.
|
||||
//
|
||||
// `TEXT | STORED` is some syntactic sugar to describe
|
||||
// that.
|
||||
//
|
||||
// `TEXT` means the field should be tokenized and indexed,
|
||||
// along with its term frequency and term positions.
|
||||
//
|
||||
// `STORED` means that the field will also be saved
|
||||
// in a compressed, row-oriented key-value store.
|
||||
// This store is useful for reconstructing the
|
||||
// documents that were selected during the search phase.
|
||||
let title = schema_builder.add_text_field("title", TEXT | STORED);
|
||||
|
||||
let schema = schema_builder.build();
|
||||
|
||||
// # Indexing documents
|
||||
//
|
||||
// Let's create a brand new index.
|
||||
//
|
||||
// This will actually just save a meta.json
|
||||
// with our schema in the directory.
|
||||
let index = Index::create_in_dir(&index_path, schema.clone())?;
|
||||
|
||||
// To insert a document we will need an index writer.
|
||||
// There must be only one writer at a time.
|
||||
// This single `IndexWriter` is already
|
||||
// multithreaded.
|
||||
//
|
||||
// Here we give tantivy a budget of `50MB`.
|
||||
// Using a bigger memory_arena for the indexer may increase
|
||||
// throughput, but 50 MB is already plenty.
|
||||
let mut index_writer = index.writer(50_000_000)?;
|
||||
|
||||
// Let's index our documents!
|
||||
// We first need a handle on the title and the body field.
|
||||
|
||||
// ### Adding documents
|
||||
//
|
||||
index_writer.add_document(doc!(
|
||||
title => "The Name of the Wind",
|
||||
))?;
|
||||
index_writer.add_document(doc!(
|
||||
title => "The Diary of Muadib",
|
||||
))?;
|
||||
index_writer.add_document(doc!(
|
||||
title => "A Dairy Cow",
|
||||
))?;
|
||||
index_writer.add_document(doc!(
|
||||
title => "The Diary of a Young Girl",
|
||||
))?;
|
||||
index_writer.commit()?;
|
||||
|
||||
// ### Committing
|
||||
//
|
||||
// At this point our documents are not searchable.
|
||||
//
|
||||
//
|
||||
// We need to call `.commit()` explicitly to force the
|
||||
// `index_writer` to finish processing the documents in the queue,
|
||||
// flush the current index to the disk, and advertise
|
||||
// the existence of new documents.
|
||||
//
|
||||
// This call is blocking.
|
||||
index_writer.commit()?;
|
||||
|
||||
// If `.commit()` returns correctly, then all of the
|
||||
// documents that have been added are guaranteed to be
|
||||
// persistently indexed.
|
||||
//
|
||||
// In the scenario of a crash or a power failure,
|
||||
// tantivy behaves as if it has rolled back to its last
|
||||
// commit.
|
||||
|
||||
// # Searching
|
||||
//
|
||||
// ### Searcher
|
||||
//
|
||||
// A reader is required first in order to search an index.
|
||||
// It acts as a `Searcher` pool that reloads itself,
|
||||
// depending on a `ReloadPolicy`.
|
||||
//
|
||||
// For a search server you will typically create one reader for the entire lifetime of your
|
||||
// program, and acquire a new searcher for every single request.
|
||||
//
|
||||
// In the code below, we rely on the 'ON_COMMIT' policy: the reader
|
||||
// will reload the index automatically after each commit.
|
||||
let reader = index
|
||||
.reader_builder()
|
||||
.reload_policy(ReloadPolicy::OnCommit)
|
||||
.try_into()?;
|
||||
|
||||
// We now need to acquire a searcher.
|
||||
//
|
||||
// A searcher points to a snapshotted, immutable version of the index.
|
||||
//
|
||||
// Some search experience might require more than
|
||||
// one query. Using the same searcher ensures that all of these queries will run on the
|
||||
// same version of the index.
|
||||
//
|
||||
// Acquiring a `searcher` is very cheap.
|
||||
//
|
||||
// You should acquire a searcher every time you start processing a request and
|
||||
// and release it right after your query is finished.
|
||||
let searcher = reader.searcher();
|
||||
|
||||
// ### FuzzyTermQuery
|
||||
{
|
||||
let term = Term::from_field_text(title, "Diary");
|
||||
let query = FuzzyTermQuery::new(term, 2, true);
|
||||
|
||||
let (top_docs, count) = searcher
|
||||
.search(&query, &(TopDocs::with_limit(5), Count))
|
||||
.unwrap();
|
||||
assert_eq!(count, 3);
|
||||
assert_eq!(top_docs.len(), 3);
|
||||
for (score, doc_address) in top_docs {
|
||||
let retrieved_doc = searcher.doc(doc_address)?;
|
||||
// Note that the score is not lower for the fuzzy hit.
|
||||
// There's an issue open for that: https://github.com/quickwit-oss/tantivy/issues/563
|
||||
println!("score {score:?} doc {}", schema.to_json(&retrieved_doc));
|
||||
// score 1.0 doc {"title":["The Diary of Muadib"]}
|
||||
//
|
||||
// score 1.0 doc {"title":["The Diary of a Young Girl"]}
|
||||
//
|
||||
// score 1.0 doc {"title":["A Dairy Cow"]}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
@@ -10,6 +10,10 @@ use tantivy::Index;
|
||||
|
||||
fn main() -> tantivy::Result<()> {
|
||||
// # Defining the schema
|
||||
// We set the IP field as `INDEXED`, so it can be searched
|
||||
// `FAST` will create a fast field. The fast field will be used to execute search queries.
|
||||
// `FAST` is not a requirement for range queries, it can also be executed on the inverted index
|
||||
// which is created by `INDEXED`.
|
||||
let mut schema_builder = Schema::builder();
|
||||
let event_type = schema_builder.add_text_field("event_type", STRING | STORED);
|
||||
let ip = schema_builder.add_ip_addr_field("ip", STORED | INDEXED | FAST);
|
||||
@@ -19,51 +23,81 @@ fn main() -> tantivy::Result<()> {
|
||||
let index = Index::create_in_ram(schema.clone());
|
||||
|
||||
let mut index_writer = index.writer(50_000_000)?;
|
||||
|
||||
// ### IPv4
|
||||
// Adding documents that contain an IPv4 address. Notice that the IP addresses are passed as
|
||||
// `String`. Since the field is of type ip, we parse the IP address from the string and store it
|
||||
// internally as IPv6.
|
||||
let doc = schema.parse_document(
|
||||
r#"{
|
||||
"ip": "192.168.0.33",
|
||||
"event_type": "login"
|
||||
}"#,
|
||||
"ip": "192.168.0.33",
|
||||
"event_type": "login"
|
||||
}"#,
|
||||
)?;
|
||||
index_writer.add_document(doc)?;
|
||||
let doc = schema.parse_document(
|
||||
r#"{
|
||||
"ip": "192.168.0.80",
|
||||
"event_type": "checkout"
|
||||
}"#,
|
||||
"ip": "192.168.0.80",
|
||||
"event_type": "checkout"
|
||||
}"#,
|
||||
)?;
|
||||
index_writer.add_document(doc)?;
|
||||
// ### IPv6
|
||||
// Adding a document that contains an IPv6 address.
|
||||
let doc = schema.parse_document(
|
||||
r#"{
|
||||
"ip": "2001:0db8:85a3:0000:0000:8a2e:0370:7334",
|
||||
"event_type": "checkout"
|
||||
}"#,
|
||||
"ip": "2001:0db8:85a3:0000:0000:8a2e:0370:7334",
|
||||
"event_type": "checkout"
|
||||
}"#,
|
||||
)?;
|
||||
|
||||
index_writer.add_document(doc)?;
|
||||
// Commit will create a segment containing our documents.
|
||||
index_writer.commit()?;
|
||||
|
||||
let reader = index.reader()?;
|
||||
let searcher = reader.searcher();
|
||||
|
||||
// # Search
|
||||
// Range queries on IPv4. Since we created a fast field, the fast field will be used to execute
|
||||
// the search.
|
||||
// ### Range Queries
|
||||
let query_parser = QueryParser::for_index(&index, vec![event_type, ip]);
|
||||
{
|
||||
let query = query_parser.parse_query("ip:[192.168.0.0 TO 192.168.0.100]")?;
|
||||
// Inclusive range queries
|
||||
let query = query_parser.parse_query("ip:[192.168.0.80 TO 192.168.0.100]")?;
|
||||
let count_docs = searcher.search(&*query, &TopDocs::with_limit(5))?;
|
||||
assert_eq!(count_docs.len(), 2);
|
||||
assert_eq!(count_docs.len(), 1);
|
||||
}
|
||||
{
|
||||
let query = query_parser.parse_query("ip:[192.168.1.0 TO 192.168.1.100]")?;
|
||||
// Exclusive range queries
|
||||
let query = query_parser.parse_query("ip:{192.168.0.80 TO 192.168.1.100]")?;
|
||||
let count_docs = searcher.search(&*query, &TopDocs::with_limit(2))?;
|
||||
assert_eq!(count_docs.len(), 0);
|
||||
}
|
||||
{
|
||||
// Find docs with IP addresses smaller equal 192.168.1.100
|
||||
let query = query_parser.parse_query("ip:[* TO 192.168.1.100]")?;
|
||||
let count_docs = searcher.search(&*query, &TopDocs::with_limit(2))?;
|
||||
assert_eq!(count_docs.len(), 2);
|
||||
}
|
||||
{
|
||||
// Find docs with IP addresses smaller than 192.168.1.100
|
||||
let query = query_parser.parse_query("ip:[* TO 192.168.1.100}")?;
|
||||
let count_docs = searcher.search(&*query, &TopDocs::with_limit(2))?;
|
||||
assert_eq!(count_docs.len(), 2);
|
||||
}
|
||||
|
||||
// ### Exact Queries
|
||||
// Exact search on IPv4.
|
||||
{
|
||||
let query = query_parser.parse_query("ip:192.168.0.80")?;
|
||||
let count_docs = searcher.search(&*query, &Count)?;
|
||||
assert_eq!(count_docs, 1);
|
||||
}
|
||||
// Exact search on IPv6.
|
||||
// IpV6 addresses need to be quoted because they contain `:`
|
||||
{
|
||||
// IpV6 needs to be escaped because it contains `:`
|
||||
let query = query_parser.parse_query("ip:\"2001:0db8:85a3:0000:0000:8a2e:0370:7334\"")?;
|
||||
let count_docs = searcher.search(&*query, &Count)?;
|
||||
assert_eq!(count_docs, 1);
|
||||
|
||||
@@ -17,7 +17,6 @@ use tantivy::{
|
||||
|
||||
type ProductId = u64;
|
||||
|
||||
/// Price
|
||||
type Price = u32;
|
||||
|
||||
pub trait PriceFetcher: Send + Sync + 'static {
|
||||
@@ -90,10 +89,10 @@ impl Warmer for DynamicPriceColumn {
|
||||
}
|
||||
}
|
||||
|
||||
/// For the sake of this example, the table is just an editable HashMap behind a RwLock.
|
||||
/// This map represents a map (ProductId -> Price)
|
||||
///
|
||||
/// In practise, it could be fetching things from an external service, like a SQL table.
|
||||
// For the sake of this example, the table is just an editable HashMap behind a RwLock.
|
||||
// This map represents a map (ProductId -> Price)
|
||||
//
|
||||
// In practise, it could be fetching things from an external service, like a SQL table.
|
||||
#[derive(Default, Clone)]
|
||||
pub struct ExternalPriceTable {
|
||||
prices: Arc<RwLock<HashMap<ProductId, Price>>>,
|
||||
|
||||
@@ -50,7 +50,7 @@ use std::collections::{HashMap, HashSet};
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
pub use super::bucket::RangeAggregation;
|
||||
use super::bucket::{HistogramAggregation, TermsAggregation};
|
||||
use super::bucket::{DateHistogramAggregationReq, HistogramAggregation, TermsAggregation};
|
||||
use super::metric::{
|
||||
AverageAggregation, CountAggregation, MaxAggregation, MinAggregation, StatsAggregation,
|
||||
SumAggregation,
|
||||
@@ -110,10 +110,13 @@ impl BucketAggregationInternal {
|
||||
_ => None,
|
||||
}
|
||||
}
|
||||
pub(crate) fn as_histogram(&self) -> Option<&HistogramAggregation> {
|
||||
pub(crate) fn as_histogram(&self) -> crate::Result<Option<HistogramAggregation>> {
|
||||
match &self.bucket_agg {
|
||||
BucketAggregationType::Histogram(histogram) => Some(histogram),
|
||||
_ => None,
|
||||
BucketAggregationType::Histogram(histogram) => Ok(Some(histogram.clone())),
|
||||
BucketAggregationType::DateHistogram(histogram) => {
|
||||
Ok(Some(histogram.to_histogram_req()?))
|
||||
}
|
||||
_ => Ok(None),
|
||||
}
|
||||
}
|
||||
pub(crate) fn as_term(&self) -> Option<&TermsAggregation> {
|
||||
@@ -124,15 +127,6 @@ impl BucketAggregationInternal {
|
||||
}
|
||||
}
|
||||
|
||||
/// Extract all fields, where the term directory is used in the tree.
|
||||
pub fn get_term_dict_field_names(aggs: &Aggregations) -> HashSet<String> {
|
||||
let mut term_dict_field_names = Default::default();
|
||||
for el in aggs.values() {
|
||||
el.get_term_dict_field_names(&mut term_dict_field_names)
|
||||
}
|
||||
term_dict_field_names
|
||||
}
|
||||
|
||||
/// Extract all fast field names used in the tree.
|
||||
pub fn get_fast_field_names(aggs: &Aggregations) -> HashSet<String> {
|
||||
let mut fast_field_names = Default::default();
|
||||
@@ -155,16 +149,12 @@ pub enum Aggregation {
|
||||
}
|
||||
|
||||
impl Aggregation {
|
||||
fn get_term_dict_field_names(&self, term_field_names: &mut HashSet<String>) {
|
||||
if let Aggregation::Bucket(bucket) = self {
|
||||
bucket.get_term_dict_field_names(term_field_names)
|
||||
}
|
||||
}
|
||||
|
||||
fn get_fast_field_names(&self, fast_field_names: &mut HashSet<String>) {
|
||||
match self {
|
||||
Aggregation::Bucket(bucket) => bucket.get_fast_field_names(fast_field_names),
|
||||
Aggregation::Metric(metric) => metric.get_fast_field_names(fast_field_names),
|
||||
Aggregation::Metric(metric) => {
|
||||
fast_field_names.insert(metric.get_fast_field_name().to_string());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -193,14 +183,9 @@ pub struct BucketAggregation {
|
||||
}
|
||||
|
||||
impl BucketAggregation {
|
||||
fn get_term_dict_field_names(&self, term_dict_field_names: &mut HashSet<String>) {
|
||||
if let BucketAggregationType::Terms(terms) = &self.bucket_agg {
|
||||
term_dict_field_names.insert(terms.field.to_string());
|
||||
}
|
||||
term_dict_field_names.extend(get_term_dict_field_names(&self.sub_aggregation));
|
||||
}
|
||||
fn get_fast_field_names(&self, fast_field_names: &mut HashSet<String>) {
|
||||
self.bucket_agg.get_fast_field_names(fast_field_names);
|
||||
let fast_field_name = self.bucket_agg.get_fast_field_name();
|
||||
fast_field_names.insert(fast_field_name.to_string());
|
||||
fast_field_names.extend(get_fast_field_names(&self.sub_aggregation));
|
||||
}
|
||||
}
|
||||
@@ -214,20 +199,22 @@ pub enum BucketAggregationType {
|
||||
/// Put data into buckets of user-defined ranges.
|
||||
#[serde(rename = "histogram")]
|
||||
Histogram(HistogramAggregation),
|
||||
/// Put data into buckets of user-defined ranges.
|
||||
#[serde(rename = "date_histogram")]
|
||||
DateHistogram(DateHistogramAggregationReq),
|
||||
/// Put data into buckets of terms.
|
||||
#[serde(rename = "terms")]
|
||||
Terms(TermsAggregation),
|
||||
}
|
||||
|
||||
impl BucketAggregationType {
|
||||
fn get_fast_field_names(&self, fast_field_names: &mut HashSet<String>) {
|
||||
fn get_fast_field_name(&self) -> &str {
|
||||
match self {
|
||||
BucketAggregationType::Terms(terms) => fast_field_names.insert(terms.field.to_string()),
|
||||
BucketAggregationType::Range(range) => fast_field_names.insert(range.field.to_string()),
|
||||
BucketAggregationType::Histogram(histogram) => {
|
||||
fast_field_names.insert(histogram.field.to_string())
|
||||
}
|
||||
};
|
||||
BucketAggregationType::Terms(terms) => terms.field.as_str(),
|
||||
BucketAggregationType::Range(range) => range.field.as_str(),
|
||||
BucketAggregationType::Histogram(histogram) => histogram.field.as_str(),
|
||||
BucketAggregationType::DateHistogram(histogram) => histogram.field.as_str(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -262,16 +249,15 @@ pub enum MetricAggregation {
|
||||
}
|
||||
|
||||
impl MetricAggregation {
|
||||
fn get_fast_field_names(&self, fast_field_names: &mut HashSet<String>) {
|
||||
let fast_field_name = match self {
|
||||
fn get_fast_field_name(&self) -> &str {
|
||||
match self {
|
||||
MetricAggregation::Average(avg) => avg.field_name(),
|
||||
MetricAggregation::Count(count) => count.field_name(),
|
||||
MetricAggregation::Max(max) => max.field_name(),
|
||||
MetricAggregation::Min(min) => min.field_name(),
|
||||
MetricAggregation::Stats(stats) => stats.field_name(),
|
||||
MetricAggregation::Sum(sum) => sum.field_name(),
|
||||
};
|
||||
fast_field_names.insert(fast_field_name.to_string());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -3,17 +3,18 @@
|
||||
use std::rc::Rc;
|
||||
use std::sync::atomic::AtomicU32;
|
||||
|
||||
use columnar::{Column, StrColumn};
|
||||
use columnar::{Column, ColumnType, StrColumn};
|
||||
|
||||
use super::agg_req::{Aggregation, Aggregations, BucketAggregationType, MetricAggregation};
|
||||
use super::bucket::{HistogramAggregation, RangeAggregation, TermsAggregation};
|
||||
use super::bucket::{
|
||||
DateHistogramAggregationReq, HistogramAggregation, RangeAggregation, TermsAggregation,
|
||||
};
|
||||
use super::metric::{
|
||||
AverageAggregation, CountAggregation, MaxAggregation, MinAggregation, StatsAggregation,
|
||||
SumAggregation,
|
||||
};
|
||||
use super::segment_agg_result::BucketCount;
|
||||
use super::VecWithNames;
|
||||
use crate::schema::Type;
|
||||
use crate::{SegmentReader, TantivyError};
|
||||
|
||||
#[derive(Clone, Default)]
|
||||
@@ -41,7 +42,7 @@ pub struct BucketAggregationWithAccessor {
|
||||
/// based on search terms. So eventually this needs to be Option or moved.
|
||||
pub(crate) accessor: Column<u64>,
|
||||
pub(crate) str_dict_column: Option<StrColumn>,
|
||||
pub(crate) field_type: Type,
|
||||
pub(crate) field_type: ColumnType,
|
||||
pub(crate) bucket_agg: BucketAggregationType,
|
||||
pub(crate) sub_aggregation: AggregationsWithAccessor,
|
||||
pub(crate) bucket_count: BucketCount,
|
||||
@@ -63,6 +64,10 @@ impl BucketAggregationWithAccessor {
|
||||
BucketAggregationType::Histogram(HistogramAggregation {
|
||||
field: field_name, ..
|
||||
}) => get_ff_reader_and_validate(reader, field_name)?,
|
||||
BucketAggregationType::DateHistogram(DateHistogramAggregationReq {
|
||||
field: field_name,
|
||||
..
|
||||
}) => get_ff_reader_and_validate(reader, field_name)?,
|
||||
BucketAggregationType::Terms(TermsAggregation {
|
||||
field: field_name, ..
|
||||
}) => {
|
||||
@@ -94,7 +99,7 @@ impl BucketAggregationWithAccessor {
|
||||
#[derive(Clone)]
|
||||
pub struct MetricAggregationWithAccessor {
|
||||
pub metric: MetricAggregation,
|
||||
pub field_type: Type,
|
||||
pub field_type: ColumnType,
|
||||
pub accessor: Column<u64>,
|
||||
}
|
||||
|
||||
@@ -158,22 +163,12 @@ pub(crate) fn get_aggs_with_accessor_and_validate(
|
||||
fn get_ff_reader_and_validate(
|
||||
reader: &SegmentReader,
|
||||
field_name: &str,
|
||||
) -> crate::Result<(columnar::Column<u64>, Type)> {
|
||||
let field = reader.schema().get_field(field_name)?;
|
||||
// TODO we should get type metadata from columnar
|
||||
let field_type = reader
|
||||
.schema()
|
||||
.get_field_entry(field)
|
||||
.field_type()
|
||||
.value_type();
|
||||
// TODO Do validation
|
||||
|
||||
) -> crate::Result<(columnar::Column<u64>, ColumnType)> {
|
||||
let ff_fields = reader.fast_fields();
|
||||
let ff_field = ff_fields.u64_lenient(field_name)?.ok_or_else(|| {
|
||||
TantivyError::InvalidArgument(format!(
|
||||
"No numerical fast field found for field: {}",
|
||||
field_name
|
||||
))
|
||||
})?;
|
||||
Ok((ff_field, field_type))
|
||||
let ff_field_with_type = ff_fields
|
||||
.u64_lenient_with_type(field_name)?
|
||||
.ok_or_else(|| {
|
||||
TantivyError::InvalidArgument(format!("No fast field found for field: {}", field_name))
|
||||
})?;
|
||||
Ok(ff_field_with_type)
|
||||
}
|
||||
|
||||
@@ -12,7 +12,6 @@ use super::bucket::GetDocCount;
|
||||
use super::intermediate_agg_result::{IntermediateBucketResult, IntermediateMetricResult};
|
||||
use super::metric::{SingleMetricResult, Stats};
|
||||
use super::Key;
|
||||
use crate::schema::Schema;
|
||||
use crate::TantivyError;
|
||||
|
||||
#[derive(Clone, Default, Debug, PartialEq, Serialize, Deserialize)]
|
||||
@@ -154,12 +153,9 @@ pub enum BucketResult {
|
||||
}
|
||||
|
||||
impl BucketResult {
|
||||
pub(crate) fn empty_from_req(
|
||||
req: &BucketAggregationInternal,
|
||||
schema: &Schema,
|
||||
) -> crate::Result<Self> {
|
||||
pub(crate) fn empty_from_req(req: &BucketAggregationInternal) -> crate::Result<Self> {
|
||||
let empty_bucket = IntermediateBucketResult::empty_from_req(&req.bucket_agg);
|
||||
empty_bucket.into_final_bucket_result(req, schema)
|
||||
empty_bucket.into_final_bucket_result(req)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
1174
src/aggregation/agg_tests.rs
Normal file
1174
src/aggregation/agg_tests.rs
Normal file
File diff suppressed because it is too large
Load Diff
@@ -1,5 +1,8 @@
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
use super::{HistogramAggregation, HistogramBounds};
|
||||
use crate::aggregation::AggregationError;
|
||||
|
||||
/// DateHistogramAggregation is similar to `HistogramAggregation`, but it can only be used with date
|
||||
/// type.
|
||||
///
|
||||
@@ -29,8 +32,16 @@ use serde::{Deserialize, Serialize};
|
||||
/// See [`BucketEntry`](crate::aggregation::agg_result::BucketEntry)
|
||||
#[derive(Clone, Debug, Default, PartialEq, Serialize, Deserialize)]
|
||||
pub struct DateHistogramAggregationReq {
|
||||
#[doc(hidden)]
|
||||
/// Only for validation
|
||||
interval: Option<String>,
|
||||
#[doc(hidden)]
|
||||
/// Only for validation
|
||||
date_interval: Option<String>,
|
||||
/// The field to aggregate on.
|
||||
pub field: String,
|
||||
/// The format to format dates.
|
||||
pub format: Option<String>,
|
||||
/// The interval to chunk your data range. Each bucket spans a value range of
|
||||
/// [0..fixed_interval). Accepted values
|
||||
///
|
||||
@@ -55,29 +66,132 @@ pub struct DateHistogramAggregationReq {
|
||||
/// Intervals implicitly defines an absolute grid of buckets `[interval * k, interval * (k +
|
||||
/// 1))`.
|
||||
pub offset: Option<String>,
|
||||
/// The minimum number of documents in a bucket to be returned. Defaults to 0.
|
||||
pub min_doc_count: Option<u64>,
|
||||
/// Limits the data range to `[min, max]` closed interval.
|
||||
///
|
||||
/// This can be used to filter values if they are not in the data range.
|
||||
///
|
||||
/// hard_bounds only limits the buckets, to force a range set both extended_bounds and
|
||||
/// hard_bounds to the same range.
|
||||
///
|
||||
/// Needs to be provided as timestamp in microseconds precision.
|
||||
///
|
||||
/// ## Example
|
||||
/// ```json
|
||||
/// {
|
||||
/// "sales_over_time": {
|
||||
/// "date_histogram": {
|
||||
/// "field": "dates",
|
||||
/// "interval": "1d",
|
||||
/// "hard_bounds": {
|
||||
/// "min": 0,
|
||||
/// "max": 1420502400000000
|
||||
/// }
|
||||
/// }
|
||||
/// }
|
||||
/// }
|
||||
/// ```
|
||||
pub hard_bounds: Option<HistogramBounds>,
|
||||
/// Can be set to extend your bounds. The range of the buckets is by default defined by the
|
||||
/// data range of the values of the documents. As the name suggests, this can only be used to
|
||||
/// extend the value range. If the bounds for min or max are not extending the range, the value
|
||||
/// has no effect on the returned buckets.
|
||||
///
|
||||
/// Cannot be set in conjunction with min_doc_count > 0, since the empty buckets from extended
|
||||
/// bounds would not be returned.
|
||||
pub extended_bounds: Option<HistogramBounds>,
|
||||
|
||||
/// Whether to return the buckets as a hash map
|
||||
#[serde(default)]
|
||||
pub keyed: bool,
|
||||
}
|
||||
|
||||
impl DateHistogramAggregationReq {
|
||||
pub(crate) fn to_histogram_req(&self) -> crate::Result<HistogramAggregation> {
|
||||
self.validate()?;
|
||||
Ok(HistogramAggregation {
|
||||
field: self.field.to_string(),
|
||||
interval: parse_into_microseconds(&self.fixed_interval)? as f64,
|
||||
offset: self
|
||||
.offset
|
||||
.as_ref()
|
||||
.map(|offset| parse_offset_into_microseconds(offset))
|
||||
.transpose()?
|
||||
.map(|el| el as f64),
|
||||
min_doc_count: self.min_doc_count,
|
||||
hard_bounds: None,
|
||||
extended_bounds: None,
|
||||
keyed: self.keyed,
|
||||
})
|
||||
}
|
||||
|
||||
fn validate(&self) -> crate::Result<()> {
|
||||
if self.interval.is_some() {
|
||||
return Err(crate::TantivyError::InvalidArgument(format!(
|
||||
"`interval` parameter {:?} in date histogram is unsupported, only \
|
||||
`fixed_interval` is supported",
|
||||
self.interval
|
||||
)));
|
||||
}
|
||||
if self.format.is_some() {
|
||||
return Err(crate::TantivyError::InvalidArgument(
|
||||
"format parameter on date_histogram is unsupported".to_string(),
|
||||
));
|
||||
}
|
||||
|
||||
if self.date_interval.is_some() {
|
||||
return Err(crate::TantivyError::InvalidArgument(
|
||||
"date_interval in date histogram is unsupported, only `fixed_interval` is \
|
||||
supported"
|
||||
.to_string(),
|
||||
));
|
||||
}
|
||||
|
||||
parse_into_microseconds(&self.fixed_interval)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, PartialEq, Eq)]
|
||||
#[derive(Debug, Clone, PartialEq, Eq, Error)]
|
||||
/// Errors when parsing the fixed interval for `DateHistogramAggregationReq`.
|
||||
pub enum DateHistogramParseError {
|
||||
/// Unit not recognized in passed String
|
||||
#[error("Unit not recognized in passed String {0:?}")]
|
||||
UnitNotRecognized(String),
|
||||
/// Number not found in passed String
|
||||
#[error("Number not found in passed String {0:?}")]
|
||||
NumberMissing(String),
|
||||
/// Unit not found in passed String
|
||||
#[error("Unit not found in passed String {0:?}")]
|
||||
UnitMissing(String),
|
||||
/// Offset invalid
|
||||
#[error("passed offset is invalid {0:?}")]
|
||||
InvalidOffset(String),
|
||||
}
|
||||
|
||||
fn parse_into_milliseconds(input: &str) -> Result<u64, DateHistogramParseError> {
|
||||
fn parse_offset_into_microseconds(input: &str) -> Result<i64, AggregationError> {
|
||||
let is_sign = |byte| &[byte] == b"-" || &[byte] == b"+";
|
||||
if input.is_empty() {
|
||||
return Err(DateHistogramParseError::InvalidOffset(input.to_string()).into());
|
||||
}
|
||||
|
||||
let has_sign = is_sign(input.as_bytes()[0]);
|
||||
if has_sign {
|
||||
let (sign, input) = input.split_at(1);
|
||||
let val = parse_into_microseconds(input)?;
|
||||
if sign == "-" {
|
||||
Ok(-val)
|
||||
} else {
|
||||
Ok(val)
|
||||
}
|
||||
} else {
|
||||
parse_into_microseconds(input)
|
||||
}
|
||||
}
|
||||
|
||||
fn parse_into_microseconds(input: &str) -> Result<i64, AggregationError> {
|
||||
let split_boundary = input
|
||||
.as_bytes()
|
||||
.iter()
|
||||
@@ -85,12 +199,12 @@ fn parse_into_milliseconds(input: &str) -> Result<u64, DateHistogramParseError>
|
||||
.count();
|
||||
let (number, unit) = input.split_at(split_boundary);
|
||||
if number.is_empty() {
|
||||
return Err(DateHistogramParseError::NumberMissing(input.to_string()));
|
||||
return Err(DateHistogramParseError::NumberMissing(input.to_string()).into());
|
||||
}
|
||||
if unit.is_empty() {
|
||||
return Err(DateHistogramParseError::UnitMissing(input.to_string()));
|
||||
return Err(DateHistogramParseError::UnitMissing(input.to_string()).into());
|
||||
}
|
||||
let number: u64 = number
|
||||
let number: i64 = number
|
||||
.parse()
|
||||
// Technically this should never happen, but there was a bug
|
||||
// here and being defensive does not hurt.
|
||||
@@ -102,36 +216,260 @@ fn parse_into_milliseconds(input: &str) -> Result<u64, DateHistogramParseError>
|
||||
"m" => 60 * 1000,
|
||||
"h" => 60 * 60 * 1000,
|
||||
"d" => 24 * 60 * 60 * 1000,
|
||||
_ => return Err(DateHistogramParseError::UnitNotRecognized(unit.to_string())),
|
||||
_ => return Err(DateHistogramParseError::UnitNotRecognized(unit.to_string()).into()),
|
||||
};
|
||||
|
||||
Ok(number * multiplier_from_unit)
|
||||
Ok(number * multiplier_from_unit * 1000)
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use pretty_assertions::assert_eq;
|
||||
|
||||
use super::*;
|
||||
use crate::aggregation::agg_req::Aggregations;
|
||||
use crate::aggregation::tests::exec_request;
|
||||
use crate::indexer::NoMergePolicy;
|
||||
use crate::schema::{Schema, FAST};
|
||||
use crate::Index;
|
||||
|
||||
#[test]
|
||||
fn test_parse_into_milliseconds() {
|
||||
assert_eq!(parse_into_milliseconds("1m").unwrap(), 60_000);
|
||||
assert_eq!(parse_into_milliseconds("2m").unwrap(), 120_000);
|
||||
fn test_parse_into_microseconds() {
|
||||
assert_eq!(parse_into_microseconds("1m").unwrap(), 60_000_000);
|
||||
assert_eq!(parse_into_microseconds("2m").unwrap(), 120_000_000);
|
||||
assert_eq!(
|
||||
parse_into_milliseconds("2y").unwrap_err(),
|
||||
DateHistogramParseError::UnitNotRecognized("y".to_string())
|
||||
parse_into_microseconds("2y").unwrap_err(),
|
||||
DateHistogramParseError::UnitNotRecognized("y".to_string()).into()
|
||||
);
|
||||
assert_eq!(
|
||||
parse_into_milliseconds("2000").unwrap_err(),
|
||||
DateHistogramParseError::UnitMissing("2000".to_string())
|
||||
parse_into_microseconds("2000").unwrap_err(),
|
||||
DateHistogramParseError::UnitMissing("2000".to_string()).into()
|
||||
);
|
||||
assert_eq!(
|
||||
parse_into_milliseconds("ms").unwrap_err(),
|
||||
DateHistogramParseError::NumberMissing("ms".to_string())
|
||||
parse_into_microseconds("ms").unwrap_err(),
|
||||
DateHistogramParseError::NumberMissing("ms".to_string()).into()
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_parse_offset_into_microseconds() {
|
||||
assert_eq!(parse_offset_into_microseconds("1m").unwrap(), 60_000_000);
|
||||
assert_eq!(parse_offset_into_microseconds("+1m").unwrap(), 60_000_000);
|
||||
assert_eq!(parse_offset_into_microseconds("-1m").unwrap(), -60_000_000);
|
||||
assert_eq!(parse_offset_into_microseconds("2m").unwrap(), 120_000_000);
|
||||
assert_eq!(parse_offset_into_microseconds("+2m").unwrap(), 120_000_000);
|
||||
assert_eq!(parse_offset_into_microseconds("-2m").unwrap(), -120_000_000);
|
||||
assert_eq!(parse_offset_into_microseconds("-2ms").unwrap(), -2_000);
|
||||
assert_eq!(
|
||||
parse_offset_into_microseconds("2y").unwrap_err(),
|
||||
DateHistogramParseError::UnitNotRecognized("y".to_string()).into()
|
||||
);
|
||||
assert_eq!(
|
||||
parse_offset_into_microseconds("2000").unwrap_err(),
|
||||
DateHistogramParseError::UnitMissing("2000".to_string()).into()
|
||||
);
|
||||
assert_eq!(
|
||||
parse_offset_into_microseconds("ms").unwrap_err(),
|
||||
DateHistogramParseError::NumberMissing("ms".to_string()).into()
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_parse_into_milliseconds_do_not_accept_non_ascii() {
|
||||
assert!(parse_into_milliseconds("1m").is_err());
|
||||
assert!(parse_into_microseconds("1m").is_err());
|
||||
}
|
||||
|
||||
pub fn get_test_index_from_docs(
|
||||
merge_segments: bool,
|
||||
segment_and_docs: &[Vec<&str>],
|
||||
) -> crate::Result<Index> {
|
||||
let mut schema_builder = Schema::builder();
|
||||
schema_builder.add_date_field("date", FAST);
|
||||
schema_builder.add_text_field("text", FAST);
|
||||
let schema = schema_builder.build();
|
||||
let index = Index::create_in_ram(schema.clone());
|
||||
{
|
||||
let mut index_writer = index.writer_with_num_threads(1, 30_000_000)?;
|
||||
index_writer.set_merge_policy(Box::new(NoMergePolicy));
|
||||
for values in segment_and_docs {
|
||||
for doc_str in values {
|
||||
let doc = schema.parse_document(doc_str)?;
|
||||
index_writer.add_document(doc)?;
|
||||
}
|
||||
// writing the segment
|
||||
index_writer.commit()?;
|
||||
}
|
||||
}
|
||||
if merge_segments {
|
||||
let segment_ids = index
|
||||
.searchable_segment_ids()
|
||||
.expect("Searchable segments failed.");
|
||||
if segment_ids.len() > 1 {
|
||||
let mut index_writer = index.writer_for_tests()?;
|
||||
index_writer.merge(&segment_ids).wait()?;
|
||||
index_writer.wait_merging_threads()?;
|
||||
}
|
||||
}
|
||||
|
||||
Ok(index)
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn histogram_test_date_force_merge_segments() -> crate::Result<()> {
|
||||
histogram_test_date_merge_segments(true)
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn histogram_test_date() -> crate::Result<()> {
|
||||
histogram_test_date_merge_segments(false)
|
||||
}
|
||||
fn histogram_test_date_merge_segments(merge_segments: bool) -> crate::Result<()> {
|
||||
let docs = vec![
|
||||
vec![r#"{ "date": "2015-01-01T12:10:30Z", "text": "aaa" }"#],
|
||||
vec![r#"{ "date": "2015-01-01T11:11:30Z", "text": "bbb" }"#],
|
||||
vec![r#"{ "date": "2015-01-02T00:00:00Z", "text": "bbb" }"#],
|
||||
vec![r#"{ "date": "2015-01-06T00:00:00Z", "text": "ccc" }"#],
|
||||
];
|
||||
|
||||
let index = get_test_index_from_docs(merge_segments, &docs)?;
|
||||
// 30day + offset
|
||||
let elasticsearch_compatible_json = json!(
|
||||
{
|
||||
"sales_over_time": {
|
||||
"date_histogram": {
|
||||
"field": "date",
|
||||
"fixed_interval": "30d",
|
||||
"offset": "-4d"
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
let agg_req: Aggregations =
|
||||
serde_json::from_str(&serde_json::to_string(&elasticsearch_compatible_json).unwrap())
|
||||
.unwrap();
|
||||
let res = exec_request(agg_req, &index)?;
|
||||
let expected_res = json!({
|
||||
"sales_over_time" : {
|
||||
"buckets" : [
|
||||
{
|
||||
"key_as_string" : "2015-01-01T00:00:00Z",
|
||||
"key" : 1420070400000000.0,
|
||||
"doc_count" : 4
|
||||
}
|
||||
]
|
||||
}
|
||||
});
|
||||
assert_eq!(res, expected_res);
|
||||
|
||||
// 30day + offset + sub_agg
|
||||
let elasticsearch_compatible_json = json!(
|
||||
{
|
||||
"sales_over_time": {
|
||||
"date_histogram": {
|
||||
"field": "date",
|
||||
"fixed_interval": "30d",
|
||||
"offset": "-4d"
|
||||
},
|
||||
"aggs": {
|
||||
"texts": {
|
||||
"terms": {"field": "text"}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
let agg_req: Aggregations =
|
||||
serde_json::from_str(&serde_json::to_string(&elasticsearch_compatible_json).unwrap())
|
||||
.unwrap();
|
||||
let res = exec_request(agg_req, &index)?;
|
||||
println!("{}", serde_json::to_string_pretty(&res).unwrap());
|
||||
let expected_res = json!({
|
||||
"sales_over_time" : {
|
||||
"buckets" : [
|
||||
{
|
||||
"key_as_string" : "2015-01-01T00:00:00Z",
|
||||
"key" : 1420070400000000.0,
|
||||
"doc_count" : 4,
|
||||
"texts": {
|
||||
"buckets": [
|
||||
{
|
||||
"doc_count": 2,
|
||||
"key": "bbb"
|
||||
},
|
||||
{
|
||||
"doc_count": 1,
|
||||
"key": "ccc"
|
||||
},
|
||||
{
|
||||
"doc_count": 1,
|
||||
"key": "aaa"
|
||||
}
|
||||
],
|
||||
"doc_count_error_upper_bound": 0,
|
||||
"sum_other_doc_count": 0
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
});
|
||||
assert_eq!(res, expected_res);
|
||||
|
||||
// 1day
|
||||
let elasticsearch_compatible_json = json!(
|
||||
{
|
||||
"sales_over_time": {
|
||||
"date_histogram": {
|
||||
"field": "date",
|
||||
"fixed_interval": "1d"
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
let agg_req: Aggregations =
|
||||
serde_json::from_str(&serde_json::to_string(&elasticsearch_compatible_json).unwrap())
|
||||
.unwrap();
|
||||
let res = exec_request(agg_req, &index)?;
|
||||
let expected_res = json!( {
|
||||
"sales_over_time": {
|
||||
"buckets": [
|
||||
{
|
||||
"doc_count": 2,
|
||||
"key": 1420070400000000.0,
|
||||
"key_as_string": "2015-01-01T00:00:00Z"
|
||||
},
|
||||
{
|
||||
"doc_count": 1,
|
||||
"key": 1420156800000000.0,
|
||||
"key_as_string": "2015-01-02T00:00:00Z"
|
||||
},
|
||||
{
|
||||
"doc_count": 0,
|
||||
"key": 1420243200000000.0,
|
||||
"key_as_string": "2015-01-03T00:00:00Z"
|
||||
},
|
||||
{
|
||||
"doc_count": 0,
|
||||
"key": 1420329600000000.0,
|
||||
"key_as_string": "2015-01-04T00:00:00Z"
|
||||
},
|
||||
{
|
||||
"doc_count": 0,
|
||||
"key": 1420416000000000.0,
|
||||
"key_as_string": "2015-01-05T00:00:00Z"
|
||||
},
|
||||
{
|
||||
"doc_count": 1,
|
||||
"key": 1420502400000000.0,
|
||||
"key_as_string": "2015-01-06T00:00:00Z"
|
||||
}
|
||||
]
|
||||
}
|
||||
});
|
||||
assert_eq!(res, expected_res);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,9 +1,11 @@
|
||||
use std::cmp::Ordering;
|
||||
use std::fmt::Display;
|
||||
|
||||
use columnar::Column;
|
||||
use columnar::ColumnType;
|
||||
use itertools::Itertools;
|
||||
use rustc_hash::FxHashMap;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use tantivy_bitpacker::minmax;
|
||||
|
||||
use crate::aggregation::agg_req::AggregationsInternal;
|
||||
use crate::aggregation::agg_req_with_accessor::{
|
||||
@@ -14,10 +16,9 @@ use crate::aggregation::intermediate_agg_result::{
|
||||
IntermediateAggregationResults, IntermediateBucketResult, IntermediateHistogramBucketEntry,
|
||||
};
|
||||
use crate::aggregation::segment_agg_result::{
|
||||
GenericSegmentAggregationResultsCollector, SegmentAggregationCollector,
|
||||
build_segment_agg_collector, SegmentAggregationCollector,
|
||||
};
|
||||
use crate::aggregation::{f64_from_fastfield_u64, format_date};
|
||||
use crate::schema::{Schema, Type};
|
||||
use crate::aggregation::{f64_from_fastfield_u64, format_date, VecWithNames};
|
||||
use crate::{DocId, TantivyError};
|
||||
|
||||
/// Histogram is a bucket aggregation, where buckets are created dynamically for given `interval`.
|
||||
@@ -176,7 +177,7 @@ impl HistogramBounds {
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug, PartialEq)]
|
||||
#[derive(Default, Clone, Debug, PartialEq)]
|
||||
pub(crate) struct SegmentHistogramBucketEntry {
|
||||
pub key: f64,
|
||||
pub doc_count: u64,
|
||||
@@ -185,7 +186,7 @@ pub(crate) struct SegmentHistogramBucketEntry {
|
||||
impl SegmentHistogramBucketEntry {
|
||||
pub(crate) fn into_intermediate_bucket_entry(
|
||||
self,
|
||||
sub_aggregation: GenericSegmentAggregationResultsCollector,
|
||||
sub_aggregation: Box<dyn SegmentAggregationCollector>,
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<IntermediateHistogramBucketEntry> {
|
||||
Ok(IntermediateHistogramBucketEntry {
|
||||
@@ -202,14 +203,85 @@ impl SegmentHistogramBucketEntry {
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct SegmentHistogramCollector {
|
||||
/// The buckets containing the aggregation data.
|
||||
buckets: Vec<SegmentHistogramBucketEntry>,
|
||||
sub_aggregations: Option<Vec<GenericSegmentAggregationResultsCollector>>,
|
||||
field_type: Type,
|
||||
buckets: FxHashMap<i64, SegmentHistogramBucketEntry>,
|
||||
sub_aggregations: FxHashMap<i64, Box<dyn SegmentAggregationCollector>>,
|
||||
sub_aggregation_blueprint: Option<Box<dyn SegmentAggregationCollector>>,
|
||||
column_type: ColumnType,
|
||||
interval: f64,
|
||||
offset: f64,
|
||||
min_doc_count: u64,
|
||||
first_bucket_num: i64,
|
||||
bounds: HistogramBounds,
|
||||
accessor_idx: usize,
|
||||
}
|
||||
|
||||
impl SegmentAggregationCollector for SegmentHistogramCollector {
|
||||
fn into_intermediate_aggregations_result(
|
||||
self: Box<Self>,
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<IntermediateAggregationResults> {
|
||||
let name = agg_with_accessor.buckets.keys[self.accessor_idx].to_string();
|
||||
let agg_with_accessor = &agg_with_accessor.buckets.values[self.accessor_idx];
|
||||
|
||||
let bucket = self.into_intermediate_bucket_result(agg_with_accessor)?;
|
||||
let buckets = Some(VecWithNames::from_entries(vec![(name, bucket)]));
|
||||
|
||||
Ok(IntermediateAggregationResults {
|
||||
metrics: None,
|
||||
buckets,
|
||||
})
|
||||
}
|
||||
|
||||
fn collect(
|
||||
&mut self,
|
||||
doc: crate::DocId,
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<()> {
|
||||
self.collect_block(&[doc], agg_with_accessor)
|
||||
}
|
||||
|
||||
fn collect_block(
|
||||
&mut self,
|
||||
docs: &[crate::DocId],
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<()> {
|
||||
let accessor = &agg_with_accessor.buckets.values[self.accessor_idx].accessor;
|
||||
let sub_aggregation_accessor =
|
||||
&agg_with_accessor.buckets.values[self.accessor_idx].sub_aggregation;
|
||||
|
||||
let bounds = self.bounds;
|
||||
let interval = self.interval;
|
||||
let offset = self.offset;
|
||||
let get_bucket_pos = |val| (get_bucket_pos_f64(val, interval, offset) as i64);
|
||||
|
||||
for doc in docs {
|
||||
for val in accessor.values_for_doc(*doc) {
|
||||
let val = self.f64_from_fastfield_u64(val);
|
||||
|
||||
let bucket_pos = get_bucket_pos(val);
|
||||
|
||||
if bounds.contains(val) {
|
||||
self.increment_bucket(
|
||||
bucket_pos,
|
||||
*doc,
|
||||
sub_aggregation_accessor,
|
||||
interval,
|
||||
offset,
|
||||
)?;
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn flush(&mut self, agg_with_accessor: &AggregationsWithAccessor) -> crate::Result<()> {
|
||||
let sub_aggregation_accessor =
|
||||
&agg_with_accessor.buckets.values[self.accessor_idx].sub_aggregation;
|
||||
|
||||
for sub_aggregation in self.sub_aggregations.values_mut() {
|
||||
sub_aggregation.flush(sub_aggregation_accessor)?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl SegmentHistogramCollector {
|
||||
@@ -217,210 +289,96 @@ impl SegmentHistogramCollector {
|
||||
self,
|
||||
agg_with_accessor: &BucketAggregationWithAccessor,
|
||||
) -> crate::Result<IntermediateBucketResult> {
|
||||
// Compute the number of buckets to validate against max num buckets
|
||||
// Note: We use min_doc_count here, but it's only an lowerbound here, since were are on the
|
||||
// intermediate level and after merging the number of documents of a bucket could exceed
|
||||
// `min_doc_count`.
|
||||
{
|
||||
let cut_off_buckets_front = self
|
||||
.buckets
|
||||
.iter()
|
||||
.take_while(|bucket| bucket.doc_count <= self.min_doc_count)
|
||||
.count();
|
||||
let cut_off_buckets_back = self.buckets[cut_off_buckets_front..]
|
||||
.iter()
|
||||
.rev()
|
||||
.take_while(|bucket| bucket.doc_count <= self.min_doc_count)
|
||||
.count();
|
||||
let estimate_num_buckets =
|
||||
self.buckets.len() - cut_off_buckets_front - cut_off_buckets_back;
|
||||
let mut buckets = Vec::with_capacity(self.buckets.len());
|
||||
|
||||
agg_with_accessor
|
||||
.bucket_count
|
||||
.add_count(estimate_num_buckets as u32);
|
||||
agg_with_accessor.bucket_count.validate_bucket_count()?;
|
||||
}
|
||||
if self.sub_aggregation_blueprint.is_some() {
|
||||
for (bucket_pos, bucket) in self.buckets.into_iter() {
|
||||
let bucket_res = bucket.into_intermediate_bucket_entry(
|
||||
self.sub_aggregations.get(&bucket_pos).unwrap().clone(),
|
||||
&agg_with_accessor.sub_aggregation,
|
||||
);
|
||||
|
||||
let mut buckets = Vec::with_capacity(
|
||||
self.buckets
|
||||
.iter()
|
||||
.filter(|bucket| bucket.doc_count != 0)
|
||||
.count(),
|
||||
);
|
||||
|
||||
// Below we remove empty buckets for two reasons
|
||||
// 1. To reduce the size of the intermediate result, which may be passed on the wire.
|
||||
// 2. To mimic elasticsearch, there are no empty buckets at the start and end.
|
||||
//
|
||||
// Empty buckets may be added later again in the final result, depending on the request.
|
||||
if let Some(sub_aggregations) = self.sub_aggregations {
|
||||
for bucket_res in self
|
||||
.buckets
|
||||
.into_iter()
|
||||
.zip(sub_aggregations.into_iter())
|
||||
.filter(|(bucket, _sub_aggregation)| bucket.doc_count != 0)
|
||||
.map(|(bucket, sub_aggregation)| {
|
||||
bucket.into_intermediate_bucket_entry(
|
||||
sub_aggregation,
|
||||
&agg_with_accessor.sub_aggregation,
|
||||
)
|
||||
})
|
||||
{
|
||||
buckets.push(bucket_res?);
|
||||
}
|
||||
} else {
|
||||
buckets.extend(
|
||||
self.buckets
|
||||
.into_iter()
|
||||
.filter(|bucket| bucket.doc_count != 0)
|
||||
.map(|bucket| bucket.into()),
|
||||
);
|
||||
buckets.extend(self.buckets.into_values().map(|bucket| bucket.into()));
|
||||
};
|
||||
buckets.sort_unstable_by(|b1, b2| b1.key.partial_cmp(&b2.key).unwrap_or(Ordering::Equal));
|
||||
|
||||
Ok(IntermediateBucketResult::Histogram { buckets })
|
||||
Ok(IntermediateBucketResult::Histogram {
|
||||
buckets,
|
||||
column_type: Some(self.column_type),
|
||||
})
|
||||
}
|
||||
|
||||
pub(crate) fn from_req_and_validate(
|
||||
req: &HistogramAggregation,
|
||||
sub_aggregation: &AggregationsWithAccessor,
|
||||
field_type: Type,
|
||||
accessor: &Column<u64>,
|
||||
field_type: ColumnType,
|
||||
accessor_idx: usize,
|
||||
) -> crate::Result<Self> {
|
||||
req.validate()?;
|
||||
let min = f64_from_fastfield_u64(accessor.min_value(), &field_type);
|
||||
let max = f64_from_fastfield_u64(accessor.max_value(), &field_type);
|
||||
|
||||
let (min, max) = get_req_min_max(req, Some((min, max)));
|
||||
|
||||
// We compute and generate the buckets range (min, max) based on the request and the min
|
||||
// max in the fast field, but this is likely not ideal when this is a subbucket, where many
|
||||
// unnecessary buckets may be generated.
|
||||
let buckets = generate_buckets(req, min, max);
|
||||
|
||||
let sub_aggregations = if sub_aggregation.is_empty() {
|
||||
let sub_aggregation_blueprint = if sub_aggregation.is_empty() {
|
||||
None
|
||||
} else {
|
||||
let sub_aggregation =
|
||||
GenericSegmentAggregationResultsCollector::from_req_and_validate(sub_aggregation)?;
|
||||
Some(buckets.iter().map(|_| sub_aggregation.clone()).collect())
|
||||
let sub_aggregation = build_segment_agg_collector(sub_aggregation)?;
|
||||
Some(sub_aggregation)
|
||||
};
|
||||
|
||||
let buckets = buckets
|
||||
.iter()
|
||||
.map(|bucket| SegmentHistogramBucketEntry {
|
||||
key: *bucket,
|
||||
doc_count: 0,
|
||||
})
|
||||
.collect();
|
||||
|
||||
let first_bucket_num =
|
||||
get_bucket_num_f64(min, req.interval, req.offset.unwrap_or(0.0)) as i64;
|
||||
|
||||
let bounds = req.hard_bounds.unwrap_or(HistogramBounds {
|
||||
min: f64::MIN,
|
||||
max: f64::MAX,
|
||||
});
|
||||
|
||||
Ok(Self {
|
||||
buckets,
|
||||
field_type,
|
||||
buckets: Default::default(),
|
||||
column_type: field_type,
|
||||
interval: req.interval,
|
||||
offset: req.offset.unwrap_or(0.0),
|
||||
first_bucket_num,
|
||||
bounds,
|
||||
sub_aggregations,
|
||||
min_doc_count: req.min_doc_count(),
|
||||
sub_aggregations: Default::default(),
|
||||
sub_aggregation_blueprint,
|
||||
accessor_idx,
|
||||
})
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub(crate) fn collect_block(
|
||||
&mut self,
|
||||
docs: &[DocId],
|
||||
bucket_with_accessor: &BucketAggregationWithAccessor,
|
||||
force_flush: bool,
|
||||
) -> crate::Result<()> {
|
||||
let bounds = self.bounds;
|
||||
let interval = self.interval;
|
||||
let offset = self.offset;
|
||||
let first_bucket_num = self.first_bucket_num;
|
||||
let get_bucket_num =
|
||||
|val| (get_bucket_num_f64(val, interval, offset) as i64 - first_bucket_num) as usize;
|
||||
|
||||
let accessor = &bucket_with_accessor.accessor;
|
||||
for doc in docs {
|
||||
for val in accessor.values(*doc) {
|
||||
let val = self.f64_from_fastfield_u64(val);
|
||||
|
||||
let bucket_pos = get_bucket_num(val);
|
||||
self.increment_bucket_if_in_bounds(
|
||||
val,
|
||||
&bounds,
|
||||
bucket_pos,
|
||||
*doc,
|
||||
&bucket_with_accessor.sub_aggregation,
|
||||
)?;
|
||||
}
|
||||
}
|
||||
if force_flush {
|
||||
if let Some(sub_aggregations) = self.sub_aggregations.as_mut() {
|
||||
for sub_aggregation in sub_aggregations {
|
||||
sub_aggregation
|
||||
.flush_staged_docs(&bucket_with_accessor.sub_aggregation, force_flush)?;
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn increment_bucket_if_in_bounds(
|
||||
&mut self,
|
||||
val: f64,
|
||||
bounds: &HistogramBounds,
|
||||
bucket_pos: usize,
|
||||
doc: DocId,
|
||||
bucket_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<()> {
|
||||
if bounds.contains(val) {
|
||||
debug_assert_eq!(
|
||||
self.buckets[bucket_pos].key,
|
||||
get_bucket_val(val, self.interval, self.offset)
|
||||
);
|
||||
|
||||
self.increment_bucket(bucket_pos, doc, bucket_with_accessor)?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn increment_bucket(
|
||||
&mut self,
|
||||
bucket_pos: usize,
|
||||
bucket_pos: i64,
|
||||
doc: DocId,
|
||||
bucket_with_accessor: &AggregationsWithAccessor,
|
||||
interval: f64,
|
||||
offset: f64,
|
||||
) -> crate::Result<()> {
|
||||
let bucket = &mut self.buckets[bucket_pos];
|
||||
let bucket = self.buckets.entry(bucket_pos).or_insert_with(|| {
|
||||
let key = get_bucket_key_from_pos(bucket_pos as f64, interval, offset);
|
||||
SegmentHistogramBucketEntry { key, doc_count: 0 }
|
||||
});
|
||||
bucket.doc_count += 1;
|
||||
if let Some(sub_aggregation) = self.sub_aggregations.as_mut() {
|
||||
sub_aggregation[bucket_pos].collect(doc, bucket_with_accessor)?;
|
||||
if let Some(sub_aggregation_blueprint) = self.sub_aggregation_blueprint.as_mut() {
|
||||
self.sub_aggregations
|
||||
.entry(bucket_pos)
|
||||
.or_insert_with(|| sub_aggregation_blueprint.clone())
|
||||
.collect(doc, bucket_with_accessor)?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn f64_from_fastfield_u64(&self, val: u64) -> f64 {
|
||||
f64_from_fastfield_u64(val, &self.field_type)
|
||||
f64_from_fastfield_u64(val, &self.column_type)
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn get_bucket_num_f64(val: f64, interval: f64, offset: f64) -> f64 {
|
||||
fn get_bucket_pos_f64(val: f64, interval: f64, offset: f64) -> f64 {
|
||||
((val - offset) / interval).floor()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn get_bucket_val(val: f64, interval: f64, offset: f64) -> f64 {
|
||||
let bucket_pos = get_bucket_num_f64(val, interval, offset);
|
||||
fn get_bucket_key_from_pos(bucket_pos: f64, interval: f64, offset: f64) -> f64 {
|
||||
bucket_pos * interval + offset
|
||||
}
|
||||
|
||||
@@ -429,19 +387,14 @@ fn intermediate_buckets_to_final_buckets_fill_gaps(
|
||||
buckets: Vec<IntermediateHistogramBucketEntry>,
|
||||
histogram_req: &HistogramAggregation,
|
||||
sub_aggregation: &AggregationsInternal,
|
||||
schema: &Schema,
|
||||
) -> crate::Result<Vec<BucketEntry>> {
|
||||
// Generate the full list of buckets without gaps.
|
||||
//
|
||||
// The bounds are the min max from the current buckets, optionally extended by
|
||||
// extended_bounds from the request
|
||||
let min_max = if buckets.is_empty() {
|
||||
None
|
||||
} else {
|
||||
let min = buckets[0].key;
|
||||
let max = buckets[buckets.len() - 1].key;
|
||||
Some((min, max))
|
||||
};
|
||||
let min_max = minmax(buckets.iter().map(|bucket| bucket.key));
|
||||
|
||||
// TODO add memory check
|
||||
let fill_gaps_buckets = generate_buckets_with_opt_minmax(histogram_req, min_max);
|
||||
|
||||
let empty_sub_aggregation = IntermediateAggregationResults::empty_from_req(sub_aggregation);
|
||||
@@ -470,43 +423,33 @@ fn intermediate_buckets_to_final_buckets_fill_gaps(
|
||||
sub_aggregation: empty_sub_aggregation.clone(),
|
||||
},
|
||||
})
|
||||
.map(|intermediate_bucket| {
|
||||
intermediate_bucket.into_final_bucket_entry(sub_aggregation, schema)
|
||||
})
|
||||
.map(|intermediate_bucket| intermediate_bucket.into_final_bucket_entry(sub_aggregation))
|
||||
.collect::<crate::Result<Vec<_>>>()
|
||||
}
|
||||
|
||||
// Convert to BucketEntry
|
||||
pub(crate) fn intermediate_histogram_buckets_to_final_buckets(
|
||||
buckets: Vec<IntermediateHistogramBucketEntry>,
|
||||
column_type: Option<ColumnType>,
|
||||
histogram_req: &HistogramAggregation,
|
||||
sub_aggregation: &AggregationsInternal,
|
||||
schema: &Schema,
|
||||
) -> crate::Result<Vec<BucketEntry>> {
|
||||
let mut buckets = if histogram_req.min_doc_count() == 0 {
|
||||
// With min_doc_count != 0, we may need to add buckets, so that there are no
|
||||
// gaps, since intermediate result does not contain empty buckets (filtered to
|
||||
// reduce serialization size).
|
||||
|
||||
intermediate_buckets_to_final_buckets_fill_gaps(
|
||||
buckets,
|
||||
histogram_req,
|
||||
sub_aggregation,
|
||||
schema,
|
||||
)?
|
||||
intermediate_buckets_to_final_buckets_fill_gaps(buckets, histogram_req, sub_aggregation)?
|
||||
} else {
|
||||
buckets
|
||||
.into_iter()
|
||||
.filter(|histogram_bucket| histogram_bucket.doc_count >= histogram_req.min_doc_count())
|
||||
.map(|histogram_bucket| {
|
||||
histogram_bucket.into_final_bucket_entry(sub_aggregation, schema)
|
||||
})
|
||||
.map(|histogram_bucket| histogram_bucket.into_final_bucket_entry(sub_aggregation))
|
||||
.collect::<crate::Result<Vec<_>>>()?
|
||||
};
|
||||
|
||||
// If we have a date type on the histogram buckets, we add the `key_as_string` field as rfc339
|
||||
let field = schema.get_field(&histogram_req.field)?;
|
||||
if schema.get_field_entry(field).field_type().is_date() {
|
||||
if column_type == Some(ColumnType::DateTime) {
|
||||
for bucket in buckets.iter_mut() {
|
||||
if let crate::aggregation::Key::F64(val) = bucket.key {
|
||||
let key_as_string = format_date(val as i64)?;
|
||||
@@ -537,12 +480,6 @@ fn get_req_min_max(req: &HistogramAggregation, min_max: Option<(f64, f64)>) -> (
|
||||
(min, max)
|
||||
}
|
||||
|
||||
/// Generates buckets with req.interval
|
||||
/// range is computed for provided min_max and request extended_bounds/hard_bounds
|
||||
pub(crate) fn generate_buckets(req: &HistogramAggregation, min: f64, max: f64) -> Vec<f64> {
|
||||
generate_buckets_with_opt_minmax(req, Some((min, max)))
|
||||
}
|
||||
|
||||
/// Generates buckets with req.interval
|
||||
/// Range is computed for provided min_max and request extended_bounds/hard_bounds
|
||||
/// returns empty vec when there is no range to span
|
||||
@@ -553,8 +490,8 @@ pub(crate) fn generate_buckets_with_opt_minmax(
|
||||
let (min, max) = get_req_min_max(req, min_max);
|
||||
|
||||
let offset = req.offset.unwrap_or(0.0);
|
||||
let first_bucket_num = get_bucket_num_f64(min, req.interval, offset) as i64;
|
||||
let last_bucket_num = get_bucket_num_f64(max, req.interval, offset) as i64;
|
||||
let first_bucket_num = get_bucket_pos_f64(min, req.interval, offset) as i64;
|
||||
let last_bucket_num = get_bucket_pos_f64(max, req.interval, offset) as i64;
|
||||
let mut buckets = Vec::with_capacity((first_bucket_num..=last_bucket_num).count());
|
||||
for bucket_pos in first_bucket_num..=last_bucket_num {
|
||||
let bucket_key = bucket_pos as f64 * req.interval + offset;
|
||||
@@ -564,118 +501,6 @@ pub(crate) fn generate_buckets_with_opt_minmax(
|
||||
buckets
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn generate_buckets_test() {
|
||||
let histogram_req = HistogramAggregation {
|
||||
field: "dummy".to_string(),
|
||||
interval: 2.0,
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let buckets = generate_buckets(&histogram_req, 0.0, 10.0);
|
||||
assert_eq!(buckets, vec![0.0, 2.0, 4.0, 6.0, 8.0, 10.0]);
|
||||
|
||||
let buckets = generate_buckets(&histogram_req, 2.5, 5.5);
|
||||
assert_eq!(buckets, vec![2.0, 4.0]);
|
||||
|
||||
// Single bucket
|
||||
let buckets = generate_buckets(&histogram_req, 0.5, 0.75);
|
||||
assert_eq!(buckets, vec![0.0]);
|
||||
|
||||
// With offset
|
||||
let histogram_req = HistogramAggregation {
|
||||
field: "dummy".to_string(),
|
||||
interval: 2.0,
|
||||
offset: Some(0.5),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let buckets = generate_buckets(&histogram_req, 0.0, 10.0);
|
||||
assert_eq!(buckets, vec![-1.5, 0.5, 2.5, 4.5, 6.5, 8.5]);
|
||||
|
||||
let buckets = generate_buckets(&histogram_req, 2.5, 5.5);
|
||||
assert_eq!(buckets, vec![2.5, 4.5]);
|
||||
|
||||
// Single bucket
|
||||
let buckets = generate_buckets(&histogram_req, 0.5, 0.75);
|
||||
assert_eq!(buckets, vec![0.5]);
|
||||
|
||||
// no bucket
|
||||
let buckets = generate_buckets(&histogram_req, f64::MAX, f64::MIN);
|
||||
assert_eq!(buckets, vec![] as Vec<f64>);
|
||||
|
||||
// With extended_bounds
|
||||
let histogram_req = HistogramAggregation {
|
||||
field: "dummy".to_string(),
|
||||
interval: 2.0,
|
||||
extended_bounds: Some(HistogramBounds {
|
||||
min: 0.0,
|
||||
max: 10.0,
|
||||
}),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let buckets = generate_buckets(&histogram_req, 0.0, 10.0);
|
||||
assert_eq!(buckets, vec![0.0, 2.0, 4.0, 6.0, 8.0, 10.0]);
|
||||
|
||||
let buckets = generate_buckets(&histogram_req, 2.5, 5.5);
|
||||
assert_eq!(buckets, vec![0.0, 2.0, 4.0, 6.0, 8.0, 10.0]);
|
||||
|
||||
// Single bucket, but extended_bounds
|
||||
let buckets = generate_buckets(&histogram_req, 0.5, 0.75);
|
||||
assert_eq!(buckets, vec![0.0, 2.0, 4.0, 6.0, 8.0, 10.0]);
|
||||
|
||||
// no bucket, but extended_bounds
|
||||
let buckets = generate_buckets(&histogram_req, f64::MAX, f64::MIN);
|
||||
assert_eq!(buckets, vec![0.0, 2.0, 4.0, 6.0, 8.0, 10.0]);
|
||||
|
||||
// With invalid extended_bounds
|
||||
let histogram_req = HistogramAggregation {
|
||||
field: "dummy".to_string(),
|
||||
interval: 2.0,
|
||||
extended_bounds: Some(HistogramBounds { min: 3.0, max: 5.0 }),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let buckets = generate_buckets(&histogram_req, 0.0, 10.0);
|
||||
assert_eq!(buckets, vec![0.0, 2.0, 4.0, 6.0, 8.0, 10.0]);
|
||||
|
||||
// With hard_bounds reducing
|
||||
let histogram_req = HistogramAggregation {
|
||||
field: "dummy".to_string(),
|
||||
interval: 2.0,
|
||||
hard_bounds: Some(HistogramBounds { min: 3.0, max: 5.0 }),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let buckets = generate_buckets(&histogram_req, 0.0, 10.0);
|
||||
assert_eq!(buckets, vec![2.0, 4.0]);
|
||||
|
||||
// With hard_bounds, extending has no effect
|
||||
let histogram_req = HistogramAggregation {
|
||||
field: "dummy".to_string(),
|
||||
interval: 2.0,
|
||||
hard_bounds: Some(HistogramBounds {
|
||||
min: 0.0,
|
||||
max: 10.0,
|
||||
}),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let buckets = generate_buckets(&histogram_req, 2.5, 5.5);
|
||||
assert_eq!(buckets, vec![2.0, 4.0]);
|
||||
|
||||
// Blubber
|
||||
let histogram_req = HistogramAggregation {
|
||||
field: "dummy".to_string(),
|
||||
interval: 2.0,
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let buckets = generate_buckets(&histogram_req, 4.0, 10.0);
|
||||
assert_eq!(buckets, vec![4.0, 6.0, 8.0, 10.0]);
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
|
||||
@@ -1496,36 +1321,4 @@ mod tests {
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn histogram_test_max_buckets_segments() -> crate::Result<()> {
|
||||
let values = vec![0.0, 70000.0];
|
||||
|
||||
let index = get_test_index_from_values(true, &values)?;
|
||||
|
||||
let agg_req: Aggregations = vec![(
|
||||
"my_interval".to_string(),
|
||||
Aggregation::Bucket(BucketAggregation {
|
||||
bucket_agg: BucketAggregationType::Histogram(HistogramAggregation {
|
||||
field: "score_f64".to_string(),
|
||||
interval: 1.0,
|
||||
..Default::default()
|
||||
}),
|
||||
sub_aggregation: Default::default(),
|
||||
}),
|
||||
)]
|
||||
.into_iter()
|
||||
.collect();
|
||||
|
||||
let res = exec_request(agg_req, &index);
|
||||
|
||||
assert_eq!(
|
||||
res.unwrap_err().to_string(),
|
||||
"An invalid argument was passed: 'Aborting aggregation because too many buckets were \
|
||||
created'"
|
||||
.to_string()
|
||||
);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
// mod date_histogram;
|
||||
mod date_histogram;
|
||||
mod histogram;
|
||||
// pub use date_histogram::*;
|
||||
pub use date_histogram::*;
|
||||
pub use histogram::*;
|
||||
|
||||
@@ -21,28 +21,25 @@ use serde::{de, Deserialize, Deserializer, Serialize, Serializer};
|
||||
pub use term_agg::*;
|
||||
|
||||
/// Order for buckets in a bucket aggregation.
|
||||
#[derive(Clone, Copy, Debug, PartialEq, Serialize, Deserialize)]
|
||||
#[derive(Clone, Copy, Debug, PartialEq, Serialize, Deserialize, Default)]
|
||||
pub enum Order {
|
||||
/// Asc order
|
||||
#[serde(rename = "asc")]
|
||||
Asc,
|
||||
/// Desc order
|
||||
#[serde(rename = "desc")]
|
||||
#[default]
|
||||
Desc,
|
||||
}
|
||||
|
||||
impl Default for Order {
|
||||
fn default() -> Self {
|
||||
Order::Desc
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug, PartialEq)]
|
||||
/// Order property by which to apply the order
|
||||
#[derive(Default)]
|
||||
pub enum OrderTarget {
|
||||
/// The key of the bucket
|
||||
Key,
|
||||
/// The doc count of the bucket
|
||||
#[default]
|
||||
Count,
|
||||
/// Order by value of the sub aggregation metric with identified by given `String`.
|
||||
///
|
||||
@@ -50,11 +47,6 @@ pub enum OrderTarget {
|
||||
SubAggregation(String),
|
||||
}
|
||||
|
||||
impl Default for OrderTarget {
|
||||
fn default() -> Self {
|
||||
OrderTarget::Count
|
||||
}
|
||||
}
|
||||
impl From<&str> for OrderTarget {
|
||||
fn from(val: &str) -> Self {
|
||||
match val {
|
||||
|
||||
@@ -1,24 +1,22 @@
|
||||
use std::fmt::Debug;
|
||||
use std::ops::Range;
|
||||
|
||||
use columnar::MonotonicallyMappableToU64;
|
||||
use columnar::{ColumnType, MonotonicallyMappableToU64};
|
||||
use rustc_hash::FxHashMap;
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
use crate::aggregation::agg_req_with_accessor::{
|
||||
AggregationsWithAccessor, BucketAggregationWithAccessor,
|
||||
};
|
||||
use crate::aggregation::agg_req_with_accessor::AggregationsWithAccessor;
|
||||
use crate::aggregation::intermediate_agg_result::{
|
||||
IntermediateBucketResult, IntermediateRangeBucketEntry, IntermediateRangeBucketResult,
|
||||
IntermediateAggregationResults, IntermediateBucketResult, IntermediateRangeBucketEntry,
|
||||
IntermediateRangeBucketResult,
|
||||
};
|
||||
use crate::aggregation::segment_agg_result::{
|
||||
BucketCount, GenericSegmentAggregationResultsCollector, SegmentAggregationCollector,
|
||||
build_segment_agg_collector, BucketCount, SegmentAggregationCollector,
|
||||
};
|
||||
use crate::aggregation::{
|
||||
f64_from_fastfield_u64, f64_to_fastfield_u64, format_date, Key, SerializedKey,
|
||||
f64_from_fastfield_u64, f64_to_fastfield_u64, format_date, Key, SerializedKey, VecWithNames,
|
||||
};
|
||||
use crate::schema::Type;
|
||||
use crate::{DocId, TantivyError};
|
||||
use crate::TantivyError;
|
||||
|
||||
/// Provide user-defined buckets to aggregate on.
|
||||
/// Two special buckets will automatically be created to cover the whole range of values.
|
||||
@@ -128,14 +126,15 @@ pub(crate) struct SegmentRangeAndBucketEntry {
|
||||
pub struct SegmentRangeCollector {
|
||||
/// The buckets containing the aggregation data.
|
||||
buckets: Vec<SegmentRangeAndBucketEntry>,
|
||||
field_type: Type,
|
||||
column_type: ColumnType,
|
||||
pub(crate) accessor_idx: usize,
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
pub(crate) struct SegmentRangeBucketEntry {
|
||||
pub key: Key,
|
||||
pub doc_count: u64,
|
||||
pub sub_aggregation: Option<GenericSegmentAggregationResultsCollector>,
|
||||
pub sub_aggregation: Option<Box<dyn SegmentAggregationCollector>>,
|
||||
/// The from range of the bucket. Equals `f64::MIN` when `None`.
|
||||
pub from: Option<f64>,
|
||||
/// The to range of the bucket. Equals `f64::MAX` when `None`. Open interval, `to` is not
|
||||
@@ -174,12 +173,14 @@ impl SegmentRangeBucketEntry {
|
||||
}
|
||||
}
|
||||
|
||||
impl SegmentRangeCollector {
|
||||
pub fn into_intermediate_bucket_result(
|
||||
self,
|
||||
agg_with_accessor: &BucketAggregationWithAccessor,
|
||||
) -> crate::Result<IntermediateBucketResult> {
|
||||
let field_type = self.field_type;
|
||||
impl SegmentAggregationCollector for SegmentRangeCollector {
|
||||
fn into_intermediate_aggregations_result(
|
||||
self: Box<Self>,
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<IntermediateAggregationResults> {
|
||||
let field_type = self.column_type;
|
||||
let name = agg_with_accessor.buckets.keys[self.accessor_idx].to_string();
|
||||
let sub_agg = &agg_with_accessor.buckets.values[self.accessor_idx].sub_aggregation;
|
||||
|
||||
let buckets: FxHashMap<SerializedKey, IntermediateRangeBucketEntry> = self
|
||||
.buckets
|
||||
@@ -189,21 +190,77 @@ impl SegmentRangeCollector {
|
||||
range_to_string(&range_bucket.range, &field_type)?,
|
||||
range_bucket
|
||||
.bucket
|
||||
.into_intermediate_bucket_entry(&agg_with_accessor.sub_aggregation)?,
|
||||
.into_intermediate_bucket_entry(sub_agg)?,
|
||||
))
|
||||
})
|
||||
.collect::<crate::Result<_>>()?;
|
||||
|
||||
Ok(IntermediateBucketResult::Range(
|
||||
IntermediateRangeBucketResult { buckets },
|
||||
))
|
||||
let bucket = IntermediateBucketResult::Range(IntermediateRangeBucketResult {
|
||||
buckets,
|
||||
column_type: Some(self.column_type),
|
||||
});
|
||||
|
||||
let buckets = Some(VecWithNames::from_entries(vec![(name, bucket)]));
|
||||
|
||||
Ok(IntermediateAggregationResults {
|
||||
metrics: None,
|
||||
buckets,
|
||||
})
|
||||
}
|
||||
|
||||
fn collect(
|
||||
&mut self,
|
||||
doc: crate::DocId,
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<()> {
|
||||
self.collect_block(&[doc], agg_with_accessor)
|
||||
}
|
||||
|
||||
fn collect_block(
|
||||
&mut self,
|
||||
docs: &[crate::DocId],
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<()> {
|
||||
let accessor = &agg_with_accessor.buckets.values[self.accessor_idx].accessor;
|
||||
let sub_aggregation_accessor =
|
||||
&agg_with_accessor.buckets.values[self.accessor_idx].sub_aggregation;
|
||||
for doc in docs {
|
||||
for val in accessor.values_for_doc(*doc) {
|
||||
let bucket_pos = self.get_bucket_pos(val);
|
||||
|
||||
let bucket = &mut self.buckets[bucket_pos];
|
||||
|
||||
bucket.bucket.doc_count += 1;
|
||||
if let Some(sub_aggregation) = &mut bucket.bucket.sub_aggregation {
|
||||
sub_aggregation.collect(*doc, sub_aggregation_accessor)?;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn flush(&mut self, agg_with_accessor: &AggregationsWithAccessor) -> crate::Result<()> {
|
||||
let sub_aggregation_accessor =
|
||||
&agg_with_accessor.buckets.values[self.accessor_idx].sub_aggregation;
|
||||
|
||||
for bucket in self.buckets.iter_mut() {
|
||||
if let Some(sub_agg) = bucket.bucket.sub_aggregation.as_mut() {
|
||||
sub_agg.flush(sub_aggregation_accessor)?;
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl SegmentRangeCollector {
|
||||
pub(crate) fn from_req_and_validate(
|
||||
req: &RangeAggregation,
|
||||
sub_aggregation: &AggregationsWithAccessor,
|
||||
bucket_count: &BucketCount,
|
||||
field_type: Type,
|
||||
field_type: ColumnType,
|
||||
accessor_idx: usize,
|
||||
) -> crate::Result<Self> {
|
||||
// The range input on the request is f64.
|
||||
// We need to convert to u64 ranges, because we read the values as u64.
|
||||
@@ -229,11 +286,7 @@ impl SegmentRangeCollector {
|
||||
let sub_aggregation = if sub_aggregation.is_empty() {
|
||||
None
|
||||
} else {
|
||||
Some(
|
||||
GenericSegmentAggregationResultsCollector::from_req_and_validate(
|
||||
sub_aggregation,
|
||||
)?,
|
||||
)
|
||||
Some(build_segment_agg_collector(sub_aggregation)?)
|
||||
};
|
||||
|
||||
Ok(SegmentRangeAndBucketEntry {
|
||||
@@ -254,52 +307,11 @@ impl SegmentRangeCollector {
|
||||
|
||||
Ok(SegmentRangeCollector {
|
||||
buckets,
|
||||
field_type,
|
||||
column_type: field_type,
|
||||
accessor_idx,
|
||||
})
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub(crate) fn collect_block(
|
||||
&mut self,
|
||||
docs: &[DocId],
|
||||
bucket_with_accessor: &BucketAggregationWithAccessor,
|
||||
force_flush: bool,
|
||||
) -> crate::Result<()> {
|
||||
let accessor = &bucket_with_accessor.accessor;
|
||||
for doc in docs {
|
||||
for val in accessor.values(*doc) {
|
||||
let bucket_pos = self.get_bucket_pos(val);
|
||||
self.increment_bucket(bucket_pos, *doc, &bucket_with_accessor.sub_aggregation)?;
|
||||
}
|
||||
}
|
||||
|
||||
if force_flush {
|
||||
for bucket in &mut self.buckets {
|
||||
if let Some(sub_aggregation) = &mut bucket.bucket.sub_aggregation {
|
||||
sub_aggregation
|
||||
.flush_staged_docs(&bucket_with_accessor.sub_aggregation, force_flush)?;
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn increment_bucket(
|
||||
&mut self,
|
||||
bucket_pos: usize,
|
||||
doc: DocId,
|
||||
bucket_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<()> {
|
||||
let bucket = &mut self.buckets[bucket_pos];
|
||||
|
||||
bucket.bucket.doc_count += 1;
|
||||
if let Some(sub_aggregation) = &mut bucket.bucket.sub_aggregation {
|
||||
sub_aggregation.collect(doc, bucket_with_accessor)?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn get_bucket_pos(&self, val: u64) -> usize {
|
||||
let pos = self
|
||||
@@ -325,7 +337,7 @@ impl SegmentRangeCollector {
|
||||
/// more computational expensive when many documents are hit.
|
||||
fn to_u64_range(
|
||||
range: &RangeAggregationRange,
|
||||
field_type: &Type,
|
||||
field_type: &ColumnType,
|
||||
) -> crate::Result<InternalRangeAggregationRange> {
|
||||
let start = if let Some(from) = range.from {
|
||||
f64_to_fastfield_u64(from, field_type)
|
||||
@@ -351,7 +363,7 @@ fn to_u64_range(
|
||||
/// beginning and end and filling gaps.
|
||||
fn extend_validate_ranges(
|
||||
buckets: &[RangeAggregationRange],
|
||||
field_type: &Type,
|
||||
field_type: &ColumnType,
|
||||
) -> crate::Result<Vec<InternalRangeAggregationRange>> {
|
||||
let mut converted_buckets = buckets
|
||||
.iter()
|
||||
@@ -393,13 +405,16 @@ fn extend_validate_ranges(
|
||||
Ok(converted_buckets)
|
||||
}
|
||||
|
||||
pub(crate) fn range_to_string(range: &Range<u64>, field_type: &Type) -> crate::Result<String> {
|
||||
pub(crate) fn range_to_string(
|
||||
range: &Range<u64>,
|
||||
field_type: &ColumnType,
|
||||
) -> crate::Result<String> {
|
||||
// is_start is there for malformed requests, e.g. ig the user passes the range u64::MIN..0.0,
|
||||
// it should be rendered as "*-0" and not "*-*"
|
||||
let to_str = |val: u64, is_start: bool| {
|
||||
if (is_start && val == u64::MIN) || (!is_start && val == u64::MAX) {
|
||||
Ok("*".to_string())
|
||||
} else if *field_type == Type::Date {
|
||||
} else if *field_type == ColumnType::DateTime {
|
||||
let val = i64::from_u64(val);
|
||||
format_date(val)
|
||||
} else {
|
||||
@@ -414,7 +429,7 @@ pub(crate) fn range_to_string(range: &Range<u64>, field_type: &Type) -> crate::R
|
||||
))
|
||||
}
|
||||
|
||||
pub(crate) fn range_to_key(range: &Range<u64>, field_type: &Type) -> crate::Result<Key> {
|
||||
pub(crate) fn range_to_key(range: &Range<u64>, field_type: &ColumnType) -> crate::Result<Key> {
|
||||
Ok(Key::Str(range_to_string(range, field_type)?))
|
||||
}
|
||||
|
||||
@@ -426,8 +441,9 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::aggregation::agg_req::{
|
||||
Aggregation, Aggregations, BucketAggregation, BucketAggregationType,
|
||||
Aggregation, Aggregations, BucketAggregation, BucketAggregationType, MetricAggregation,
|
||||
};
|
||||
use crate::aggregation::metric::AverageAggregation;
|
||||
use crate::aggregation::tests::{
|
||||
exec_request, exec_request_with_query, get_test_index_2_segments,
|
||||
get_test_index_with_num_docs,
|
||||
@@ -435,7 +451,7 @@ mod tests {
|
||||
|
||||
pub fn get_collector_from_ranges(
|
||||
ranges: Vec<RangeAggregationRange>,
|
||||
field_type: Type,
|
||||
field_type: ColumnType,
|
||||
) -> SegmentRangeCollector {
|
||||
let req = RangeAggregation {
|
||||
field: "dummy".to_string(),
|
||||
@@ -448,6 +464,7 @@ mod tests {
|
||||
&Default::default(),
|
||||
&Default::default(),
|
||||
field_type,
|
||||
0,
|
||||
)
|
||||
.expect("unexpected error")
|
||||
}
|
||||
@@ -484,6 +501,47 @@ mod tests {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn range_fraction_test_with_sub_agg() -> crate::Result<()> {
|
||||
let index = get_test_index_with_num_docs(false, 100)?;
|
||||
|
||||
let sub_agg_req: Aggregations = vec![(
|
||||
"score_f64".to_string(),
|
||||
Aggregation::Metric(MetricAggregation::Average(
|
||||
AverageAggregation::from_field_name("score_f64".to_string()),
|
||||
)),
|
||||
)]
|
||||
.into_iter()
|
||||
.collect();
|
||||
|
||||
let agg_req: Aggregations = vec![(
|
||||
"range".to_string(),
|
||||
Aggregation::Bucket(BucketAggregation {
|
||||
bucket_agg: BucketAggregationType::Range(RangeAggregation {
|
||||
field: "fraction_f64".to_string(),
|
||||
ranges: vec![(0f64..0.1f64).into(), (0.1f64..0.2f64).into()],
|
||||
..Default::default()
|
||||
}),
|
||||
sub_aggregation: sub_agg_req,
|
||||
}),
|
||||
)]
|
||||
.into_iter()
|
||||
.collect();
|
||||
|
||||
let res = exec_request_with_query(agg_req, &index, None)?;
|
||||
|
||||
assert_eq!(res["range"]["buckets"][0]["key"], "*-0");
|
||||
assert_eq!(res["range"]["buckets"][0]["doc_count"], 0);
|
||||
assert_eq!(res["range"]["buckets"][1]["key"], "0-0.1");
|
||||
assert_eq!(res["range"]["buckets"][1]["doc_count"], 10);
|
||||
assert_eq!(res["range"]["buckets"][2]["key"], "0.1-0.2");
|
||||
assert_eq!(res["range"]["buckets"][2]["doc_count"], 10);
|
||||
assert_eq!(res["range"]["buckets"][3]["key"], "0.2-*");
|
||||
assert_eq!(res["range"]["buckets"][3]["doc_count"], 80);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn range_keyed_buckets_test() -> crate::Result<()> {
|
||||
let index = get_test_index_with_num_docs(false, 100)?;
|
||||
@@ -683,7 +741,7 @@ mod tests {
|
||||
#[test]
|
||||
fn bucket_test_extend_range_hole() {
|
||||
let buckets = vec![(10f64..20f64).into(), (30f64..40f64).into()];
|
||||
let collector = get_collector_from_ranges(buckets, Type::F64);
|
||||
let collector = get_collector_from_ranges(buckets, ColumnType::F64);
|
||||
|
||||
let buckets = collector.buckets;
|
||||
assert_eq!(buckets[0].range.start, u64::MIN);
|
||||
@@ -706,7 +764,7 @@ mod tests {
|
||||
(10f64..20f64).into(),
|
||||
(20f64..f64::MAX).into(),
|
||||
];
|
||||
let collector = get_collector_from_ranges(buckets, Type::F64);
|
||||
let collector = get_collector_from_ranges(buckets, ColumnType::F64);
|
||||
|
||||
let buckets = collector.buckets;
|
||||
assert_eq!(buckets[0].range.start, u64::MIN);
|
||||
@@ -721,7 +779,7 @@ mod tests {
|
||||
#[test]
|
||||
fn bucket_range_test_negative_vals() {
|
||||
let buckets = vec![(-10f64..-1f64).into()];
|
||||
let collector = get_collector_from_ranges(buckets, Type::F64);
|
||||
let collector = get_collector_from_ranges(buckets, ColumnType::F64);
|
||||
|
||||
let buckets = collector.buckets;
|
||||
assert_eq!(&buckets[0].bucket.key.to_string(), "*--10");
|
||||
@@ -730,7 +788,7 @@ mod tests {
|
||||
#[test]
|
||||
fn bucket_range_test_positive_vals() {
|
||||
let buckets = vec![(0f64..10f64).into()];
|
||||
let collector = get_collector_from_ranges(buckets, Type::F64);
|
||||
let collector = get_collector_from_ranges(buckets, ColumnType::F64);
|
||||
|
||||
let buckets = collector.buckets;
|
||||
assert_eq!(&buckets[0].bucket.key.to_string(), "*-0");
|
||||
@@ -740,7 +798,7 @@ mod tests {
|
||||
#[test]
|
||||
fn range_binary_search_test_u64() {
|
||||
let check_ranges = |ranges: Vec<RangeAggregationRange>| {
|
||||
let collector = get_collector_from_ranges(ranges, Type::U64);
|
||||
let collector = get_collector_from_ranges(ranges, ColumnType::U64);
|
||||
let search = |val: u64| collector.get_bucket_pos(val);
|
||||
|
||||
assert_eq!(search(u64::MIN), 0);
|
||||
@@ -786,7 +844,7 @@ mod tests {
|
||||
fn range_binary_search_test_f64() {
|
||||
let ranges = vec![(10.0..100.0).into()];
|
||||
|
||||
let collector = get_collector_from_ranges(ranges, Type::F64);
|
||||
let collector = get_collector_from_ranges(ranges, ColumnType::F64);
|
||||
let search = |val: u64| collector.get_bucket_pos(val);
|
||||
|
||||
assert_eq!(search(u64::MIN), 0);
|
||||
@@ -821,7 +879,7 @@ mod bench {
|
||||
buckets.push((bucket_start..bucket_start + bucket_size as f64).into())
|
||||
}
|
||||
|
||||
get_collector_from_ranges(buckets, Type::U64)
|
||||
get_collector_from_ranges(buckets, ColumnType::U64)
|
||||
}
|
||||
|
||||
fn get_rand_docs(total_docs: u64, num_docs_returned: u64) -> Vec<u64> {
|
||||
|
||||
@@ -1,5 +1,6 @@
|
||||
use std::fmt::Debug;
|
||||
|
||||
use columnar::{Cardinality, ColumnType};
|
||||
use rustc_hash::FxHashMap;
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
@@ -8,13 +9,15 @@ use crate::aggregation::agg_req_with_accessor::{
|
||||
AggregationsWithAccessor, BucketAggregationWithAccessor,
|
||||
};
|
||||
use crate::aggregation::intermediate_agg_result::{
|
||||
IntermediateBucketResult, IntermediateTermBucketEntry, IntermediateTermBucketResult,
|
||||
IntermediateAggregationResults, IntermediateBucketResult, IntermediateTermBucketEntry,
|
||||
IntermediateTermBucketResult,
|
||||
};
|
||||
use crate::aggregation::segment_agg_result::{
|
||||
build_segment_agg_collector, SegmentAggregationCollector,
|
||||
};
|
||||
use crate::aggregation::{f64_from_fastfield_u64, Key, VecWithNames};
|
||||
use crate::error::DataCorruption;
|
||||
use crate::{DocId, TantivyError};
|
||||
use crate::TantivyError;
|
||||
|
||||
/// Creates a bucket for every unique term and counts the number of occurences.
|
||||
/// Note that doc_count in the response buckets equals term count here.
|
||||
@@ -22,6 +25,10 @@ use crate::{DocId, TantivyError};
|
||||
/// If the text is untokenized and single value, that means one term per document and therefore it
|
||||
/// is in fact doc count.
|
||||
///
|
||||
/// ## Prerequisite
|
||||
/// Term aggregations work only on [fast fields](`crate::fastfield`) of type `u64`, `f64`, `i64` and
|
||||
/// text.
|
||||
///
|
||||
/// ### Terminology
|
||||
/// Shard parameters are supposed to be equivalent to elasticsearch shard parameter.
|
||||
/// Since they are
|
||||
@@ -196,9 +203,9 @@ impl TermsAggregationInternal {
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug, Default)]
|
||||
/// Container to store term_ids and their buckets.
|
||||
/// Container to store term_ids/or u64 values and their buckets.
|
||||
struct TermBuckets {
|
||||
pub(crate) entries: FxHashMap<u32, TermBucketEntry>,
|
||||
pub(crate) entries: FxHashMap<u64, TermBucketEntry>,
|
||||
}
|
||||
|
||||
#[derive(Clone, Default)]
|
||||
@@ -244,7 +251,7 @@ impl TermBuckets {
|
||||
fn force_flush(&mut self, agg_with_accessor: &AggregationsWithAccessor) -> crate::Result<()> {
|
||||
for entry in &mut self.entries.values_mut() {
|
||||
if let Some(sub_aggregations) = entry.sub_aggregations.as_mut() {
|
||||
sub_aggregations.flush_staged_docs(agg_with_accessor, false)?;
|
||||
sub_aggregations.flush(agg_with_accessor)?;
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
@@ -259,6 +266,8 @@ pub struct SegmentTermCollector {
|
||||
term_buckets: TermBuckets,
|
||||
req: TermsAggregationInternal,
|
||||
blueprint: Option<Box<dyn SegmentAggregationCollector>>,
|
||||
field_type: ColumnType,
|
||||
accessor_idx: usize,
|
||||
}
|
||||
|
||||
pub(crate) fn get_agg_name_and_property(name: &str) -> (&str, &str) {
|
||||
@@ -266,10 +275,86 @@ pub(crate) fn get_agg_name_and_property(name: &str) -> (&str, &str) {
|
||||
(agg_name, agg_property)
|
||||
}
|
||||
|
||||
impl SegmentAggregationCollector for SegmentTermCollector {
|
||||
fn into_intermediate_aggregations_result(
|
||||
self: Box<Self>,
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<IntermediateAggregationResults> {
|
||||
let name = agg_with_accessor.buckets.keys[self.accessor_idx].to_string();
|
||||
let agg_with_accessor = &agg_with_accessor.buckets.values[self.accessor_idx];
|
||||
|
||||
let bucket = self.into_intermediate_bucket_result(agg_with_accessor)?;
|
||||
let buckets = Some(VecWithNames::from_entries(vec![(name, bucket)]));
|
||||
|
||||
Ok(IntermediateAggregationResults {
|
||||
metrics: None,
|
||||
buckets,
|
||||
})
|
||||
}
|
||||
|
||||
fn collect(
|
||||
&mut self,
|
||||
doc: crate::DocId,
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<()> {
|
||||
self.collect_block(&[doc], agg_with_accessor)
|
||||
}
|
||||
|
||||
fn collect_block(
|
||||
&mut self,
|
||||
docs: &[crate::DocId],
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<()> {
|
||||
let accessor = &agg_with_accessor.buckets.values[self.accessor_idx].accessor;
|
||||
let sub_aggregation_accessor =
|
||||
&agg_with_accessor.buckets.values[self.accessor_idx].sub_aggregation;
|
||||
|
||||
if accessor.get_cardinality() == Cardinality::Full {
|
||||
for doc in docs {
|
||||
let term_id = accessor.values.get_val(*doc);
|
||||
let entry = self
|
||||
.term_buckets
|
||||
.entries
|
||||
.entry(term_id)
|
||||
.or_insert_with(|| TermBucketEntry::from_blueprint(&self.blueprint));
|
||||
entry.doc_count += 1;
|
||||
if let Some(sub_aggregations) = entry.sub_aggregations.as_mut() {
|
||||
sub_aggregations.collect(*doc, sub_aggregation_accessor)?;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for doc in docs {
|
||||
for term_id in accessor.values_for_doc(*doc) {
|
||||
let entry = self
|
||||
.term_buckets
|
||||
.entries
|
||||
.entry(term_id)
|
||||
.or_insert_with(|| TermBucketEntry::from_blueprint(&self.blueprint));
|
||||
entry.doc_count += 1;
|
||||
if let Some(sub_aggregations) = entry.sub_aggregations.as_mut() {
|
||||
sub_aggregations.collect(*doc, sub_aggregation_accessor)?;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn flush(&mut self, agg_with_accessor: &AggregationsWithAccessor) -> crate::Result<()> {
|
||||
let sub_aggregation_accessor =
|
||||
&agg_with_accessor.buckets.values[self.accessor_idx].sub_aggregation;
|
||||
|
||||
self.term_buckets.force_flush(sub_aggregation_accessor)?;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl SegmentTermCollector {
|
||||
pub(crate) fn from_req_and_validate(
|
||||
req: &TermsAggregation,
|
||||
sub_aggregations: &AggregationsWithAccessor,
|
||||
field_type: ColumnType,
|
||||
accessor_idx: usize,
|
||||
) -> crate::Result<Self> {
|
||||
let term_buckets = TermBuckets::default();
|
||||
|
||||
@@ -299,6 +384,8 @@ impl SegmentTermCollector {
|
||||
req: TermsAggregationInternal::from_req(req),
|
||||
term_buckets,
|
||||
blueprint,
|
||||
field_type,
|
||||
accessor_idx,
|
||||
})
|
||||
}
|
||||
|
||||
@@ -306,7 +393,7 @@ impl SegmentTermCollector {
|
||||
self,
|
||||
agg_with_accessor: &BucketAggregationWithAccessor,
|
||||
) -> crate::Result<IntermediateBucketResult> {
|
||||
let mut entries: Vec<(u32, TermBucketEntry)> =
|
||||
let mut entries: Vec<(u64, TermBucketEntry)> =
|
||||
self.term_buckets.entries.into_iter().collect();
|
||||
|
||||
let order_by_sub_aggregation =
|
||||
@@ -343,41 +430,52 @@ impl SegmentTermCollector {
|
||||
cut_off_buckets(&mut entries, self.req.segment_size as usize)
|
||||
};
|
||||
|
||||
let inverted_index = agg_with_accessor
|
||||
.str_dict_column
|
||||
.as_ref()
|
||||
.expect("internal error: inverted index not loaded for term aggregation");
|
||||
let term_dict = inverted_index;
|
||||
let mut dict: FxHashMap<Key, IntermediateTermBucketEntry> = Default::default();
|
||||
dict.reserve(entries.len());
|
||||
if self.field_type == ColumnType::Str {
|
||||
let term_dict = agg_with_accessor
|
||||
.str_dict_column
|
||||
.as_ref()
|
||||
.expect("internal error: term dictionary not found for term aggregation");
|
||||
|
||||
let mut dict: FxHashMap<String, IntermediateTermBucketEntry> = Default::default();
|
||||
let mut buffer = String::new();
|
||||
for (term_id, entry) in entries {
|
||||
if !term_dict.ord_to_str(term_id as u64, &mut buffer)? {
|
||||
return Err(TantivyError::InternalError(format!(
|
||||
"Couldn't find term_id {} in dict",
|
||||
term_id
|
||||
)));
|
||||
}
|
||||
dict.insert(
|
||||
buffer.to_string(),
|
||||
entry.into_intermediate_bucket_entry(&agg_with_accessor.sub_aggregation)?,
|
||||
);
|
||||
}
|
||||
if self.req.min_doc_count == 0 {
|
||||
// TODO: Handle rev streaming for descending sorting by keys
|
||||
let mut stream = term_dict.dictionary().stream()?;
|
||||
while let Some((key, _ord)) = stream.next() {
|
||||
if dict.len() >= self.req.segment_size as usize {
|
||||
break;
|
||||
let mut buffer = String::new();
|
||||
for (term_id, entry) in entries {
|
||||
if !term_dict.ord_to_str(term_id, &mut buffer)? {
|
||||
return Err(TantivyError::InternalError(format!(
|
||||
"Couldn't find term_id {} in dict",
|
||||
term_id
|
||||
)));
|
||||
}
|
||||
dict.insert(
|
||||
Key::Str(buffer.to_string()),
|
||||
entry.into_intermediate_bucket_entry(&agg_with_accessor.sub_aggregation)?,
|
||||
);
|
||||
}
|
||||
if self.req.min_doc_count == 0 {
|
||||
// TODO: Handle rev streaming for descending sorting by keys
|
||||
let mut stream = term_dict.dictionary().stream()?;
|
||||
while let Some((key, _ord)) = stream.next() {
|
||||
if dict.len() >= self.req.segment_size as usize {
|
||||
break;
|
||||
}
|
||||
|
||||
let key = std::str::from_utf8(key)
|
||||
.map_err(|utf8_err| DataCorruption::comment_only(utf8_err.to_string()))?;
|
||||
if !dict.contains_key(key) {
|
||||
dict.insert(key.to_owned(), Default::default());
|
||||
let key = Key::Str(
|
||||
std::str::from_utf8(key)
|
||||
.map_err(|utf8_err| DataCorruption::comment_only(utf8_err.to_string()))?
|
||||
.to_string(),
|
||||
);
|
||||
dict.entry(key).or_default();
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for (val, entry) in entries {
|
||||
let val = f64_from_fastfield_u64(val, &self.field_type);
|
||||
dict.insert(
|
||||
Key::F64(val),
|
||||
entry.into_intermediate_bucket_entry(&agg_with_accessor.sub_aggregation)?,
|
||||
);
|
||||
}
|
||||
};
|
||||
|
||||
Ok(IntermediateBucketResult::Terms(
|
||||
IntermediateTermBucketResult {
|
||||
@@ -387,36 +485,6 @@ impl SegmentTermCollector {
|
||||
},
|
||||
))
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub(crate) fn collect_block(
|
||||
&mut self,
|
||||
docs: &[DocId],
|
||||
bucket_with_accessor: &BucketAggregationWithAccessor,
|
||||
force_flush: bool,
|
||||
) -> crate::Result<()> {
|
||||
let accessor = &bucket_with_accessor.accessor;
|
||||
|
||||
for doc in docs {
|
||||
for term_id in accessor.values(*doc) {
|
||||
let entry = self
|
||||
.term_buckets
|
||||
.entries
|
||||
.entry(term_id as u32)
|
||||
.or_insert_with(|| TermBucketEntry::from_blueprint(&self.blueprint));
|
||||
entry.doc_count += 1;
|
||||
if let Some(sub_aggregations) = entry.sub_aggregations.as_mut() {
|
||||
sub_aggregations.collect(*doc, &bucket_with_accessor.sub_aggregation)?;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if force_flush {
|
||||
self.term_buckets
|
||||
.force_flush(&bucket_with_accessor.sub_aggregation)?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) trait GetDocCount {
|
||||
@@ -427,6 +495,11 @@ impl GetDocCount for (u32, TermBucketEntry) {
|
||||
self.1.doc_count
|
||||
}
|
||||
}
|
||||
impl GetDocCount for (u64, TermBucketEntry) {
|
||||
fn doc_count(&self) -> u64 {
|
||||
self.1.doc_count
|
||||
}
|
||||
}
|
||||
impl GetDocCount for (String, IntermediateTermBucketEntry) {
|
||||
fn doc_count(&self) -> u64 {
|
||||
self.1.doc_count
|
||||
@@ -455,8 +528,7 @@ pub(crate) fn cut_off_buckets<T: GetDocCount + Debug>(
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::aggregation::agg_req::{
|
||||
get_term_dict_field_names, Aggregation, Aggregations, BucketAggregation,
|
||||
BucketAggregationType, MetricAggregation,
|
||||
Aggregation, Aggregations, BucketAggregation, BucketAggregationType, MetricAggregation,
|
||||
};
|
||||
use crate::aggregation::metric::{AverageAggregation, StatsAggregation};
|
||||
use crate::aggregation::tests::{
|
||||
@@ -557,12 +629,6 @@ mod tests {
|
||||
serde_json::Value::Null
|
||||
);
|
||||
assert_eq!(res["my_texts"]["sum_other_doc_count"], 0); // TODO sum_other_doc_count with min_doc_count
|
||||
|
||||
assert_eq!(
|
||||
get_term_dict_field_names(&agg_req),
|
||||
vec!["string_id".to_string(),].into_iter().collect()
|
||||
);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@@ -577,7 +643,8 @@ mod tests {
|
||||
fn terms_aggregation_test_order_count_merge_segment(merge_segments: bool) -> crate::Result<()> {
|
||||
let segment_and_terms = vec![
|
||||
vec![(5.0, "terma".to_string())],
|
||||
vec![(4.0, "termb".to_string())],
|
||||
vec![(2.0, "termb".to_string())],
|
||||
vec![(2.0, "terma".to_string())],
|
||||
vec![(1.0, "termc".to_string())],
|
||||
vec![(1.0, "termc".to_string())],
|
||||
vec![(1.0, "termc".to_string())],
|
||||
@@ -618,7 +685,7 @@ mod tests {
|
||||
}),
|
||||
..Default::default()
|
||||
}),
|
||||
sub_aggregation: sub_agg,
|
||||
sub_aggregation: sub_agg.clone(),
|
||||
}),
|
||||
)]
|
||||
.into_iter()
|
||||
@@ -627,15 +694,114 @@ mod tests {
|
||||
let res = exec_request(agg_req, &index)?;
|
||||
assert_eq!(res["my_texts"]["buckets"][0]["key"], "termb");
|
||||
assert_eq!(res["my_texts"]["buckets"][0]["doc_count"], 2);
|
||||
assert_eq!(res["my_texts"]["buckets"][0]["avg_score"]["value"], 5.0);
|
||||
|
||||
assert_eq!(res["my_texts"]["buckets"][1]["key"], "termc");
|
||||
assert_eq!(res["my_texts"]["buckets"][1]["doc_count"], 3);
|
||||
assert_eq!(res["my_texts"]["buckets"][1]["avg_score"]["value"], 1.0);
|
||||
|
||||
assert_eq!(res["my_texts"]["buckets"][2]["key"], "terma");
|
||||
assert_eq!(res["my_texts"]["buckets"][2]["doc_count"], 5);
|
||||
assert_eq!(res["my_texts"]["buckets"][2]["doc_count"], 6);
|
||||
assert_eq!(res["my_texts"]["buckets"][2]["avg_score"]["value"], 4.5);
|
||||
|
||||
assert_eq!(res["my_texts"]["sum_other_doc_count"], 0);
|
||||
|
||||
// Agg on non string
|
||||
//
|
||||
let agg_req: Aggregations = vec![
|
||||
(
|
||||
"my_scores1".to_string(),
|
||||
Aggregation::Bucket(BucketAggregation {
|
||||
bucket_agg: BucketAggregationType::Terms(TermsAggregation {
|
||||
field: "score".to_string(),
|
||||
order: Some(CustomOrder {
|
||||
order: Order::Asc,
|
||||
target: OrderTarget::Count,
|
||||
}),
|
||||
..Default::default()
|
||||
}),
|
||||
sub_aggregation: sub_agg.clone(),
|
||||
}),
|
||||
),
|
||||
(
|
||||
"my_scores2".to_string(),
|
||||
Aggregation::Bucket(BucketAggregation {
|
||||
bucket_agg: BucketAggregationType::Terms(TermsAggregation {
|
||||
field: "score_f64".to_string(),
|
||||
order: Some(CustomOrder {
|
||||
order: Order::Asc,
|
||||
target: OrderTarget::Count,
|
||||
}),
|
||||
..Default::default()
|
||||
}),
|
||||
sub_aggregation: sub_agg.clone(),
|
||||
}),
|
||||
),
|
||||
(
|
||||
"my_scores3".to_string(),
|
||||
Aggregation::Bucket(BucketAggregation {
|
||||
bucket_agg: BucketAggregationType::Terms(TermsAggregation {
|
||||
field: "score_i64".to_string(),
|
||||
order: Some(CustomOrder {
|
||||
order: Order::Asc,
|
||||
target: OrderTarget::Count,
|
||||
}),
|
||||
..Default::default()
|
||||
}),
|
||||
sub_aggregation: sub_agg,
|
||||
}),
|
||||
),
|
||||
]
|
||||
.into_iter()
|
||||
.collect();
|
||||
|
||||
let res = exec_request(agg_req, &index)?;
|
||||
assert_eq!(res["my_scores1"]["buckets"][0]["key"], 8.0);
|
||||
assert_eq!(res["my_scores1"]["buckets"][0]["doc_count"], 1);
|
||||
assert_eq!(res["my_scores1"]["buckets"][0]["avg_score"]["value"], 8.0);
|
||||
|
||||
assert_eq!(res["my_scores1"]["buckets"][1]["key"], 2.0);
|
||||
assert_eq!(res["my_scores1"]["buckets"][1]["doc_count"], 2);
|
||||
assert_eq!(res["my_scores1"]["buckets"][1]["avg_score"]["value"], 2.0);
|
||||
|
||||
assert_eq!(res["my_scores1"]["buckets"][2]["key"], 1.0);
|
||||
assert_eq!(res["my_scores1"]["buckets"][2]["doc_count"], 3);
|
||||
assert_eq!(res["my_scores1"]["buckets"][2]["avg_score"]["value"], 1.0);
|
||||
|
||||
assert_eq!(res["my_scores1"]["buckets"][3]["key"], 5.0);
|
||||
assert_eq!(res["my_scores1"]["buckets"][3]["doc_count"], 5);
|
||||
assert_eq!(res["my_scores1"]["buckets"][3]["avg_score"]["value"], 5.0);
|
||||
|
||||
assert_eq!(res["my_scores1"]["sum_other_doc_count"], 0);
|
||||
|
||||
assert_eq!(res["my_scores2"]["buckets"][0]["key"], 8.0);
|
||||
assert_eq!(res["my_scores2"]["buckets"][0]["doc_count"], 1);
|
||||
assert_eq!(res["my_scores2"]["buckets"][0]["avg_score"]["value"], 8.0);
|
||||
|
||||
assert_eq!(res["my_scores2"]["buckets"][1]["key"], 2.0);
|
||||
assert_eq!(res["my_scores2"]["buckets"][1]["doc_count"], 2);
|
||||
assert_eq!(res["my_scores2"]["buckets"][1]["avg_score"]["value"], 2.0);
|
||||
|
||||
assert_eq!(res["my_scores2"]["buckets"][2]["key"], 1.0);
|
||||
assert_eq!(res["my_scores2"]["buckets"][2]["doc_count"], 3);
|
||||
assert_eq!(res["my_scores2"]["buckets"][2]["avg_score"]["value"], 1.0);
|
||||
|
||||
assert_eq!(res["my_scores2"]["sum_other_doc_count"], 0);
|
||||
|
||||
assert_eq!(res["my_scores3"]["buckets"][0]["key"], 8.0);
|
||||
assert_eq!(res["my_scores3"]["buckets"][0]["doc_count"], 1);
|
||||
assert_eq!(res["my_scores3"]["buckets"][0]["avg_score"]["value"], 8.0);
|
||||
|
||||
assert_eq!(res["my_scores3"]["buckets"][1]["key"], 2.0);
|
||||
assert_eq!(res["my_scores3"]["buckets"][1]["doc_count"], 2);
|
||||
assert_eq!(res["my_scores3"]["buckets"][1]["avg_score"]["value"], 2.0);
|
||||
|
||||
assert_eq!(res["my_scores3"]["buckets"][2]["key"], 1.0);
|
||||
assert_eq!(res["my_scores3"]["buckets"][2]["doc_count"], 3);
|
||||
assert_eq!(res["my_scores3"]["buckets"][2]["avg_score"]["value"], 1.0);
|
||||
|
||||
assert_eq!(res["my_scores3"]["sum_other_doc_count"], 0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
|
||||
79
src/aggregation/buf_collector.rs
Normal file
79
src/aggregation/buf_collector.rs
Normal file
@@ -0,0 +1,79 @@
|
||||
use super::agg_req_with_accessor::AggregationsWithAccessor;
|
||||
use super::intermediate_agg_result::IntermediateAggregationResults;
|
||||
use super::segment_agg_result::SegmentAggregationCollector;
|
||||
use crate::DocId;
|
||||
|
||||
pub(crate) const DOC_BLOCK_SIZE: usize = 64;
|
||||
pub(crate) type DocBlock = [DocId; DOC_BLOCK_SIZE];
|
||||
|
||||
/// BufAggregationCollector buffers documents before calling collect_block().
|
||||
#[derive(Clone)]
|
||||
pub(crate) struct BufAggregationCollector {
|
||||
pub(crate) collector: Box<dyn SegmentAggregationCollector>,
|
||||
staged_docs: DocBlock,
|
||||
num_staged_docs: usize,
|
||||
}
|
||||
|
||||
impl std::fmt::Debug for BufAggregationCollector {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("SegmentAggregationResultsCollector")
|
||||
.field("staged_docs", &&self.staged_docs[..self.num_staged_docs])
|
||||
.field("num_staged_docs", &self.num_staged_docs)
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
impl BufAggregationCollector {
|
||||
pub fn new(collector: Box<dyn SegmentAggregationCollector>) -> Self {
|
||||
Self {
|
||||
collector,
|
||||
num_staged_docs: 0,
|
||||
staged_docs: [0; DOC_BLOCK_SIZE],
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl SegmentAggregationCollector for BufAggregationCollector {
|
||||
fn into_intermediate_aggregations_result(
|
||||
self: Box<Self>,
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<IntermediateAggregationResults> {
|
||||
Box::new(self.collector).into_intermediate_aggregations_result(agg_with_accessor)
|
||||
}
|
||||
|
||||
fn collect(
|
||||
&mut self,
|
||||
doc: crate::DocId,
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<()> {
|
||||
self.staged_docs[self.num_staged_docs] = doc;
|
||||
self.num_staged_docs += 1;
|
||||
if self.num_staged_docs == self.staged_docs.len() {
|
||||
self.collector
|
||||
.collect_block(&self.staged_docs[..self.num_staged_docs], agg_with_accessor)?;
|
||||
self.num_staged_docs = 0;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn collect_block(
|
||||
&mut self,
|
||||
docs: &[crate::DocId],
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<()> {
|
||||
for doc in docs {
|
||||
self.collect(*doc, agg_with_accessor)?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn flush(&mut self, agg_with_accessor: &AggregationsWithAccessor) -> crate::Result<()> {
|
||||
self.collector
|
||||
.collect_block(&self.staged_docs[..self.num_staged_docs], agg_with_accessor)?;
|
||||
self.num_staged_docs = 0;
|
||||
|
||||
self.collector.flush(agg_with_accessor)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
@@ -3,11 +3,11 @@ use std::rc::Rc;
|
||||
use super::agg_req::Aggregations;
|
||||
use super::agg_req_with_accessor::AggregationsWithAccessor;
|
||||
use super::agg_result::AggregationResults;
|
||||
use super::buf_collector::BufAggregationCollector;
|
||||
use super::intermediate_agg_result::IntermediateAggregationResults;
|
||||
use super::segment_agg_result::{build_segment_agg_collector, SegmentAggregationCollector};
|
||||
use crate::aggregation::agg_req_with_accessor::get_aggs_with_accessor_and_validate;
|
||||
use crate::collector::{Collector, SegmentCollector};
|
||||
use crate::schema::Schema;
|
||||
use crate::{SegmentReader, TantivyError};
|
||||
|
||||
/// The default max bucket count, before the aggregation fails.
|
||||
@@ -17,7 +17,6 @@ pub const MAX_BUCKET_COUNT: u32 = 65000;
|
||||
///
|
||||
/// The collector collects all aggregations by the underlying aggregation request.
|
||||
pub struct AggregationCollector {
|
||||
schema: Schema,
|
||||
agg: Aggregations,
|
||||
max_bucket_count: u32,
|
||||
}
|
||||
@@ -27,9 +26,8 @@ impl AggregationCollector {
|
||||
///
|
||||
/// Aggregation fails when the total bucket count is higher than max_bucket_count.
|
||||
/// max_bucket_count will default to `MAX_BUCKET_COUNT` (65000) when unset
|
||||
pub fn from_aggs(agg: Aggregations, max_bucket_count: Option<u32>, schema: Schema) -> Self {
|
||||
pub fn from_aggs(agg: Aggregations, max_bucket_count: Option<u32>) -> Self {
|
||||
Self {
|
||||
schema,
|
||||
agg,
|
||||
max_bucket_count: max_bucket_count.unwrap_or(MAX_BUCKET_COUNT),
|
||||
}
|
||||
@@ -116,7 +114,7 @@ impl Collector for AggregationCollector {
|
||||
segment_fruits: Vec<<Self::Child as SegmentCollector>::Fruit>,
|
||||
) -> crate::Result<Self::Fruit> {
|
||||
let res = merge_fruits(segment_fruits)?;
|
||||
res.into_final_bucket_result(self.agg.clone(), &self.schema)
|
||||
res.into_final_bucket_result(self.agg.clone())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -137,7 +135,7 @@ fn merge_fruits(
|
||||
/// `AggregationSegmentCollector` does the aggregation collection on a segment.
|
||||
pub struct AggregationSegmentCollector {
|
||||
aggs_with_accessor: AggregationsWithAccessor,
|
||||
result: Box<dyn SegmentAggregationCollector>,
|
||||
result: BufAggregationCollector,
|
||||
error: Option<TantivyError>,
|
||||
}
|
||||
|
||||
@@ -151,7 +149,8 @@ impl AggregationSegmentCollector {
|
||||
) -> crate::Result<Self> {
|
||||
let aggs_with_accessor =
|
||||
get_aggs_with_accessor_and_validate(agg, reader, Rc::default(), max_bucket_count)?;
|
||||
let result = build_segment_agg_collector(&aggs_with_accessor)?;
|
||||
let result =
|
||||
BufAggregationCollector::new(build_segment_agg_collector(&aggs_with_accessor)?);
|
||||
Ok(AggregationSegmentCollector {
|
||||
aggs_with_accessor,
|
||||
result,
|
||||
@@ -177,9 +176,7 @@ impl SegmentCollector for AggregationSegmentCollector {
|
||||
if let Some(err) = self.error {
|
||||
return Err(err);
|
||||
}
|
||||
self.result
|
||||
.flush_staged_docs(&self.aggs_with_accessor, true)?;
|
||||
self.result
|
||||
.into_intermediate_aggregations_result(&self.aggs_with_accessor)
|
||||
self.result.flush(&self.aggs_with_accessor)?;
|
||||
Box::new(self.result).into_intermediate_aggregations_result(&self.aggs_with_accessor)
|
||||
}
|
||||
}
|
||||
|
||||
9
src/aggregation/error.rs
Normal file
9
src/aggregation/error.rs
Normal file
@@ -0,0 +1,9 @@
|
||||
use super::bucket::DateHistogramParseError;
|
||||
|
||||
/// Error that may occur when opening a directory
|
||||
#[derive(Debug, Clone, PartialEq, Eq, Error)]
|
||||
pub enum AggregationError {
|
||||
/// Failed to open the directory.
|
||||
#[error("Date histogram parse error: {0:?}")]
|
||||
DateHistogramParseError(#[from] DateHistogramParseError),
|
||||
}
|
||||
@@ -4,6 +4,7 @@
|
||||
|
||||
use std::cmp::Ordering;
|
||||
|
||||
use columnar::ColumnType;
|
||||
use itertools::Itertools;
|
||||
use rustc_hash::FxHashMap;
|
||||
use serde::{Deserialize, Serialize};
|
||||
@@ -21,11 +22,9 @@ use super::metric::{
|
||||
IntermediateAverage, IntermediateCount, IntermediateMax, IntermediateMin, IntermediateStats,
|
||||
IntermediateSum,
|
||||
};
|
||||
use super::segment_agg_result::SegmentMetricResultCollector;
|
||||
use super::{format_date, Key, SerializedKey, VecWithNames};
|
||||
use crate::aggregation::agg_result::{AggregationResults, BucketEntries, BucketEntry};
|
||||
use crate::aggregation::bucket::TermsAggregationInternal;
|
||||
use crate::schema::Schema;
|
||||
|
||||
/// Contains the intermediate aggregation result, which is optimized to be merged with other
|
||||
/// intermediate results.
|
||||
@@ -39,12 +38,8 @@ pub struct IntermediateAggregationResults {
|
||||
|
||||
impl IntermediateAggregationResults {
|
||||
/// Convert intermediate result and its aggregation request to the final result.
|
||||
pub fn into_final_bucket_result(
|
||||
self,
|
||||
req: Aggregations,
|
||||
schema: &Schema,
|
||||
) -> crate::Result<AggregationResults> {
|
||||
self.into_final_bucket_result_internal(&(req.into()), schema)
|
||||
pub fn into_final_bucket_result(self, req: Aggregations) -> crate::Result<AggregationResults> {
|
||||
self.into_final_bucket_result_internal(&(req.into()))
|
||||
}
|
||||
|
||||
/// Convert intermediate result and its aggregation request to the final result.
|
||||
@@ -54,7 +49,6 @@ impl IntermediateAggregationResults {
|
||||
pub(crate) fn into_final_bucket_result_internal(
|
||||
self,
|
||||
req: &AggregationsInternal,
|
||||
schema: &Schema,
|
||||
) -> crate::Result<AggregationResults> {
|
||||
// Important assumption:
|
||||
// When the tree contains buckets/metric, we expect it to have all buckets/metrics from the
|
||||
@@ -62,11 +56,11 @@ impl IntermediateAggregationResults {
|
||||
let mut results: FxHashMap<String, AggregationResult> = FxHashMap::default();
|
||||
|
||||
if let Some(buckets) = self.buckets {
|
||||
convert_and_add_final_buckets_to_result(&mut results, buckets, &req.buckets, schema)?
|
||||
convert_and_add_final_buckets_to_result(&mut results, buckets, &req.buckets)?
|
||||
} else {
|
||||
// When there are no buckets, we create empty buckets, so that the serialized json
|
||||
// format is constant
|
||||
add_empty_final_buckets_to_result(&mut results, &req.buckets, schema)?
|
||||
add_empty_final_buckets_to_result(&mut results, &req.buckets)?
|
||||
};
|
||||
|
||||
if let Some(metrics) = self.metrics {
|
||||
@@ -167,12 +161,10 @@ fn add_empty_final_metrics_to_result(
|
||||
fn add_empty_final_buckets_to_result(
|
||||
results: &mut FxHashMap<String, AggregationResult>,
|
||||
req_buckets: &VecWithNames<BucketAggregationInternal>,
|
||||
schema: &Schema,
|
||||
) -> crate::Result<()> {
|
||||
let requested_buckets = req_buckets.iter();
|
||||
for (key, req) in requested_buckets {
|
||||
let empty_bucket =
|
||||
AggregationResult::BucketResult(BucketResult::empty_from_req(req, schema)?);
|
||||
let empty_bucket = AggregationResult::BucketResult(BucketResult::empty_from_req(req)?);
|
||||
results.insert(key.to_string(), empty_bucket);
|
||||
}
|
||||
Ok(())
|
||||
@@ -182,13 +174,12 @@ fn convert_and_add_final_buckets_to_result(
|
||||
results: &mut FxHashMap<String, AggregationResult>,
|
||||
buckets: VecWithNames<IntermediateBucketResult>,
|
||||
req_buckets: &VecWithNames<BucketAggregationInternal>,
|
||||
schema: &Schema,
|
||||
) -> crate::Result<()> {
|
||||
assert_eq!(buckets.len(), req_buckets.len());
|
||||
|
||||
let buckets_with_request = buckets.into_iter().zip(req_buckets.values());
|
||||
for ((key, bucket), req) in buckets_with_request {
|
||||
let result = AggregationResult::BucketResult(bucket.into_final_bucket_result(req, schema)?);
|
||||
let result = AggregationResult::BucketResult(bucket.into_final_bucket_result(req)?);
|
||||
results.insert(key, result);
|
||||
}
|
||||
Ok(())
|
||||
@@ -220,32 +211,6 @@ pub enum IntermediateMetricResult {
|
||||
Sum(IntermediateSum),
|
||||
}
|
||||
|
||||
impl From<SegmentMetricResultCollector> for IntermediateMetricResult {
|
||||
fn from(tree: SegmentMetricResultCollector) -> Self {
|
||||
use super::metric::SegmentStatsType;
|
||||
match tree {
|
||||
SegmentMetricResultCollector::Stats(collector) => match collector.collecting_for {
|
||||
SegmentStatsType::Average => IntermediateMetricResult::Average(
|
||||
IntermediateAverage::from_collector(collector),
|
||||
),
|
||||
SegmentStatsType::Count => {
|
||||
IntermediateMetricResult::Count(IntermediateCount::from_collector(collector))
|
||||
}
|
||||
SegmentStatsType::Max => {
|
||||
IntermediateMetricResult::Max(IntermediateMax::from_collector(collector))
|
||||
}
|
||||
SegmentStatsType::Min => {
|
||||
IntermediateMetricResult::Min(IntermediateMin::from_collector(collector))
|
||||
}
|
||||
SegmentStatsType::Stats => IntermediateMetricResult::Stats(collector.stats),
|
||||
SegmentStatsType::Sum => {
|
||||
IntermediateMetricResult::Sum(IntermediateSum::from_collector(collector))
|
||||
}
|
||||
},
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl IntermediateMetricResult {
|
||||
pub(crate) fn empty_from_req(req: &MetricAggregation) -> Self {
|
||||
match req {
|
||||
@@ -309,6 +274,8 @@ pub enum IntermediateBucketResult {
|
||||
/// This is the histogram entry for a bucket, which contains a key, count, and optionally
|
||||
/// sub_aggregations.
|
||||
Histogram {
|
||||
/// The column_type of the underlying `Column`
|
||||
column_type: Option<ColumnType>,
|
||||
/// The buckets
|
||||
buckets: Vec<IntermediateHistogramBucketEntry>,
|
||||
},
|
||||
@@ -320,7 +287,6 @@ impl IntermediateBucketResult {
|
||||
pub(crate) fn into_final_bucket_result(
|
||||
self,
|
||||
req: &BucketAggregationInternal,
|
||||
schema: &Schema,
|
||||
) -> crate::Result<BucketResult> {
|
||||
match self {
|
||||
IntermediateBucketResult::Range(range_res) => {
|
||||
@@ -330,9 +296,9 @@ impl IntermediateBucketResult {
|
||||
.map(|bucket| {
|
||||
bucket.into_final_bucket_entry(
|
||||
&req.sub_aggregation,
|
||||
schema,
|
||||
req.as_range()
|
||||
.expect("unexpected aggregation, expected histogram aggregation"),
|
||||
range_res.column_type,
|
||||
)
|
||||
})
|
||||
.collect::<crate::Result<Vec<_>>>()?;
|
||||
@@ -359,16 +325,21 @@ impl IntermediateBucketResult {
|
||||
};
|
||||
Ok(BucketResult::Range { buckets })
|
||||
}
|
||||
IntermediateBucketResult::Histogram { buckets } => {
|
||||
IntermediateBucketResult::Histogram {
|
||||
column_type,
|
||||
buckets,
|
||||
} => {
|
||||
let histogram_req = &req
|
||||
.as_histogram()?
|
||||
.expect("unexpected aggregation, expected histogram aggregation");
|
||||
let buckets = intermediate_histogram_buckets_to_final_buckets(
|
||||
buckets,
|
||||
req.as_histogram()
|
||||
.expect("unexpected aggregation, expected histogram aggregation"),
|
||||
column_type,
|
||||
histogram_req,
|
||||
&req.sub_aggregation,
|
||||
schema,
|
||||
)?;
|
||||
|
||||
let buckets = if req.as_histogram().unwrap().keyed {
|
||||
let buckets = if histogram_req.keyed {
|
||||
let mut bucket_map =
|
||||
FxHashMap::with_capacity_and_hasher(buckets.len(), Default::default());
|
||||
for bucket in buckets {
|
||||
@@ -384,7 +355,6 @@ impl IntermediateBucketResult {
|
||||
req.as_term()
|
||||
.expect("unexpected aggregation, expected term aggregation"),
|
||||
&req.sub_aggregation,
|
||||
schema,
|
||||
),
|
||||
}
|
||||
}
|
||||
@@ -393,8 +363,11 @@ impl IntermediateBucketResult {
|
||||
match req {
|
||||
BucketAggregationType::Terms(_) => IntermediateBucketResult::Terms(Default::default()),
|
||||
BucketAggregationType::Range(_) => IntermediateBucketResult::Range(Default::default()),
|
||||
BucketAggregationType::Histogram(_) => {
|
||||
IntermediateBucketResult::Histogram { buckets: vec![] }
|
||||
BucketAggregationType::Histogram(_) | BucketAggregationType::DateHistogram(_) => {
|
||||
IntermediateBucketResult::Histogram {
|
||||
buckets: vec![],
|
||||
column_type: None,
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -404,7 +377,7 @@ impl IntermediateBucketResult {
|
||||
IntermediateBucketResult::Terms(term_res_left),
|
||||
IntermediateBucketResult::Terms(term_res_right),
|
||||
) => {
|
||||
merge_maps(&mut term_res_left.entries, term_res_right.entries);
|
||||
merge_key_maps(&mut term_res_left.entries, term_res_right.entries);
|
||||
term_res_left.sum_other_doc_count += term_res_right.sum_other_doc_count;
|
||||
term_res_left.doc_count_error_upper_bound +=
|
||||
term_res_right.doc_count_error_upper_bound;
|
||||
@@ -414,7 +387,7 @@ impl IntermediateBucketResult {
|
||||
IntermediateBucketResult::Range(range_res_left),
|
||||
IntermediateBucketResult::Range(range_res_right),
|
||||
) => {
|
||||
merge_maps(&mut range_res_left.buckets, range_res_right.buckets);
|
||||
merge_serialized_key_maps(&mut range_res_left.buckets, range_res_right.buckets);
|
||||
}
|
||||
(
|
||||
IntermediateBucketResult::Histogram {
|
||||
@@ -460,12 +433,13 @@ impl IntermediateBucketResult {
|
||||
/// Range aggregation including error counts
|
||||
pub struct IntermediateRangeBucketResult {
|
||||
pub(crate) buckets: FxHashMap<SerializedKey, IntermediateRangeBucketEntry>,
|
||||
pub(crate) column_type: Option<ColumnType>,
|
||||
}
|
||||
|
||||
#[derive(Default, Clone, Debug, PartialEq, Serialize, Deserialize)]
|
||||
/// Term aggregation including error counts
|
||||
pub struct IntermediateTermBucketResult {
|
||||
pub(crate) entries: FxHashMap<String, IntermediateTermBucketEntry>,
|
||||
pub(crate) entries: FxHashMap<Key, IntermediateTermBucketEntry>,
|
||||
pub(crate) sum_other_doc_count: u64,
|
||||
pub(crate) doc_count_error_upper_bound: u64,
|
||||
}
|
||||
@@ -475,7 +449,6 @@ impl IntermediateTermBucketResult {
|
||||
self,
|
||||
req: &TermsAggregation,
|
||||
sub_aggregation_req: &AggregationsInternal,
|
||||
schema: &Schema,
|
||||
) -> crate::Result<BucketResult> {
|
||||
let req = TermsAggregationInternal::from_req(req);
|
||||
let mut buckets: Vec<BucketEntry> = self
|
||||
@@ -485,11 +458,11 @@ impl IntermediateTermBucketResult {
|
||||
.map(|(key, entry)| {
|
||||
Ok(BucketEntry {
|
||||
key_as_string: None,
|
||||
key: Key::Str(key),
|
||||
key,
|
||||
doc_count: entry.doc_count,
|
||||
sub_aggregation: entry
|
||||
.sub_aggregation
|
||||
.into_final_bucket_result_internal(sub_aggregation_req, schema)?,
|
||||
.into_final_bucket_result_internal(sub_aggregation_req)?,
|
||||
})
|
||||
})
|
||||
.collect::<crate::Result<_>>()?;
|
||||
@@ -563,7 +536,7 @@ trait MergeFruits {
|
||||
fn merge_fruits(&mut self, other: Self);
|
||||
}
|
||||
|
||||
fn merge_maps<V: MergeFruits + Clone>(
|
||||
fn merge_serialized_key_maps<V: MergeFruits + Clone>(
|
||||
entries_left: &mut FxHashMap<SerializedKey, V>,
|
||||
mut entries_right: FxHashMap<SerializedKey, V>,
|
||||
) {
|
||||
@@ -578,6 +551,21 @@ fn merge_maps<V: MergeFruits + Clone>(
|
||||
}
|
||||
}
|
||||
|
||||
fn merge_key_maps<V: MergeFruits + Clone>(
|
||||
entries_left: &mut FxHashMap<Key, V>,
|
||||
mut entries_right: FxHashMap<Key, V>,
|
||||
) {
|
||||
for (name, entry_left) in entries_left.iter_mut() {
|
||||
if let Some(entry_right) = entries_right.remove(name) {
|
||||
entry_left.merge_fruits(entry_right);
|
||||
}
|
||||
}
|
||||
|
||||
for (key, res) in entries_right.into_iter() {
|
||||
entries_left.entry(key).or_insert(res);
|
||||
}
|
||||
}
|
||||
|
||||
/// This is the histogram entry for a bucket, which contains a key, count, and optionally
|
||||
/// sub_aggregations.
|
||||
#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)]
|
||||
@@ -594,7 +582,6 @@ impl IntermediateHistogramBucketEntry {
|
||||
pub(crate) fn into_final_bucket_entry(
|
||||
self,
|
||||
req: &AggregationsInternal,
|
||||
schema: &Schema,
|
||||
) -> crate::Result<BucketEntry> {
|
||||
Ok(BucketEntry {
|
||||
key_as_string: None,
|
||||
@@ -602,7 +589,7 @@ impl IntermediateHistogramBucketEntry {
|
||||
doc_count: self.doc_count,
|
||||
sub_aggregation: self
|
||||
.sub_aggregation
|
||||
.into_final_bucket_result_internal(req, schema)?,
|
||||
.into_final_bucket_result_internal(req)?,
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -639,15 +626,15 @@ impl IntermediateRangeBucketEntry {
|
||||
pub(crate) fn into_final_bucket_entry(
|
||||
self,
|
||||
req: &AggregationsInternal,
|
||||
schema: &Schema,
|
||||
range_req: &RangeAggregation,
|
||||
_range_req: &RangeAggregation,
|
||||
column_type: Option<ColumnType>,
|
||||
) -> crate::Result<RangeBucketEntry> {
|
||||
let mut range_bucket_entry = RangeBucketEntry {
|
||||
key: self.key,
|
||||
doc_count: self.doc_count,
|
||||
sub_aggregation: self
|
||||
.sub_aggregation
|
||||
.into_final_bucket_result_internal(req, schema)?,
|
||||
.into_final_bucket_result_internal(req)?,
|
||||
to: self.to,
|
||||
from: self.from,
|
||||
to_as_string: None,
|
||||
@@ -656,8 +643,7 @@ impl IntermediateRangeBucketEntry {
|
||||
|
||||
// If we have a date type on the histogram buckets, we add the `key_as_string` field as
|
||||
// rfc339
|
||||
let field = schema.get_field(&range_req.field)?;
|
||||
if schema.get_field_entry(field).field_type().is_date() {
|
||||
if column_type == Some(ColumnType::DateTime) {
|
||||
if let Some(val) = range_bucket_entry.to {
|
||||
let key_as_string = format_date(val as i64)?;
|
||||
range_bucket_entry.to_as_string = Some(key_as_string);
|
||||
@@ -728,7 +714,10 @@ mod tests {
|
||||
}
|
||||
map.insert(
|
||||
"my_agg_level2".to_string(),
|
||||
IntermediateBucketResult::Range(IntermediateRangeBucketResult { buckets }),
|
||||
IntermediateBucketResult::Range(IntermediateRangeBucketResult {
|
||||
buckets,
|
||||
column_type: None,
|
||||
}),
|
||||
);
|
||||
IntermediateAggregationResults {
|
||||
buckets: Some(VecWithNames::from_entries(map.into_iter().collect())),
|
||||
@@ -758,7 +747,10 @@ mod tests {
|
||||
}
|
||||
map.insert(
|
||||
"my_agg_level1".to_string(),
|
||||
IntermediateBucketResult::Range(IntermediateRangeBucketResult { buckets }),
|
||||
IntermediateBucketResult::Range(IntermediateRangeBucketResult {
|
||||
buckets,
|
||||
column_type: None,
|
||||
}),
|
||||
);
|
||||
IntermediateAggregationResults {
|
||||
buckets: Some(VecWithNames::from_entries(map.into_iter().collect())),
|
||||
|
||||
@@ -81,7 +81,7 @@ mod tests {
|
||||
"price_sum": { "sum": { "field": "price" } }
|
||||
}"#;
|
||||
let aggregations: Aggregations = serde_json::from_str(aggregations_json).unwrap();
|
||||
let collector = AggregationCollector::from_aggs(aggregations, None, index.schema());
|
||||
let collector = AggregationCollector::from_aggs(aggregations, None);
|
||||
let reader = index.reader().unwrap();
|
||||
let searcher = reader.searcher();
|
||||
let aggregations_res: AggregationResults = searcher.search(&AllQuery, &collector).unwrap();
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
use columnar::Column;
|
||||
use columnar::{Cardinality, Column, ColumnType};
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
use super::*;
|
||||
@@ -8,7 +8,6 @@ use crate::aggregation::intermediate_agg_result::{
|
||||
};
|
||||
use crate::aggregation::segment_agg_result::SegmentAggregationCollector;
|
||||
use crate::aggregation::{f64_from_fastfield_u64, VecWithNames};
|
||||
use crate::schema::Type;
|
||||
use crate::{DocId, TantivyError};
|
||||
|
||||
/// A multi-value metric aggregation that computes a collection of statistics on numeric values that
|
||||
@@ -153,26 +152,40 @@ pub(crate) enum SegmentStatsType {
|
||||
|
||||
#[derive(Clone, Debug, PartialEq)]
|
||||
pub(crate) struct SegmentStatsCollector {
|
||||
field_type: Type,
|
||||
field_type: ColumnType,
|
||||
pub(crate) collecting_for: SegmentStatsType,
|
||||
pub(crate) stats: IntermediateStats,
|
||||
pub(crate) accessor_idx: usize,
|
||||
}
|
||||
|
||||
impl SegmentStatsCollector {
|
||||
pub fn from_req(field_type: Type, collecting_for: SegmentStatsType) -> Self {
|
||||
pub fn from_req(
|
||||
field_type: ColumnType,
|
||||
collecting_for: SegmentStatsType,
|
||||
accessor_idx: usize,
|
||||
) -> Self {
|
||||
Self {
|
||||
field_type,
|
||||
collecting_for,
|
||||
stats: IntermediateStats::default(),
|
||||
accessor_idx,
|
||||
}
|
||||
}
|
||||
pub(crate) fn collect_block(&mut self, docs: &[DocId], field: &Column<u64>) {
|
||||
// TODO special case for Required, Optional column type
|
||||
for doc in docs {
|
||||
for val in field.values(*doc) {
|
||||
#[inline]
|
||||
pub(crate) fn collect_block_with_field(&mut self, docs: &[DocId], field: &Column<u64>) {
|
||||
if field.get_cardinality() == Cardinality::Full {
|
||||
for doc in docs {
|
||||
let val = field.values.get_val(*doc);
|
||||
let val1 = f64_from_fastfield_u64(val, &self.field_type);
|
||||
self.stats.collect(val1);
|
||||
}
|
||||
} else {
|
||||
for doc in docs {
|
||||
for val in field.values_for_doc(*doc) {
|
||||
let val1 = f64_from_fastfield_u64(val, &self.field_type);
|
||||
self.stats.collect(val1);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -182,7 +195,7 @@ impl SegmentAggregationCollector for SegmentStatsCollector {
|
||||
self: Box<Self>,
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<IntermediateAggregationResults> {
|
||||
let name = agg_with_accessor.metrics.keys[0].to_string();
|
||||
let name = agg_with_accessor.metrics.keys[self.accessor_idx].to_string();
|
||||
|
||||
let intermediate_metric_result = match self.collecting_for {
|
||||
SegmentStatsType::Average => {
|
||||
@@ -219,8 +232,9 @@ impl SegmentAggregationCollector for SegmentStatsCollector {
|
||||
doc: crate::DocId,
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<()> {
|
||||
let accessor = &agg_with_accessor.metrics.values[0].accessor;
|
||||
for val in accessor.values(doc) {
|
||||
let field = &agg_with_accessor.metrics.values[self.accessor_idx].accessor;
|
||||
|
||||
for val in field.values_for_doc(doc) {
|
||||
let val1 = f64_from_fastfield_u64(val, &self.field_type);
|
||||
self.stats.collect(val1);
|
||||
}
|
||||
@@ -228,11 +242,14 @@ impl SegmentAggregationCollector for SegmentStatsCollector {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn flush_staged_docs(
|
||||
#[inline]
|
||||
fn collect_block(
|
||||
&mut self,
|
||||
_agg_with_accessor: &AggregationsWithAccessor,
|
||||
_force_flush: bool,
|
||||
docs: &[crate::DocId],
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<()> {
|
||||
let field = &agg_with_accessor.metrics.values[self.accessor_idx].accessor;
|
||||
self.collect_block_with_field(docs, field);
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
@@ -272,7 +289,7 @@ mod tests {
|
||||
.into_iter()
|
||||
.collect();
|
||||
|
||||
let collector = AggregationCollector::from_aggs(agg_req_1, None, index.schema());
|
||||
let collector = AggregationCollector::from_aggs(agg_req_1, None);
|
||||
|
||||
let reader = index.reader()?;
|
||||
let searcher = reader.searcher();
|
||||
@@ -293,6 +310,43 @@ mod tests {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_aggregation_stats_simple() -> crate::Result<()> {
|
||||
// test index without segments
|
||||
let values = vec![10.0];
|
||||
|
||||
let index = get_test_index_from_values(false, &values)?;
|
||||
|
||||
let agg_req_1: Aggregations = vec![(
|
||||
"stats".to_string(),
|
||||
Aggregation::Metric(MetricAggregation::Stats(StatsAggregation::from_field_name(
|
||||
"score".to_string(),
|
||||
))),
|
||||
)]
|
||||
.into_iter()
|
||||
.collect();
|
||||
|
||||
let collector = AggregationCollector::from_aggs(agg_req_1, None);
|
||||
|
||||
let reader = index.reader()?;
|
||||
let searcher = reader.searcher();
|
||||
let agg_res: AggregationResults = searcher.search(&AllQuery, &collector).unwrap();
|
||||
|
||||
let res: Value = serde_json::from_str(&serde_json::to_string(&agg_res)?)?;
|
||||
assert_eq!(
|
||||
res["stats"],
|
||||
json!({
|
||||
"avg": 10.0,
|
||||
"count": 1,
|
||||
"max": 10.0,
|
||||
"min": 10.0,
|
||||
"sum": 10.0
|
||||
})
|
||||
);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_aggregation_stats() -> crate::Result<()> {
|
||||
let index = get_test_index_2_segments(false)?;
|
||||
@@ -349,7 +403,7 @@ mod tests {
|
||||
.into_iter()
|
||||
.collect();
|
||||
|
||||
let collector = AggregationCollector::from_aggs(agg_req_1, None, index.schema());
|
||||
let collector = AggregationCollector::from_aggs(agg_req_1, None);
|
||||
|
||||
let searcher = reader.searcher();
|
||||
let agg_res: AggregationResults = searcher.search(&term_query, &collector).unwrap();
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@@ -13,17 +13,14 @@ use super::agg_req_with_accessor::{
|
||||
};
|
||||
use super::bucket::{SegmentHistogramCollector, SegmentRangeCollector, SegmentTermCollector};
|
||||
use super::collector::MAX_BUCKET_COUNT;
|
||||
use super::intermediate_agg_result::{IntermediateAggregationResults, IntermediateBucketResult};
|
||||
use super::intermediate_agg_result::IntermediateAggregationResults;
|
||||
use super::metric::{
|
||||
AverageAggregation, CountAggregation, MaxAggregation, MinAggregation, SegmentStatsCollector,
|
||||
SegmentStatsType, StatsAggregation, SumAggregation,
|
||||
};
|
||||
use super::VecWithNames;
|
||||
use crate::aggregation::agg_req::BucketAggregationType;
|
||||
use crate::{DocId, TantivyError};
|
||||
|
||||
pub(crate) const DOC_BLOCK_SIZE: usize = 64;
|
||||
pub(crate) type DocBlock = [DocId; DOC_BLOCK_SIZE];
|
||||
use crate::TantivyError;
|
||||
|
||||
pub(crate) trait SegmentAggregationCollector: CollectorClone + Debug {
|
||||
fn into_intermediate_aggregations_result(
|
||||
@@ -37,11 +34,17 @@ pub(crate) trait SegmentAggregationCollector: CollectorClone + Debug {
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<()>;
|
||||
|
||||
fn flush_staged_docs(
|
||||
fn collect_block(
|
||||
&mut self,
|
||||
docs: &[crate::DocId],
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
force_flush: bool,
|
||||
) -> crate::Result<()>;
|
||||
|
||||
/// Finalize method. Some Aggregator collect blocks of docs before calling `collect_block`.
|
||||
/// This method ensures those staged docs will be collected.
|
||||
fn flush(&mut self, _agg_with_accessor: &AggregationsWithAccessor) -> crate::Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) trait CollectorClone {
|
||||
@@ -68,54 +71,97 @@ pub(crate) fn build_segment_agg_collector(
|
||||
// Single metric special case
|
||||
if req.buckets.is_empty() && req.metrics.len() == 1 {
|
||||
let req = &req.metrics.values[0];
|
||||
let stats_collector = match &req.metric {
|
||||
MetricAggregation::Average(AverageAggregation { .. }) => {
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Average)
|
||||
}
|
||||
MetricAggregation::Count(CountAggregation { .. }) => {
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Count)
|
||||
}
|
||||
MetricAggregation::Max(MaxAggregation { .. }) => {
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Max)
|
||||
}
|
||||
MetricAggregation::Min(MinAggregation { .. }) => {
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Min)
|
||||
}
|
||||
MetricAggregation::Stats(StatsAggregation { .. }) => {
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Stats)
|
||||
}
|
||||
MetricAggregation::Sum(SumAggregation { .. }) => {
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Sum)
|
||||
}
|
||||
};
|
||||
let accessor_idx = 0;
|
||||
return build_metric_segment_agg_collector(req, accessor_idx);
|
||||
}
|
||||
|
||||
return Ok(Box::new(stats_collector));
|
||||
// Single bucket special case
|
||||
if req.metrics.is_empty() && req.buckets.len() == 1 {
|
||||
let req = &req.buckets.values[0];
|
||||
let accessor_idx = 0;
|
||||
return build_bucket_segment_agg_collector(req, accessor_idx);
|
||||
}
|
||||
|
||||
let agg = GenericSegmentAggregationResultsCollector::from_req_and_validate(req)?;
|
||||
Ok(Box::new(agg))
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
pub(crate) fn build_metric_segment_agg_collector(
|
||||
req: &MetricAggregationWithAccessor,
|
||||
accessor_idx: usize,
|
||||
) -> crate::Result<Box<dyn SegmentAggregationCollector>> {
|
||||
let stats_collector = match &req.metric {
|
||||
MetricAggregation::Average(AverageAggregation { .. }) => {
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Average, accessor_idx)
|
||||
}
|
||||
MetricAggregation::Count(CountAggregation { .. }) => {
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Count, accessor_idx)
|
||||
}
|
||||
MetricAggregation::Max(MaxAggregation { .. }) => {
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Max, accessor_idx)
|
||||
}
|
||||
MetricAggregation::Min(MinAggregation { .. }) => {
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Min, accessor_idx)
|
||||
}
|
||||
MetricAggregation::Stats(StatsAggregation { .. }) => {
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Stats, accessor_idx)
|
||||
}
|
||||
MetricAggregation::Sum(SumAggregation { .. }) => {
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Sum, accessor_idx)
|
||||
}
|
||||
};
|
||||
|
||||
Ok(Box::new(stats_collector))
|
||||
}
|
||||
|
||||
pub(crate) fn build_bucket_segment_agg_collector(
|
||||
req: &BucketAggregationWithAccessor,
|
||||
accessor_idx: usize,
|
||||
) -> crate::Result<Box<dyn SegmentAggregationCollector>> {
|
||||
match &req.bucket_agg {
|
||||
BucketAggregationType::Terms(terms_req) => {
|
||||
Ok(Box::new(SegmentTermCollector::from_req_and_validate(
|
||||
terms_req,
|
||||
&req.sub_aggregation,
|
||||
req.field_type,
|
||||
accessor_idx,
|
||||
)?))
|
||||
}
|
||||
BucketAggregationType::Range(range_req) => {
|
||||
Ok(Box::new(SegmentRangeCollector::from_req_and_validate(
|
||||
range_req,
|
||||
&req.sub_aggregation,
|
||||
&req.bucket_count,
|
||||
req.field_type,
|
||||
accessor_idx,
|
||||
)?))
|
||||
}
|
||||
BucketAggregationType::Histogram(histogram) => {
|
||||
Ok(Box::new(SegmentHistogramCollector::from_req_and_validate(
|
||||
histogram,
|
||||
&req.sub_aggregation,
|
||||
req.field_type,
|
||||
accessor_idx,
|
||||
)?))
|
||||
}
|
||||
BucketAggregationType::DateHistogram(histogram) => {
|
||||
Ok(Box::new(SegmentHistogramCollector::from_req_and_validate(
|
||||
&histogram.to_histogram_req()?,
|
||||
&req.sub_aggregation,
|
||||
req.field_type,
|
||||
accessor_idx,
|
||||
)?))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Default)]
|
||||
/// The GenericSegmentAggregationResultsCollector is the generic version of the collector, which
|
||||
/// can handle arbitrary complexity of sub-aggregations. Ideally we never have to pick this one
|
||||
/// and can provide specialized versions instead, that remove some of its overhead.
|
||||
pub(crate) struct GenericSegmentAggregationResultsCollector {
|
||||
pub(crate) metrics: Option<VecWithNames<SegmentMetricResultCollector>>,
|
||||
pub(crate) buckets: Option<VecWithNames<SegmentBucketResultCollector>>,
|
||||
staged_docs: DocBlock,
|
||||
num_staged_docs: usize,
|
||||
}
|
||||
|
||||
impl Default for GenericSegmentAggregationResultsCollector {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
metrics: Default::default(),
|
||||
buckets: Default::default(),
|
||||
staged_docs: [0; DOC_BLOCK_SIZE],
|
||||
num_staged_docs: Default::default(),
|
||||
}
|
||||
}
|
||||
pub(crate) metrics: Option<Vec<Box<dyn SegmentAggregationCollector>>>,
|
||||
pub(crate) buckets: Option<Vec<Box<dyn SegmentAggregationCollector>>>,
|
||||
}
|
||||
|
||||
impl Debug for GenericSegmentAggregationResultsCollector {
|
||||
@@ -123,8 +169,6 @@ impl Debug for GenericSegmentAggregationResultsCollector {
|
||||
f.debug_struct("SegmentAggregationResultsCollector")
|
||||
.field("metrics", &self.metrics)
|
||||
.field("buckets", &self.buckets)
|
||||
.field("staged_docs", &&self.staged_docs[..self.num_staged_docs])
|
||||
.field("num_staged_docs", &self.num_staged_docs)
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
@@ -135,16 +179,29 @@ impl SegmentAggregationCollector for GenericSegmentAggregationResultsCollector {
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<IntermediateAggregationResults> {
|
||||
let buckets = if let Some(buckets) = self.buckets {
|
||||
let entries = buckets
|
||||
.into_iter()
|
||||
.zip(agg_with_accessor.buckets.values())
|
||||
.map(|((key, bucket), acc)| Ok((key, bucket.into_intermediate_bucket_result(acc)?)))
|
||||
.collect::<crate::Result<Vec<(String, _)>>>()?;
|
||||
Some(VecWithNames::from_entries(entries))
|
||||
let mut intermeditate_buckets = VecWithNames::default();
|
||||
for bucket in buckets {
|
||||
// TODO too many allocations?
|
||||
let res = bucket.into_intermediate_aggregations_result(agg_with_accessor)?;
|
||||
// unwrap is fine since we only have buckets here
|
||||
intermeditate_buckets.extend(res.buckets.unwrap());
|
||||
}
|
||||
Some(intermeditate_buckets)
|
||||
} else {
|
||||
None
|
||||
};
|
||||
let metrics = if let Some(metrics) = self.metrics {
|
||||
let mut intermeditate_metrics = VecWithNames::default();
|
||||
for metric in metrics {
|
||||
// TODO too many allocations?
|
||||
let res = metric.into_intermediate_aggregations_result(agg_with_accessor)?;
|
||||
// unwrap is fine since we only have metrics here
|
||||
intermeditate_metrics.extend(res.metrics.unwrap());
|
||||
}
|
||||
Some(intermeditate_metrics)
|
||||
} else {
|
||||
None
|
||||
};
|
||||
let metrics = self.metrics.map(VecWithNames::from_other);
|
||||
|
||||
Ok(IntermediateAggregationResults { metrics, buckets })
|
||||
}
|
||||
@@ -154,229 +211,77 @@ impl SegmentAggregationCollector for GenericSegmentAggregationResultsCollector {
|
||||
doc: crate::DocId,
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<()> {
|
||||
self.staged_docs[self.num_staged_docs] = doc;
|
||||
self.num_staged_docs += 1;
|
||||
if self.num_staged_docs == self.staged_docs.len() {
|
||||
self.flush_staged_docs(agg_with_accessor, false)?;
|
||||
}
|
||||
self.collect_block(&[doc], agg_with_accessor)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn flush_staged_docs(
|
||||
fn collect_block(
|
||||
&mut self,
|
||||
docs: &[crate::DocId],
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
force_flush: bool,
|
||||
) -> crate::Result<()> {
|
||||
if self.num_staged_docs == 0 {
|
||||
return Ok(());
|
||||
if let Some(metrics) = self.metrics.as_mut() {
|
||||
for collector in metrics {
|
||||
collector.collect_block(docs, agg_with_accessor)?;
|
||||
}
|
||||
}
|
||||
|
||||
if let Some(buckets) = self.buckets.as_mut() {
|
||||
for collector in buckets {
|
||||
collector.collect_block(docs, agg_with_accessor)?;
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn flush(&mut self, agg_with_accessor: &AggregationsWithAccessor) -> crate::Result<()> {
|
||||
if let Some(metrics) = &mut self.metrics {
|
||||
for (collector, agg_with_accessor) in
|
||||
metrics.values_mut().zip(agg_with_accessor.metrics.values())
|
||||
{
|
||||
collector
|
||||
.collect_block(&self.staged_docs[..self.num_staged_docs], agg_with_accessor);
|
||||
for collector in metrics {
|
||||
collector.flush(agg_with_accessor)?;
|
||||
}
|
||||
}
|
||||
|
||||
if let Some(buckets) = &mut self.buckets {
|
||||
for (collector, agg_with_accessor) in
|
||||
buckets.values_mut().zip(agg_with_accessor.buckets.values())
|
||||
{
|
||||
collector.collect_block(
|
||||
&self.staged_docs[..self.num_staged_docs],
|
||||
agg_with_accessor,
|
||||
force_flush,
|
||||
)?;
|
||||
for collector in buckets {
|
||||
collector.flush(agg_with_accessor)?;
|
||||
}
|
||||
}
|
||||
|
||||
self.num_staged_docs = 0;
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl GenericSegmentAggregationResultsCollector {
|
||||
pub fn into_intermediate_aggregations_result(
|
||||
self,
|
||||
agg_with_accessor: &AggregationsWithAccessor,
|
||||
) -> crate::Result<IntermediateAggregationResults> {
|
||||
let buckets = if let Some(buckets) = self.buckets {
|
||||
let entries = buckets
|
||||
.into_iter()
|
||||
.zip(agg_with_accessor.buckets.values())
|
||||
.map(|((key, bucket), acc)| Ok((key, bucket.into_intermediate_bucket_result(acc)?)))
|
||||
.collect::<crate::Result<Vec<(String, _)>>>()?;
|
||||
Some(VecWithNames::from_entries(entries))
|
||||
} else {
|
||||
None
|
||||
};
|
||||
let metrics = self.metrics.map(VecWithNames::from_other);
|
||||
|
||||
Ok(IntermediateAggregationResults { metrics, buckets })
|
||||
}
|
||||
|
||||
pub(crate) fn from_req_and_validate(req: &AggregationsWithAccessor) -> crate::Result<Self> {
|
||||
let buckets = req
|
||||
.buckets
|
||||
.iter()
|
||||
.map(|(key, req)| {
|
||||
Ok((
|
||||
key.to_string(),
|
||||
SegmentBucketResultCollector::from_req_and_validate(req)?,
|
||||
))
|
||||
.enumerate()
|
||||
.map(|(accessor_idx, (_key, req))| {
|
||||
build_bucket_segment_agg_collector(req, accessor_idx)
|
||||
})
|
||||
.collect::<crate::Result<Vec<(String, _)>>>()?;
|
||||
.collect::<crate::Result<Vec<Box<dyn SegmentAggregationCollector>>>>()?;
|
||||
let metrics = req
|
||||
.metrics
|
||||
.iter()
|
||||
.map(|(key, req)| {
|
||||
Ok((
|
||||
key.to_string(),
|
||||
SegmentMetricResultCollector::from_req_and_validate(req)?,
|
||||
))
|
||||
.enumerate()
|
||||
.map(|(accessor_idx, (_key, req))| {
|
||||
build_metric_segment_agg_collector(req, accessor_idx)
|
||||
})
|
||||
.collect::<crate::Result<Vec<(String, _)>>>()?;
|
||||
.collect::<crate::Result<Vec<Box<dyn SegmentAggregationCollector>>>>()?;
|
||||
|
||||
let metrics = if metrics.is_empty() {
|
||||
None
|
||||
} else {
|
||||
Some(VecWithNames::from_entries(metrics))
|
||||
Some(metrics)
|
||||
};
|
||||
|
||||
let buckets = if buckets.is_empty() {
|
||||
None
|
||||
} else {
|
||||
Some(VecWithNames::from_entries(buckets))
|
||||
Some(buckets)
|
||||
};
|
||||
Ok(GenericSegmentAggregationResultsCollector {
|
||||
metrics,
|
||||
buckets,
|
||||
staged_docs: [0; DOC_BLOCK_SIZE],
|
||||
num_staged_docs: 0,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug, PartialEq)]
|
||||
pub(crate) enum SegmentMetricResultCollector {
|
||||
Stats(SegmentStatsCollector),
|
||||
}
|
||||
|
||||
impl SegmentMetricResultCollector {
|
||||
pub fn from_req_and_validate(req: &MetricAggregationWithAccessor) -> crate::Result<Self> {
|
||||
match &req.metric {
|
||||
MetricAggregation::Average(AverageAggregation { .. }) => {
|
||||
Ok(SegmentMetricResultCollector::Stats(
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Average),
|
||||
))
|
||||
}
|
||||
MetricAggregation::Count(CountAggregation { .. }) => {
|
||||
Ok(SegmentMetricResultCollector::Stats(
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Count),
|
||||
))
|
||||
}
|
||||
MetricAggregation::Max(MaxAggregation { .. }) => {
|
||||
Ok(SegmentMetricResultCollector::Stats(
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Max),
|
||||
))
|
||||
}
|
||||
MetricAggregation::Min(MinAggregation { .. }) => {
|
||||
Ok(SegmentMetricResultCollector::Stats(
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Min),
|
||||
))
|
||||
}
|
||||
MetricAggregation::Stats(StatsAggregation { .. }) => {
|
||||
Ok(SegmentMetricResultCollector::Stats(
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Stats),
|
||||
))
|
||||
}
|
||||
MetricAggregation::Sum(SumAggregation { .. }) => {
|
||||
Ok(SegmentMetricResultCollector::Stats(
|
||||
SegmentStatsCollector::from_req(req.field_type, SegmentStatsType::Sum),
|
||||
))
|
||||
}
|
||||
}
|
||||
}
|
||||
pub(crate) fn collect_block(&mut self, doc: &[DocId], metric: &MetricAggregationWithAccessor) {
|
||||
match self {
|
||||
SegmentMetricResultCollector::Stats(stats_collector) => {
|
||||
stats_collector.collect_block(doc, &metric.accessor);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// SegmentBucketAggregationResultCollectors will have specialized buckets for collection inside
|
||||
/// segments.
|
||||
/// The typical structure of Map<Key, Bucket> is not suitable during collection for performance
|
||||
/// reasons.
|
||||
#[derive(Clone, Debug)]
|
||||
pub(crate) enum SegmentBucketResultCollector {
|
||||
Range(SegmentRangeCollector),
|
||||
Histogram(Box<SegmentHistogramCollector>),
|
||||
Terms(Box<SegmentTermCollector>),
|
||||
}
|
||||
|
||||
impl SegmentBucketResultCollector {
|
||||
pub fn into_intermediate_bucket_result(
|
||||
self,
|
||||
agg_with_accessor: &BucketAggregationWithAccessor,
|
||||
) -> crate::Result<IntermediateBucketResult> {
|
||||
match self {
|
||||
SegmentBucketResultCollector::Terms(terms) => {
|
||||
terms.into_intermediate_bucket_result(agg_with_accessor)
|
||||
}
|
||||
SegmentBucketResultCollector::Range(range) => {
|
||||
range.into_intermediate_bucket_result(agg_with_accessor)
|
||||
}
|
||||
SegmentBucketResultCollector::Histogram(histogram) => {
|
||||
histogram.into_intermediate_bucket_result(agg_with_accessor)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub fn from_req_and_validate(req: &BucketAggregationWithAccessor) -> crate::Result<Self> {
|
||||
match &req.bucket_agg {
|
||||
BucketAggregationType::Terms(terms_req) => Ok(Self::Terms(Box::new(
|
||||
SegmentTermCollector::from_req_and_validate(terms_req, &req.sub_aggregation)?,
|
||||
))),
|
||||
BucketAggregationType::Range(range_req) => {
|
||||
Ok(Self::Range(SegmentRangeCollector::from_req_and_validate(
|
||||
range_req,
|
||||
&req.sub_aggregation,
|
||||
&req.bucket_count,
|
||||
req.field_type,
|
||||
)?))
|
||||
}
|
||||
BucketAggregationType::Histogram(histogram) => Ok(Self::Histogram(Box::new(
|
||||
SegmentHistogramCollector::from_req_and_validate(
|
||||
histogram,
|
||||
&req.sub_aggregation,
|
||||
req.field_type,
|
||||
&req.accessor,
|
||||
)?,
|
||||
))),
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub(crate) fn collect_block(
|
||||
&mut self,
|
||||
doc: &[DocId],
|
||||
bucket_with_accessor: &BucketAggregationWithAccessor,
|
||||
force_flush: bool,
|
||||
) -> crate::Result<()> {
|
||||
match self {
|
||||
SegmentBucketResultCollector::Range(range) => {
|
||||
range.collect_block(doc, bucket_with_accessor, force_flush)?;
|
||||
}
|
||||
SegmentBucketResultCollector::Histogram(histogram) => {
|
||||
histogram.collect_block(doc, bucket_with_accessor, force_flush)?;
|
||||
}
|
||||
SegmentBucketResultCollector::Terms(terms) => {
|
||||
terms.collect_block(doc, bucket_with_accessor, force_flush)?;
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
Ok(GenericSegmentAggregationResultsCollector { metrics, buckets })
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -515,8 +515,7 @@ mod tests {
|
||||
expected_compressed_collapsed_mapping: &[usize],
|
||||
expected_unique_facet_ords: &[(u64, usize)],
|
||||
) {
|
||||
let (compressed_collapsed_mapping, unique_facet_ords) =
|
||||
compress_mapping(&collapsed_mapping);
|
||||
let (compressed_collapsed_mapping, unique_facet_ords) = compress_mapping(collapsed_mapping);
|
||||
assert_eq!(
|
||||
compressed_collapsed_mapping,
|
||||
expected_compressed_collapsed_mapping
|
||||
|
||||
@@ -106,7 +106,7 @@ mod tweak_score_top_collector;
|
||||
pub use self::tweak_score_top_collector::{ScoreSegmentTweaker, ScoreTweaker};
|
||||
mod facet_collector;
|
||||
pub use self::facet_collector::{FacetCollector, FacetCounts};
|
||||
use crate::query::Weight;
|
||||
use crate::query::{for_each_docset, for_each_scorer, Weight};
|
||||
|
||||
mod docset_collector;
|
||||
pub use self::docset_collector::DocSetCollector;
|
||||
@@ -173,28 +173,32 @@ pub trait Collector: Sync + Send {
|
||||
|
||||
match (reader.alive_bitset(), self.requires_scoring()) {
|
||||
(Some(alive_bitset), true) => {
|
||||
weight.for_each(reader, &mut |doc, score| {
|
||||
let mut scorer = weight.scorer(reader, 1.0)?;
|
||||
for_each_scorer(scorer.as_mut(), |doc, score| {
|
||||
if alive_bitset.is_alive(doc) {
|
||||
segment_collector.collect(doc, score);
|
||||
}
|
||||
})?;
|
||||
});
|
||||
}
|
||||
(Some(alive_bitset), false) => {
|
||||
weight.for_each_no_score(reader, &mut |doc| {
|
||||
let mut docset = weight.scorer(reader, 1.0)?;
|
||||
for_each_docset(docset.as_mut(), |doc| {
|
||||
if alive_bitset.is_alive(doc) {
|
||||
segment_collector.collect(doc, 0.0);
|
||||
}
|
||||
})?;
|
||||
});
|
||||
}
|
||||
(None, true) => {
|
||||
weight.for_each(reader, &mut |doc, score| {
|
||||
let mut scorer = weight.scorer(reader, 1.0)?;
|
||||
for_each_scorer(scorer.as_mut(), |doc, score| {
|
||||
segment_collector.collect(doc, score);
|
||||
})?;
|
||||
});
|
||||
}
|
||||
(None, false) => {
|
||||
weight.for_each_no_score(reader, &mut |doc| {
|
||||
let mut docset = weight.scorer(reader, 1.0)?;
|
||||
for_each_docset(docset.as_mut(), |doc| {
|
||||
segment_collector.collect(doc, 0.0);
|
||||
})?;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -56,9 +56,8 @@ pub fn test_filter_collector() -> crate::Result<()> {
|
||||
assert_eq!(filtered_top_docs.len(), 0);
|
||||
|
||||
fn date_filter(value: DateTime) -> bool {
|
||||
(crate::DateTime::from(value).into_utc()
|
||||
- OffsetDateTime::parse("2019-04-09T00:00:00+00:00", &Rfc3339).unwrap())
|
||||
.whole_weeks()
|
||||
(value.into_utc() - OffsetDateTime::parse("2019-04-09T00:00:00+00:00", &Rfc3339).unwrap())
|
||||
.whole_weeks()
|
||||
> 0
|
||||
}
|
||||
|
||||
@@ -201,7 +200,7 @@ impl SegmentCollector for FastFieldSegmentCollector {
|
||||
type Fruit = Vec<u64>;
|
||||
|
||||
fn collect(&mut self, doc: DocId, _score: Score) {
|
||||
self.vals.extend(self.reader.values(doc));
|
||||
self.vals.extend(self.reader.values_for_doc(doc));
|
||||
}
|
||||
|
||||
fn harvest(self) -> Vec<u64> {
|
||||
|
||||
@@ -662,304 +662,3 @@ impl fmt::Debug for Index {
|
||||
write!(f, "Index({:?})", self.directory)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use crate::collector::Count;
|
||||
use crate::directory::{RamDirectory, WatchCallback};
|
||||
use crate::query::TermQuery;
|
||||
use crate::schema::{Field, IndexRecordOption, Schema, INDEXED, TEXT};
|
||||
use crate::tokenizer::TokenizerManager;
|
||||
use crate::{Directory, Index, IndexBuilder, IndexReader, IndexSettings, ReloadPolicy, Term};
|
||||
|
||||
#[test]
|
||||
fn test_indexer_for_field() {
|
||||
let mut schema_builder = Schema::builder();
|
||||
let num_likes_field = schema_builder.add_u64_field("num_likes", INDEXED);
|
||||
let body_field = schema_builder.add_text_field("body", TEXT);
|
||||
let schema = schema_builder.build();
|
||||
let index = Index::create_in_ram(schema);
|
||||
assert!(index.tokenizer_for_field(body_field).is_ok());
|
||||
assert_eq!(
|
||||
format!("{:?}", index.tokenizer_for_field(num_likes_field).err()),
|
||||
"Some(SchemaError(\"\\\"num_likes\\\" is not a text field.\"))"
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_set_tokenizer_manager() {
|
||||
let mut schema_builder = Schema::builder();
|
||||
schema_builder.add_u64_field("num_likes", INDEXED);
|
||||
schema_builder.add_text_field("body", TEXT);
|
||||
let schema = schema_builder.build();
|
||||
let index = IndexBuilder::new()
|
||||
// set empty tokenizer manager
|
||||
.tokenizers(TokenizerManager::new())
|
||||
.schema(schema)
|
||||
.create_in_ram()
|
||||
.unwrap();
|
||||
assert!(index.tokenizers().get("raw").is_none());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_index_exists() {
|
||||
let directory: Box<dyn Directory> = Box::new(RamDirectory::create());
|
||||
assert!(!Index::exists(directory.as_ref()).unwrap());
|
||||
assert!(Index::create(
|
||||
directory.clone(),
|
||||
throw_away_schema(),
|
||||
IndexSettings::default()
|
||||
)
|
||||
.is_ok());
|
||||
assert!(Index::exists(directory.as_ref()).unwrap());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn open_or_create_should_create() {
|
||||
let directory = RamDirectory::create();
|
||||
assert!(!Index::exists(&directory).unwrap());
|
||||
assert!(Index::open_or_create(directory.clone(), throw_away_schema()).is_ok());
|
||||
assert!(Index::exists(&directory).unwrap());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn open_or_create_should_open() {
|
||||
let directory: Box<dyn Directory> = Box::new(RamDirectory::create());
|
||||
assert!(Index::create(
|
||||
directory.clone(),
|
||||
throw_away_schema(),
|
||||
IndexSettings::default()
|
||||
)
|
||||
.is_ok());
|
||||
assert!(Index::exists(directory.as_ref()).unwrap());
|
||||
assert!(Index::open_or_create(directory, throw_away_schema()).is_ok());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn create_should_wipeoff_existing() {
|
||||
let directory: Box<dyn Directory> = Box::new(RamDirectory::create());
|
||||
assert!(Index::create(
|
||||
directory.clone(),
|
||||
throw_away_schema(),
|
||||
IndexSettings::default()
|
||||
)
|
||||
.is_ok());
|
||||
assert!(Index::exists(directory.as_ref()).unwrap());
|
||||
assert!(Index::create(
|
||||
directory,
|
||||
Schema::builder().build(),
|
||||
IndexSettings::default()
|
||||
)
|
||||
.is_ok());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn open_or_create_exists_but_schema_does_not_match() {
|
||||
let directory = RamDirectory::create();
|
||||
assert!(Index::create(
|
||||
directory.clone(),
|
||||
throw_away_schema(),
|
||||
IndexSettings::default()
|
||||
)
|
||||
.is_ok());
|
||||
assert!(Index::exists(&directory).unwrap());
|
||||
assert!(Index::open_or_create(directory.clone(), throw_away_schema()).is_ok());
|
||||
let err = Index::open_or_create(directory, Schema::builder().build());
|
||||
assert_eq!(
|
||||
format!("{:?}", err.unwrap_err()),
|
||||
"SchemaError(\"An index exists but the schema does not match.\")"
|
||||
);
|
||||
}
|
||||
|
||||
fn throw_away_schema() -> Schema {
|
||||
let mut schema_builder = Schema::builder();
|
||||
let _ = schema_builder.add_u64_field("num_likes", INDEXED);
|
||||
schema_builder.build()
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_index_on_commit_reload_policy() -> crate::Result<()> {
|
||||
let schema = throw_away_schema();
|
||||
let field = schema.get_field("num_likes").unwrap();
|
||||
let index = Index::create_in_ram(schema);
|
||||
let reader = index
|
||||
.reader_builder()
|
||||
.reload_policy(ReloadPolicy::OnCommit)
|
||||
.try_into()
|
||||
.unwrap();
|
||||
assert_eq!(reader.searcher().num_docs(), 0);
|
||||
test_index_on_commit_reload_policy_aux(field, &index, &reader)
|
||||
}
|
||||
|
||||
#[cfg(feature = "mmap")]
|
||||
mod mmap_specific {
|
||||
|
||||
use std::path::PathBuf;
|
||||
|
||||
use tempfile::TempDir;
|
||||
|
||||
use super::*;
|
||||
use crate::Directory;
|
||||
|
||||
#[test]
|
||||
fn test_index_on_commit_reload_policy_mmap() -> crate::Result<()> {
|
||||
let schema = throw_away_schema();
|
||||
let field = schema.get_field("num_likes").unwrap();
|
||||
let tempdir = TempDir::new().unwrap();
|
||||
let tempdir_path = PathBuf::from(tempdir.path());
|
||||
let index = Index::create_in_dir(tempdir_path, schema).unwrap();
|
||||
let reader = index
|
||||
.reader_builder()
|
||||
.reload_policy(ReloadPolicy::OnCommit)
|
||||
.try_into()
|
||||
.unwrap();
|
||||
assert_eq!(reader.searcher().num_docs(), 0);
|
||||
test_index_on_commit_reload_policy_aux(field, &index, &reader)
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_index_manual_policy_mmap() -> crate::Result<()> {
|
||||
let schema = throw_away_schema();
|
||||
let field = schema.get_field("num_likes").unwrap();
|
||||
let mut index = Index::create_from_tempdir(schema)?;
|
||||
let mut writer = index.writer_for_tests()?;
|
||||
writer.commit()?;
|
||||
let reader = index
|
||||
.reader_builder()
|
||||
.reload_policy(ReloadPolicy::Manual)
|
||||
.try_into()?;
|
||||
assert_eq!(reader.searcher().num_docs(), 0);
|
||||
writer.add_document(doc!(field=>1u64))?;
|
||||
let (sender, receiver) = crossbeam_channel::unbounded();
|
||||
let _handle = index.directory_mut().watch(WatchCallback::new(move || {
|
||||
let _ = sender.send(());
|
||||
}));
|
||||
writer.commit()?;
|
||||
assert!(receiver.recv().is_ok());
|
||||
assert_eq!(reader.searcher().num_docs(), 0);
|
||||
reader.reload()?;
|
||||
assert_eq!(reader.searcher().num_docs(), 1);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_index_on_commit_reload_policy_different_directories() -> crate::Result<()> {
|
||||
let schema = throw_away_schema();
|
||||
let field = schema.get_field("num_likes").unwrap();
|
||||
let tempdir = TempDir::new().unwrap();
|
||||
let tempdir_path = PathBuf::from(tempdir.path());
|
||||
let write_index = Index::create_in_dir(&tempdir_path, schema).unwrap();
|
||||
let read_index = Index::open_in_dir(&tempdir_path).unwrap();
|
||||
let reader = read_index
|
||||
.reader_builder()
|
||||
.reload_policy(ReloadPolicy::OnCommit)
|
||||
.try_into()
|
||||
.unwrap();
|
||||
assert_eq!(reader.searcher().num_docs(), 0);
|
||||
test_index_on_commit_reload_policy_aux(field, &write_index, &reader)
|
||||
}
|
||||
}
|
||||
fn test_index_on_commit_reload_policy_aux(
|
||||
field: Field,
|
||||
index: &Index,
|
||||
reader: &IndexReader,
|
||||
) -> crate::Result<()> {
|
||||
let mut reader_index = reader.index();
|
||||
let (sender, receiver) = crossbeam_channel::unbounded();
|
||||
let _watch_handle = reader_index
|
||||
.directory_mut()
|
||||
.watch(WatchCallback::new(move || {
|
||||
let _ = sender.send(());
|
||||
}));
|
||||
let mut writer = index.writer_for_tests()?;
|
||||
assert_eq!(reader.searcher().num_docs(), 0);
|
||||
writer.add_document(doc!(field=>1u64))?;
|
||||
writer.commit().unwrap();
|
||||
// We need a loop here because it is possible for notify to send more than
|
||||
// one modify event. It was observed on CI on MacOS.
|
||||
loop {
|
||||
assert!(receiver.recv().is_ok());
|
||||
if reader.searcher().num_docs() == 1 {
|
||||
break;
|
||||
}
|
||||
}
|
||||
writer.add_document(doc!(field=>2u64))?;
|
||||
writer.commit().unwrap();
|
||||
// ... Same as above
|
||||
loop {
|
||||
assert!(receiver.recv().is_ok());
|
||||
if reader.searcher().num_docs() == 2 {
|
||||
break;
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
// This test will not pass on windows, because windows
|
||||
// prevent deleting files that are MMapped.
|
||||
#[cfg(not(target_os = "windows"))]
|
||||
#[test]
|
||||
fn garbage_collect_works_as_intended() -> crate::Result<()> {
|
||||
let directory = RamDirectory::create();
|
||||
let schema = throw_away_schema();
|
||||
let field = schema.get_field("num_likes").unwrap();
|
||||
let index = Index::create(directory.clone(), schema, IndexSettings::default())?;
|
||||
|
||||
let mut writer = index.writer_with_num_threads(1, 32_000_000).unwrap();
|
||||
for _seg in 0..8 {
|
||||
for i in 0u64..1_000u64 {
|
||||
writer.add_document(doc!(field => i))?;
|
||||
}
|
||||
writer.commit()?;
|
||||
}
|
||||
|
||||
let mem_right_after_commit = directory.total_mem_usage();
|
||||
|
||||
let reader = index
|
||||
.reader_builder()
|
||||
.reload_policy(ReloadPolicy::Manual)
|
||||
.try_into()?;
|
||||
assert_eq!(reader.searcher().num_docs(), 8_000);
|
||||
assert_eq!(reader.searcher().segment_readers().len(), 8);
|
||||
|
||||
writer.wait_merging_threads()?;
|
||||
|
||||
let mem_right_after_merge_finished = directory.total_mem_usage();
|
||||
|
||||
reader.reload().unwrap();
|
||||
let searcher = reader.searcher();
|
||||
assert_eq!(searcher.segment_readers().len(), 1);
|
||||
assert_eq!(searcher.num_docs(), 8_000);
|
||||
assert!(
|
||||
mem_right_after_merge_finished < mem_right_after_commit,
|
||||
"(mem after merge){} is expected < (mem before merge){}",
|
||||
mem_right_after_merge_finished,
|
||||
mem_right_after_commit
|
||||
);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_single_segment_index_writer() -> crate::Result<()> {
|
||||
let mut schema_builder = Schema::builder();
|
||||
let text_field = schema_builder.add_text_field("text", TEXT);
|
||||
let schema = schema_builder.build();
|
||||
let directory = RamDirectory::default();
|
||||
let mut single_segment_index_writer = Index::builder()
|
||||
.schema(schema)
|
||||
.single_segment_index_writer(directory, 10_000_000)?;
|
||||
for _ in 0..10 {
|
||||
let doc = doc!(text_field=>"hello");
|
||||
single_segment_index_writer.add_document(doc)?;
|
||||
}
|
||||
let index = single_segment_index_writer.finalize()?;
|
||||
let searcher = index.reader()?.searcher();
|
||||
let term_query = TermQuery::new(
|
||||
Term::from_field_text(text_field, "hello"),
|
||||
IndexRecordOption::Basic,
|
||||
);
|
||||
let count = searcher.search(&term_query, &Count)?;
|
||||
assert_eq!(count, 10);
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,10 +1,11 @@
|
||||
use columnar::MonotonicallyMappableToU64;
|
||||
use common::replace_in_place;
|
||||
use murmurhash32::murmurhash2;
|
||||
use rustc_hash::FxHashMap;
|
||||
|
||||
use crate::fastfield::FastValue;
|
||||
use crate::postings::{IndexingContext, IndexingPosition, PostingsWriter};
|
||||
use crate::schema::term::{JSON_END_OF_PATH, JSON_PATH_SEGMENT_SEP};
|
||||
use crate::schema::term::{JSON_END_OF_PATH, JSON_PATH_SEGMENT_SEP, JSON_PATH_SEGMENT_SEP_STR};
|
||||
use crate::schema::{Field, Type};
|
||||
use crate::time::format_description::well_known::Rfc3339;
|
||||
use crate::time::{OffsetDateTime, UtcOffset};
|
||||
@@ -199,7 +200,7 @@ fn infer_type_from_str(text: &str) -> TextOrDateTime {
|
||||
}
|
||||
}
|
||||
|
||||
// Tries to infer a JSON type from a string
|
||||
// Tries to infer a JSON type from a string.
|
||||
pub(crate) fn convert_to_fast_value_and_get_term(
|
||||
json_term_writer: &mut JsonTermWriter,
|
||||
phrase: &str,
|
||||
@@ -295,6 +296,32 @@ fn split_json_path(json_path: &str) -> Vec<String> {
|
||||
json_path_segments
|
||||
}
|
||||
|
||||
/// Takes a field name, a json path as supplied by a user, and whether we should expand dots, and
|
||||
/// return a column key, as expected by the columnar crate.
|
||||
///
|
||||
/// This function will detect unescaped dots in the path, and split over them.
|
||||
/// If expand_dots is enabled, then even escaped dots will be split over.
|
||||
///
|
||||
/// The resulting list of segment then gets stitched together, joined by \1 separator,
|
||||
/// as defined in the columnar crate.
|
||||
pub(crate) fn encode_column_name(
|
||||
field_name: &str,
|
||||
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);
|
||||
}
|
||||
column_key
|
||||
}
|
||||
|
||||
impl<'a> JsonTermWriter<'a> {
|
||||
pub fn from_field_and_json_path(
|
||||
field: Field,
|
||||
@@ -343,18 +370,10 @@ impl<'a> JsonTermWriter<'a> {
|
||||
if self.path_stack.len() > 1 {
|
||||
buffer[buffer_len - 1] = JSON_PATH_SEGMENT_SEP;
|
||||
}
|
||||
if self.expand_dots_enabled && segment.as_bytes().contains(&b'.') {
|
||||
let appended_segment = self.term_buffer.append_bytes(segment.as_bytes());
|
||||
if self.expand_dots_enabled {
|
||||
// We need to replace `.` by JSON_PATH_SEGMENT_SEP.
|
||||
self.term_buffer
|
||||
.append_bytes(segment.as_bytes())
|
||||
.iter_mut()
|
||||
.for_each(|byte| {
|
||||
if *byte == b'.' {
|
||||
*byte = JSON_PATH_SEGMENT_SEP;
|
||||
}
|
||||
});
|
||||
} else {
|
||||
self.term_buffer.append_bytes(segment.as_bytes());
|
||||
replace_in_place(b'.', JSON_PATH_SEGMENT_SEP, appended_segment);
|
||||
}
|
||||
self.term_buffer.push_byte(JSON_PATH_SEGMENT_SEP);
|
||||
self.path_stack.push(self.term_buffer.len_bytes());
|
||||
@@ -373,7 +392,7 @@ impl<'a> JsonTermWriter<'a> {
|
||||
&self.term().value_bytes()[..end_of_path - 1]
|
||||
}
|
||||
|
||||
pub fn set_fast_value<T: FastValue>(&mut self, val: T) {
|
||||
pub(crate) fn set_fast_value<T: FastValue>(&mut self, val: T) {
|
||||
self.close_path_and_set_type(T::to_type());
|
||||
let value = if T::to_type() == Type::Date {
|
||||
DateTime::from_u64(val.to_u64())
|
||||
@@ -2,6 +2,7 @@ mod executor;
|
||||
pub mod index;
|
||||
mod index_meta;
|
||||
mod inverted_index_reader;
|
||||
pub mod json_utils;
|
||||
pub mod searcher;
|
||||
mod segment;
|
||||
mod segment_component;
|
||||
@@ -36,3 +37,6 @@ pub static META_FILEPATH: Lazy<&'static Path> = Lazy::new(|| Path::new("meta.jso
|
||||
/// Removing this file is safe, but will prevent the garbage collection of all of the file that
|
||||
/// are currently in the directory
|
||||
pub static MANAGED_FILEPATH: Lazy<&'static Path> = Lazy::new(|| Path::new(".managed.json"));
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests;
|
||||
|
||||
@@ -4,7 +4,7 @@ use std::{fmt, io};
|
||||
|
||||
use crate::collector::Collector;
|
||||
use crate::core::{Executor, SegmentReader};
|
||||
use crate::query::{EnableScoring, Query};
|
||||
use crate::query::{Bm25StatisticsProvider, EnableScoring, Query};
|
||||
use crate::schema::{Document, Schema, Term};
|
||||
use crate::space_usage::SearcherSpaceUsage;
|
||||
use crate::store::{CacheStats, StoreReader};
|
||||
@@ -176,8 +176,27 @@ impl Searcher {
|
||||
query: &dyn Query,
|
||||
collector: &C,
|
||||
) -> crate::Result<C::Fruit> {
|
||||
self.search_with_statistics_provider(query, collector, self)
|
||||
}
|
||||
|
||||
/// Same as [`search(...)`](Searcher::search) but allows specifying
|
||||
/// a [Bm25StatisticsProvider].
|
||||
///
|
||||
/// This can be used to adjust the statistics used in computing BM25
|
||||
/// scores.
|
||||
pub fn search_with_statistics_provider<C: Collector>(
|
||||
&self,
|
||||
query: &dyn Query,
|
||||
collector: &C,
|
||||
statistics_provider: &dyn Bm25StatisticsProvider,
|
||||
) -> crate::Result<C::Fruit> {
|
||||
let enabled_scoring = if collector.requires_scoring() {
|
||||
EnableScoring::enabled_from_statistics_provider(statistics_provider, self)
|
||||
} else {
|
||||
EnableScoring::disabled_from_searcher(self)
|
||||
};
|
||||
let executor = self.inner.index.search_executor();
|
||||
self.search_with_executor(query, collector, executor)
|
||||
self.search_with_executor(query, collector, executor, enabled_scoring)
|
||||
}
|
||||
|
||||
/// Same as [`search(...)`](Searcher::search) but multithreaded.
|
||||
@@ -197,12 +216,8 @@ impl Searcher {
|
||||
query: &dyn Query,
|
||||
collector: &C,
|
||||
executor: &Executor,
|
||||
enabled_scoring: EnableScoring,
|
||||
) -> crate::Result<C::Fruit> {
|
||||
let enabled_scoring = if collector.requires_scoring() {
|
||||
EnableScoring::enabled_from_searcher(self)
|
||||
} else {
|
||||
EnableScoring::disabled_from_searcher(self)
|
||||
};
|
||||
let weight = query.weight(enabled_scoring)?;
|
||||
let segment_readers = self.segment_readers();
|
||||
let fruits = executor.map(
|
||||
|
||||
@@ -38,7 +38,7 @@ pub struct SegmentReader {
|
||||
termdict_composite: CompositeFile,
|
||||
postings_composite: CompositeFile,
|
||||
positions_composite: CompositeFile,
|
||||
fast_fields_readers: Arc<FastFieldReaders>,
|
||||
fast_fields_readers: FastFieldReaders,
|
||||
fieldnorm_readers: FieldNormReaders,
|
||||
|
||||
store_file: FileSlice,
|
||||
@@ -167,7 +167,7 @@ impl SegmentReader {
|
||||
let schema = segment.schema();
|
||||
|
||||
let fast_fields_data = segment.open_read(SegmentComponent::FastFields)?;
|
||||
let fast_fields_readers = Arc::new(FastFieldReaders::open(fast_fields_data)?);
|
||||
let fast_fields_readers = FastFieldReaders::open(fast_fields_data, schema.clone())?;
|
||||
let fieldnorm_data = segment.open_read(SegmentComponent::FieldNorms)?;
|
||||
let fieldnorm_readers = FieldNormReaders::open(fieldnorm_data)?;
|
||||
|
||||
|
||||
347
src/core/tests.rs
Normal file
347
src/core/tests.rs
Normal file
@@ -0,0 +1,347 @@
|
||||
use crate::collector::Count;
|
||||
use crate::directory::{RamDirectory, WatchCallback};
|
||||
use crate::indexer::NoMergePolicy;
|
||||
use crate::query::TermQuery;
|
||||
use crate::schema::{Field, IndexRecordOption, Schema, INDEXED, STRING, TEXT};
|
||||
use crate::tokenizer::TokenizerManager;
|
||||
use crate::{
|
||||
Directory, Document, Index, IndexBuilder, IndexReader, IndexSettings, ReloadPolicy, SegmentId,
|
||||
Term,
|
||||
};
|
||||
|
||||
#[test]
|
||||
fn test_indexer_for_field() {
|
||||
let mut schema_builder = Schema::builder();
|
||||
let num_likes_field = schema_builder.add_u64_field("num_likes", INDEXED);
|
||||
let body_field = schema_builder.add_text_field("body", TEXT);
|
||||
let schema = schema_builder.build();
|
||||
let index = Index::create_in_ram(schema);
|
||||
assert!(index.tokenizer_for_field(body_field).is_ok());
|
||||
assert_eq!(
|
||||
format!("{:?}", index.tokenizer_for_field(num_likes_field).err()),
|
||||
"Some(SchemaError(\"\\\"num_likes\\\" is not a text field.\"))"
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_set_tokenizer_manager() {
|
||||
let mut schema_builder = Schema::builder();
|
||||
schema_builder.add_u64_field("num_likes", INDEXED);
|
||||
schema_builder.add_text_field("body", TEXT);
|
||||
let schema = schema_builder.build();
|
||||
let index = IndexBuilder::new()
|
||||
// set empty tokenizer manager
|
||||
.tokenizers(TokenizerManager::new())
|
||||
.schema(schema)
|
||||
.create_in_ram()
|
||||
.unwrap();
|
||||
assert!(index.tokenizers().get("raw").is_none());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_index_exists() {
|
||||
let directory: Box<dyn Directory> = Box::new(RamDirectory::create());
|
||||
assert!(!Index::exists(directory.as_ref()).unwrap());
|
||||
assert!(Index::create(
|
||||
directory.clone(),
|
||||
throw_away_schema(),
|
||||
IndexSettings::default()
|
||||
)
|
||||
.is_ok());
|
||||
assert!(Index::exists(directory.as_ref()).unwrap());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn open_or_create_should_create() {
|
||||
let directory = RamDirectory::create();
|
||||
assert!(!Index::exists(&directory).unwrap());
|
||||
assert!(Index::open_or_create(directory.clone(), throw_away_schema()).is_ok());
|
||||
assert!(Index::exists(&directory).unwrap());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn open_or_create_should_open() {
|
||||
let directory: Box<dyn Directory> = Box::new(RamDirectory::create());
|
||||
assert!(Index::create(
|
||||
directory.clone(),
|
||||
throw_away_schema(),
|
||||
IndexSettings::default()
|
||||
)
|
||||
.is_ok());
|
||||
assert!(Index::exists(directory.as_ref()).unwrap());
|
||||
assert!(Index::open_or_create(directory, throw_away_schema()).is_ok());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn create_should_wipeoff_existing() {
|
||||
let directory: Box<dyn Directory> = Box::new(RamDirectory::create());
|
||||
assert!(Index::create(
|
||||
directory.clone(),
|
||||
throw_away_schema(),
|
||||
IndexSettings::default()
|
||||
)
|
||||
.is_ok());
|
||||
assert!(Index::exists(directory.as_ref()).unwrap());
|
||||
assert!(Index::create(
|
||||
directory,
|
||||
Schema::builder().build(),
|
||||
IndexSettings::default()
|
||||
)
|
||||
.is_ok());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn open_or_create_exists_but_schema_does_not_match() {
|
||||
let directory = RamDirectory::create();
|
||||
assert!(Index::create(
|
||||
directory.clone(),
|
||||
throw_away_schema(),
|
||||
IndexSettings::default()
|
||||
)
|
||||
.is_ok());
|
||||
assert!(Index::exists(&directory).unwrap());
|
||||
assert!(Index::open_or_create(directory.clone(), throw_away_schema()).is_ok());
|
||||
let err = Index::open_or_create(directory, Schema::builder().build());
|
||||
assert_eq!(
|
||||
format!("{:?}", err.unwrap_err()),
|
||||
"SchemaError(\"An index exists but the schema does not match.\")"
|
||||
);
|
||||
}
|
||||
|
||||
fn throw_away_schema() -> Schema {
|
||||
let mut schema_builder = Schema::builder();
|
||||
let _ = schema_builder.add_u64_field("num_likes", INDEXED);
|
||||
schema_builder.build()
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_index_on_commit_reload_policy() -> crate::Result<()> {
|
||||
let schema = throw_away_schema();
|
||||
let field = schema.get_field("num_likes").unwrap();
|
||||
let index = Index::create_in_ram(schema);
|
||||
let reader = index
|
||||
.reader_builder()
|
||||
.reload_policy(ReloadPolicy::OnCommit)
|
||||
.try_into()
|
||||
.unwrap();
|
||||
assert_eq!(reader.searcher().num_docs(), 0);
|
||||
test_index_on_commit_reload_policy_aux(field, &index, &reader)
|
||||
}
|
||||
|
||||
#[cfg(feature = "mmap")]
|
||||
mod mmap_specific {
|
||||
|
||||
use std::path::PathBuf;
|
||||
|
||||
use tempfile::TempDir;
|
||||
|
||||
use super::*;
|
||||
use crate::Directory;
|
||||
|
||||
#[test]
|
||||
fn test_index_on_commit_reload_policy_mmap() -> crate::Result<()> {
|
||||
let schema = throw_away_schema();
|
||||
let field = schema.get_field("num_likes").unwrap();
|
||||
let tempdir = TempDir::new().unwrap();
|
||||
let tempdir_path = PathBuf::from(tempdir.path());
|
||||
let index = Index::create_in_dir(tempdir_path, schema).unwrap();
|
||||
let reader = index
|
||||
.reader_builder()
|
||||
.reload_policy(ReloadPolicy::OnCommit)
|
||||
.try_into()
|
||||
.unwrap();
|
||||
assert_eq!(reader.searcher().num_docs(), 0);
|
||||
test_index_on_commit_reload_policy_aux(field, &index, &reader)
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_index_manual_policy_mmap() -> crate::Result<()> {
|
||||
let schema = throw_away_schema();
|
||||
let field = schema.get_field("num_likes").unwrap();
|
||||
let mut index = Index::create_from_tempdir(schema)?;
|
||||
let mut writer = index.writer_for_tests()?;
|
||||
writer.commit()?;
|
||||
let reader = index
|
||||
.reader_builder()
|
||||
.reload_policy(ReloadPolicy::Manual)
|
||||
.try_into()?;
|
||||
assert_eq!(reader.searcher().num_docs(), 0);
|
||||
writer.add_document(doc!(field=>1u64))?;
|
||||
let (sender, receiver) = crossbeam_channel::unbounded();
|
||||
let _handle = index.directory_mut().watch(WatchCallback::new(move || {
|
||||
let _ = sender.send(());
|
||||
}));
|
||||
writer.commit()?;
|
||||
assert!(receiver.recv().is_ok());
|
||||
assert_eq!(reader.searcher().num_docs(), 0);
|
||||
reader.reload()?;
|
||||
assert_eq!(reader.searcher().num_docs(), 1);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_index_on_commit_reload_policy_different_directories() -> crate::Result<()> {
|
||||
let schema = throw_away_schema();
|
||||
let field = schema.get_field("num_likes").unwrap();
|
||||
let tempdir = TempDir::new().unwrap();
|
||||
let tempdir_path = PathBuf::from(tempdir.path());
|
||||
let write_index = Index::create_in_dir(&tempdir_path, schema).unwrap();
|
||||
let read_index = Index::open_in_dir(&tempdir_path).unwrap();
|
||||
let reader = read_index
|
||||
.reader_builder()
|
||||
.reload_policy(ReloadPolicy::OnCommit)
|
||||
.try_into()
|
||||
.unwrap();
|
||||
assert_eq!(reader.searcher().num_docs(), 0);
|
||||
test_index_on_commit_reload_policy_aux(field, &write_index, &reader)
|
||||
}
|
||||
}
|
||||
fn test_index_on_commit_reload_policy_aux(
|
||||
field: Field,
|
||||
index: &Index,
|
||||
reader: &IndexReader,
|
||||
) -> crate::Result<()> {
|
||||
let mut reader_index = reader.index();
|
||||
let (sender, receiver) = crossbeam_channel::unbounded();
|
||||
let _watch_handle = reader_index
|
||||
.directory_mut()
|
||||
.watch(WatchCallback::new(move || {
|
||||
let _ = sender.send(());
|
||||
}));
|
||||
let mut writer = index.writer_for_tests()?;
|
||||
assert_eq!(reader.searcher().num_docs(), 0);
|
||||
writer.add_document(doc!(field=>1u64))?;
|
||||
writer.commit().unwrap();
|
||||
// We need a loop here because it is possible for notify to send more than
|
||||
// one modify event. It was observed on CI on MacOS.
|
||||
loop {
|
||||
assert!(receiver.recv().is_ok());
|
||||
if reader.searcher().num_docs() == 1 {
|
||||
break;
|
||||
}
|
||||
}
|
||||
writer.add_document(doc!(field=>2u64))?;
|
||||
writer.commit().unwrap();
|
||||
// ... Same as above
|
||||
loop {
|
||||
assert!(receiver.recv().is_ok());
|
||||
if reader.searcher().num_docs() == 2 {
|
||||
break;
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
// This test will not pass on windows, because windows
|
||||
// prevent deleting files that are MMapped.
|
||||
#[cfg(not(target_os = "windows"))]
|
||||
#[test]
|
||||
fn garbage_collect_works_as_intended() -> crate::Result<()> {
|
||||
let directory = RamDirectory::create();
|
||||
let schema = throw_away_schema();
|
||||
let field = schema.get_field("num_likes").unwrap();
|
||||
let index = Index::create(directory.clone(), schema, IndexSettings::default())?;
|
||||
|
||||
let mut writer = index.writer_with_num_threads(1, 32_000_000).unwrap();
|
||||
for _seg in 0..8 {
|
||||
for i in 0u64..1_000u64 {
|
||||
writer.add_document(doc!(field => i))?;
|
||||
}
|
||||
writer.commit()?;
|
||||
}
|
||||
|
||||
let mem_right_after_commit = directory.total_mem_usage();
|
||||
|
||||
let reader = index
|
||||
.reader_builder()
|
||||
.reload_policy(ReloadPolicy::Manual)
|
||||
.try_into()?;
|
||||
assert_eq!(reader.searcher().num_docs(), 8_000);
|
||||
assert_eq!(reader.searcher().segment_readers().len(), 8);
|
||||
|
||||
writer.wait_merging_threads()?;
|
||||
|
||||
let mem_right_after_merge_finished = directory.total_mem_usage();
|
||||
|
||||
reader.reload().unwrap();
|
||||
let searcher = reader.searcher();
|
||||
assert_eq!(searcher.segment_readers().len(), 1);
|
||||
assert_eq!(searcher.num_docs(), 8_000);
|
||||
assert!(
|
||||
mem_right_after_merge_finished < mem_right_after_commit,
|
||||
"(mem after merge){} is expected < (mem before merge){}",
|
||||
mem_right_after_merge_finished,
|
||||
mem_right_after_commit
|
||||
);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_single_segment_index_writer() -> crate::Result<()> {
|
||||
let mut schema_builder = Schema::builder();
|
||||
let text_field = schema_builder.add_text_field("text", TEXT);
|
||||
let schema = schema_builder.build();
|
||||
let directory = RamDirectory::default();
|
||||
let mut single_segment_index_writer = Index::builder()
|
||||
.schema(schema)
|
||||
.single_segment_index_writer(directory, 10_000_000)?;
|
||||
for _ in 0..10 {
|
||||
let doc = doc!(text_field=>"hello");
|
||||
single_segment_index_writer.add_document(doc)?;
|
||||
}
|
||||
let index = single_segment_index_writer.finalize()?;
|
||||
let searcher = index.reader()?.searcher();
|
||||
let term_query = TermQuery::new(
|
||||
Term::from_field_text(text_field, "hello"),
|
||||
IndexRecordOption::Basic,
|
||||
);
|
||||
let count = searcher.search(&term_query, &Count)?;
|
||||
assert_eq!(count, 10);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_merging_segment_update_docfreq() {
|
||||
let mut schema_builder = Schema::builder();
|
||||
let text_field = schema_builder.add_text_field("text", TEXT);
|
||||
let id_field = schema_builder.add_text_field("id", STRING);
|
||||
let schema = schema_builder.build();
|
||||
let index = Index::create_in_ram(schema);
|
||||
let mut writer = index.writer_for_tests().unwrap();
|
||||
writer.set_merge_policy(Box::new(NoMergePolicy));
|
||||
for _ in 0..5 {
|
||||
writer.add_document(doc!(text_field=>"hello")).unwrap();
|
||||
}
|
||||
writer
|
||||
.add_document(doc!(text_field=>"hello", id_field=>"TO_BE_DELETED"))
|
||||
.unwrap();
|
||||
writer
|
||||
.add_document(doc!(text_field=>"hello", id_field=>"TO_BE_DELETED"))
|
||||
.unwrap();
|
||||
writer.add_document(Document::default()).unwrap();
|
||||
writer.commit().unwrap();
|
||||
for _ in 0..7 {
|
||||
writer.add_document(doc!(text_field=>"hello")).unwrap();
|
||||
}
|
||||
writer.add_document(Document::default()).unwrap();
|
||||
writer.add_document(Document::default()).unwrap();
|
||||
writer.delete_term(Term::from_field_text(id_field, "TO_BE_DELETED"));
|
||||
writer.commit().unwrap();
|
||||
|
||||
let segment_ids: Vec<SegmentId> = index
|
||||
.list_all_segment_metas()
|
||||
.into_iter()
|
||||
.map(|reader| reader.id())
|
||||
.collect();
|
||||
writer.merge(&segment_ids[..]).wait().unwrap();
|
||||
let index_reader = index.reader().unwrap();
|
||||
let searcher = index_reader.searcher();
|
||||
assert_eq!(searcher.segment_readers().len(), 1);
|
||||
assert_eq!(searcher.num_docs(), 15);
|
||||
let segment_reader = searcher.segment_reader(0);
|
||||
assert_eq!(segment_reader.max_doc(), 15);
|
||||
let inv_index = segment_reader.inverted_index(text_field).unwrap();
|
||||
let term = Term::from_field_text(text_field, "hello");
|
||||
let term_info = inv_index.get_term_info(&term).unwrap().unwrap();
|
||||
assert_eq!(term_info.doc_freq, 12);
|
||||
}
|
||||
@@ -6,6 +6,7 @@ use std::{fmt, io};
|
||||
|
||||
use thiserror::Error;
|
||||
|
||||
use crate::aggregation::AggregationError;
|
||||
use crate::directory::error::{
|
||||
Incompatibility, LockError, OpenDirectoryError, OpenReadError, OpenWriteError,
|
||||
};
|
||||
@@ -53,6 +54,9 @@ impl fmt::Debug for DataCorruption {
|
||||
/// The library's error enum
|
||||
#[derive(Debug, Clone, Error)]
|
||||
pub enum TantivyError {
|
||||
/// Error when handling aggregations.
|
||||
#[error("AggregationError {0:?}")]
|
||||
AggregationError(#[from] AggregationError),
|
||||
/// Failed to open the directory.
|
||||
#[error("Failed to open the directory: '{0:?}'")]
|
||||
OpenDirectoryError(#[from] OpenDirectoryError),
|
||||
|
||||
@@ -51,7 +51,7 @@ impl FacetReader {
|
||||
|
||||
/// Return the list of facet ordinals associated with a document.
|
||||
pub fn facet_ords(&self, doc: DocId) -> impl Iterator<Item = u64> + '_ {
|
||||
self.facet_column.ords().values(doc)
|
||||
self.facet_column.ords().values_for_doc(doc)
|
||||
}
|
||||
|
||||
/// Accessor to the facet dictionary.
|
||||
|
||||
@@ -19,9 +19,8 @@
|
||||
//!
|
||||
//! Read access performance is comparable to that of an array lookup.
|
||||
|
||||
use std::net::Ipv6Addr;
|
||||
|
||||
pub use columnar::Column;
|
||||
use columnar::MonotonicallyMappableToU64;
|
||||
|
||||
pub use self::alive_bitset::{intersect_alive_bitsets, write_alive_bitset, AliveBitSet};
|
||||
pub use self::error::{FastFieldNotAvailableError, Result};
|
||||
@@ -37,38 +36,9 @@ mod facet_reader;
|
||||
mod readers;
|
||||
mod writer;
|
||||
|
||||
/// Trait for types that provide a zero value.
|
||||
///
|
||||
/// The resulting value is never used, just as placeholder, e.g. for `vec.resize()`.
|
||||
pub trait MakeZero {
|
||||
/// Build a default value. This default value is never used, so the value does not
|
||||
/// really matter.
|
||||
fn make_zero() -> Self;
|
||||
}
|
||||
|
||||
impl<T: FastValue> MakeZero for T {
|
||||
fn make_zero() -> Self {
|
||||
T::from_u64(0)
|
||||
}
|
||||
}
|
||||
|
||||
impl MakeZero for u128 {
|
||||
fn make_zero() -> Self {
|
||||
0
|
||||
}
|
||||
}
|
||||
|
||||
impl MakeZero for Ipv6Addr {
|
||||
fn make_zero() -> Self {
|
||||
Ipv6Addr::from(0u128.to_be_bytes())
|
||||
}
|
||||
}
|
||||
|
||||
/// Trait for types that are allowed for fast fields:
|
||||
/// (u64, i64 and f64, bool, DateTime).
|
||||
pub trait FastValue:
|
||||
Copy + Send + Sync + columnar::MonotonicallyMappableToU64 + PartialOrd + 'static
|
||||
{
|
||||
pub trait FastValue: MonotonicallyMappableToU64 {
|
||||
/// Returns the `schema::Type` for this FastValue.
|
||||
fn to_type() -> Type;
|
||||
}
|
||||
@@ -105,10 +75,11 @@ impl FastValue for DateTime {
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
|
||||
use std::net::Ipv6Addr;
|
||||
use std::ops::{Range, RangeInclusive};
|
||||
use std::path::Path;
|
||||
|
||||
use columnar::{Column, MonotonicallyMappableToU64};
|
||||
use columnar::{Column, MonotonicallyMappableToU64, StrColumn};
|
||||
use common::{HasLen, TerminatingWrite};
|
||||
use once_cell::sync::Lazy;
|
||||
use rand::prelude::SliceRandom;
|
||||
@@ -119,7 +90,8 @@ mod tests {
|
||||
use crate::directory::{Directory, RamDirectory, WritePtr};
|
||||
use crate::merge_policy::NoMergePolicy;
|
||||
use crate::schema::{
|
||||
Document, Facet, FacetOptions, Field, Schema, SchemaBuilder, FAST, INDEXED, STRING, TEXT,
|
||||
Document, Facet, FacetOptions, Field, JsonObjectOptions, Schema, SchemaBuilder, FAST,
|
||||
INDEXED, STORED, STRING, TEXT,
|
||||
};
|
||||
use crate::time::OffsetDateTime;
|
||||
use crate::{DateOptions, DatePrecision, Index, SegmentId, SegmentReader};
|
||||
@@ -159,7 +131,7 @@ mod tests {
|
||||
let file = directory.open_read(path).unwrap();
|
||||
|
||||
assert_eq!(file.len(), 161);
|
||||
let fast_field_readers = FastFieldReaders::open(file).unwrap();
|
||||
let fast_field_readers = FastFieldReaders::open(file, SCHEMA.clone()).unwrap();
|
||||
let column = fast_field_readers
|
||||
.u64("field")
|
||||
.unwrap()
|
||||
@@ -209,7 +181,7 @@ mod tests {
|
||||
}
|
||||
let file = directory.open_read(path).unwrap();
|
||||
assert_eq!(file.len(), 189);
|
||||
let fast_field_readers = FastFieldReaders::open(file).unwrap();
|
||||
let fast_field_readers = FastFieldReaders::open(file, SCHEMA.clone()).unwrap();
|
||||
let col = fast_field_readers
|
||||
.u64("field")
|
||||
.unwrap()
|
||||
@@ -242,7 +214,7 @@ mod tests {
|
||||
}
|
||||
let file = directory.open_read(path).unwrap();
|
||||
assert_eq!(file.len(), 162);
|
||||
let fast_field_readers = FastFieldReaders::open(file).unwrap();
|
||||
let fast_field_readers = FastFieldReaders::open(file, SCHEMA.clone()).unwrap();
|
||||
let fast_field_reader = fast_field_readers
|
||||
.u64("field")
|
||||
.unwrap()
|
||||
@@ -266,7 +238,7 @@ mod tests {
|
||||
.unwrap();
|
||||
for doc_id in 1u64..10_000u64 {
|
||||
fast_field_writers
|
||||
.add_document(&doc!(*FIELD=>5_000_000_000_000_000_000u64 + doc_id as u64))
|
||||
.add_document(&doc!(*FIELD=>5_000_000_000_000_000_000u64 + doc_id))
|
||||
.unwrap();
|
||||
}
|
||||
fast_field_writers.serialize(&mut write, None).unwrap();
|
||||
@@ -275,7 +247,7 @@ mod tests {
|
||||
let file = directory.open_read(path).unwrap();
|
||||
assert_eq!(file.len(), 4557);
|
||||
{
|
||||
let fast_field_readers = FastFieldReaders::open(file).unwrap();
|
||||
let fast_field_readers = FastFieldReaders::open(file, SCHEMA.clone()).unwrap();
|
||||
let col = fast_field_readers
|
||||
.u64("field")
|
||||
.unwrap()
|
||||
@@ -309,7 +281,7 @@ mod tests {
|
||||
assert_eq!(file.len(), 333_usize);
|
||||
|
||||
{
|
||||
let fast_field_readers = FastFieldReaders::open(file).unwrap();
|
||||
let fast_field_readers = FastFieldReaders::open(file, schema).unwrap();
|
||||
let col = fast_field_readers
|
||||
.i64("field")
|
||||
.unwrap()
|
||||
@@ -346,7 +318,7 @@ mod tests {
|
||||
}
|
||||
|
||||
let file = directory.open_read(path).unwrap();
|
||||
let fast_field_readers = FastFieldReaders::open(file).unwrap();
|
||||
let fast_field_readers = FastFieldReaders::open(file, schema).unwrap();
|
||||
let col = fast_field_readers.i64("field").unwrap();
|
||||
assert_eq!(col.first(0), None);
|
||||
|
||||
@@ -379,7 +351,7 @@ mod tests {
|
||||
}
|
||||
|
||||
let file = directory.open_read(path).unwrap();
|
||||
let fast_field_readers = FastFieldReaders::open(file).unwrap();
|
||||
let fast_field_readers = FastFieldReaders::open(file, schema).unwrap();
|
||||
let col = fast_field_readers
|
||||
.date("date")
|
||||
.unwrap()
|
||||
@@ -415,7 +387,7 @@ mod tests {
|
||||
write.terminate().unwrap();
|
||||
}
|
||||
let file = directory.open_read(path).unwrap();
|
||||
let fast_field_readers = FastFieldReaders::open(file).unwrap();
|
||||
let fast_field_readers = FastFieldReaders::open(file, SCHEMA.clone()).unwrap();
|
||||
let col = fast_field_readers
|
||||
.u64("field")
|
||||
.unwrap()
|
||||
@@ -466,7 +438,7 @@ mod tests {
|
||||
|
||||
fn get_vals_for_docs(column: &Column<u64>, docs: Range<u32>) -> Vec<u64> {
|
||||
docs.into_iter()
|
||||
.flat_map(|doc| column.values(doc))
|
||||
.flat_map(|doc| column.values_for_doc(doc))
|
||||
.collect()
|
||||
}
|
||||
|
||||
@@ -514,11 +486,11 @@ mod tests {
|
||||
let segment_reader = searcher.segment_reader(0);
|
||||
let fast_fields = segment_reader.fast_fields();
|
||||
let str_column = fast_fields.str("text").unwrap().unwrap();
|
||||
assert!(str_column.ords().values(0u32).eq([1, 0]),);
|
||||
assert!(str_column.ords().values(1u32).next().is_none());
|
||||
assert!(str_column.ords().values(2u32).eq([0]),);
|
||||
assert!(str_column.ords().values(3u32).eq([0, 1]),);
|
||||
assert!(str_column.ords().values(4u32).eq([2]),);
|
||||
assert!(str_column.ords().values_for_doc(0u32).eq([1, 0]),);
|
||||
assert!(str_column.ords().values_for_doc(1u32).next().is_none());
|
||||
assert!(str_column.ords().values_for_doc(2u32).eq([0]),);
|
||||
assert!(str_column.ords().values_for_doc(3u32).eq([0, 1]),);
|
||||
assert!(str_column.ords().values_for_doc(4u32).eq([2]),);
|
||||
|
||||
let mut str_term = String::default();
|
||||
assert!(str_column.ord_to_str(0, &mut str_term).unwrap());
|
||||
@@ -557,7 +529,7 @@ mod tests {
|
||||
let fast_fields = segment_reader.fast_fields();
|
||||
let text_fast_field = fast_fields.str("text").unwrap().unwrap();
|
||||
|
||||
assert_eq!(&get_vals_for_docs(&text_fast_field.ords(), 0..2), &[0, 1]);
|
||||
assert_eq!(&get_vals_for_docs(text_fast_field.ords(), 0..2), &[0, 1]);
|
||||
}
|
||||
|
||||
// TODO uncomment once merging is available
|
||||
@@ -653,7 +625,7 @@ mod tests {
|
||||
let fast_fields = segment_reader.fast_fields();
|
||||
let text_col = fast_fields.str("text").unwrap().unwrap();
|
||||
|
||||
assert_eq!(get_vals_for_docs(&text_col.ords(), 0..6), vec![1, 0, 0, 2]);
|
||||
assert_eq!(get_vals_for_docs(text_col.ords(), 0..6), vec![1, 0, 0, 2]);
|
||||
|
||||
let inverted_index = segment_reader.inverted_index(text_field)?;
|
||||
assert_eq!(inverted_index.terms().num_terms(), 3);
|
||||
@@ -701,7 +673,7 @@ mod tests {
|
||||
let text_fast_field = fast_fields.str("text").unwrap().unwrap();
|
||||
|
||||
assert_eq!(
|
||||
get_vals_for_docs(&text_fast_field.ords(), 0..9),
|
||||
get_vals_for_docs(text_fast_field.ords(), 0..9),
|
||||
vec![1, 0, 0, 3 /* next segment */, 0, 2]
|
||||
);
|
||||
|
||||
@@ -801,7 +773,7 @@ mod tests {
|
||||
}
|
||||
let file = directory.open_read(path).unwrap();
|
||||
assert_eq!(file.len(), 175);
|
||||
let fast_field_readers = FastFieldReaders::open(file).unwrap();
|
||||
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));
|
||||
assert_eq!(bool_col.first(1), Some(false));
|
||||
@@ -833,7 +805,7 @@ mod tests {
|
||||
}
|
||||
let file = directory.open_read(path).unwrap();
|
||||
assert_eq!(file.len(), 187);
|
||||
let readers = FastFieldReaders::open(file).unwrap();
|
||||
let readers = FastFieldReaders::open(file, schema).unwrap();
|
||||
let bool_col = readers.bool("field_bool").unwrap();
|
||||
for i in 0..25 {
|
||||
assert_eq!(bool_col.first(i * 2), Some(true));
|
||||
@@ -858,7 +830,7 @@ mod tests {
|
||||
}
|
||||
let file = directory.open_read(path).unwrap();
|
||||
assert_eq!(file.len(), 177);
|
||||
let fastfield_readers = FastFieldReaders::open(file).unwrap();
|
||||
let fastfield_readers = FastFieldReaders::open(file, schema).unwrap();
|
||||
let col = fastfield_readers.bool("field_bool").unwrap();
|
||||
assert_eq!(col.first(0), None);
|
||||
let col = fastfield_readers
|
||||
@@ -920,11 +892,11 @@ mod tests {
|
||||
let directory = get_index(&docs[..], &schema).unwrap();
|
||||
let path = Path::new("test");
|
||||
let file = directory.open_read(path).unwrap();
|
||||
let readers = FastFieldReaders::open(file).unwrap();
|
||||
let readers = FastFieldReaders::open(file, schema).unwrap();
|
||||
let col = readers.date("field").unwrap();
|
||||
|
||||
for (i, time) in times.iter().enumerate() {
|
||||
let dt: DateTime = col.first(i as u32).unwrap().into();
|
||||
let dt: DateTime = col.first(i as u32).unwrap();
|
||||
assert_eq!(dt, time.truncate(precision));
|
||||
}
|
||||
readers.column_num_bytes("field").unwrap()
|
||||
@@ -1071,4 +1043,158 @@ mod tests {
|
||||
test_range_variant(1000, 1000);
|
||||
test_range_variant(1000, 1002);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_json_object_fast_field() {
|
||||
let mut schema_builder = Schema::builder();
|
||||
let without_fast_field = schema_builder.add_json_field("without", STORED);
|
||||
let with_fast_field = schema_builder.add_json_field("with", STORED | FAST);
|
||||
let schema = schema_builder.build();
|
||||
let index = Index::create_in_ram(schema);
|
||||
let mut writer = index.writer_for_tests().unwrap();
|
||||
writer
|
||||
.add_document(doc!(without_fast_field=>json!({"hello": "without"})))
|
||||
.unwrap();
|
||||
writer
|
||||
.add_document(doc!(with_fast_field=>json!({"hello": "with"})))
|
||||
.unwrap();
|
||||
writer
|
||||
.add_document(doc!(with_fast_field=>json!({"hello": "with2"})))
|
||||
.unwrap();
|
||||
writer
|
||||
.add_document(doc!(with_fast_field=>json!({"hello": "with1"})))
|
||||
.unwrap();
|
||||
writer.commit().unwrap();
|
||||
let searcher = index.reader().unwrap().searcher();
|
||||
let segment_reader = searcher.segment_reader(0u32);
|
||||
let fast_fields = segment_reader.fast_fields();
|
||||
let column_without_opt: Option<StrColumn> = fast_fields.str("without.hello").unwrap();
|
||||
assert!(column_without_opt.is_none());
|
||||
let column_with_opt: Option<StrColumn> = fast_fields.str("with.hello").unwrap();
|
||||
let column_with: StrColumn = column_with_opt.unwrap();
|
||||
assert!(column_with.term_ords(0).next().is_none());
|
||||
assert!(column_with.term_ords(1).eq([0]));
|
||||
assert!(column_with.term_ords(2).eq([2]));
|
||||
assert!(column_with.term_ords(3).eq([1]));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_fast_field_in_json_field_expand_dots_disabled() {
|
||||
let mut schema_builder = Schema::builder();
|
||||
let json_option = JsonObjectOptions::default().set_fast();
|
||||
let json = schema_builder.add_json_field("json", json_option);
|
||||
let schema = schema_builder.build();
|
||||
let index = Index::create_in_ram(schema);
|
||||
let mut index_writer = index.writer_for_tests().unwrap();
|
||||
index_writer
|
||||
.add_document(doc!(json => json!({"attr.age": 32})))
|
||||
.unwrap();
|
||||
index_writer.commit().unwrap();
|
||||
let searcher = index.reader().unwrap().searcher();
|
||||
let fast_field_reader = searcher.segment_reader(0u32).fast_fields();
|
||||
assert!(fast_field_reader
|
||||
.column_opt::<i64>("json.attr.age")
|
||||
.unwrap()
|
||||
.is_none());
|
||||
let column = fast_field_reader
|
||||
.column_opt::<i64>(r#"json.attr\.age"#)
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
let vals: Vec<i64> = column.values_for_doc(0u32).collect();
|
||||
assert_eq!(&vals, &[32])
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_fast_field_in_json_field_expand_dots_enabled() {
|
||||
let mut schema_builder = Schema::builder();
|
||||
let json_option = JsonObjectOptions::default()
|
||||
.set_fast()
|
||||
.set_expand_dots_enabled();
|
||||
let json = schema_builder.add_json_field("json", json_option);
|
||||
let schema = schema_builder.build();
|
||||
let index = Index::create_in_ram(schema);
|
||||
let mut index_writer = index.writer_for_tests().unwrap();
|
||||
index_writer
|
||||
.add_document(doc!(json => json!({"attr.age": 32})))
|
||||
.unwrap();
|
||||
index_writer.commit().unwrap();
|
||||
let searcher = index.reader().unwrap().searcher();
|
||||
let fast_field_reader = searcher.segment_reader(0u32).fast_fields();
|
||||
for test_column_name in &["json.attr.age", "json.attr\\.age"] {
|
||||
let column = fast_field_reader
|
||||
.column_opt::<i64>(test_column_name)
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
let vals: Vec<i64> = column.values_for_doc(0u32).collect();
|
||||
assert_eq!(&vals, &[32]);
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_fast_field_dot_in_schema_field_name() {
|
||||
let mut schema_builder = Schema::builder();
|
||||
let field_with_dot = schema_builder.add_i64_field("field.with.dot", FAST);
|
||||
let schema = schema_builder.build();
|
||||
let index = Index::create_in_ram(schema);
|
||||
let mut index_writer = index.writer_for_tests().unwrap();
|
||||
index_writer
|
||||
.add_document(doc!(field_with_dot => 32i64))
|
||||
.unwrap();
|
||||
index_writer.commit().unwrap();
|
||||
let searcher = index.reader().unwrap().searcher();
|
||||
let fast_field_reader = searcher.segment_reader(0u32).fast_fields();
|
||||
let column = fast_field_reader
|
||||
.column_opt::<i64>("field.with.dot")
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
let vals: Vec<i64> = column.values_for_doc(0u32).collect();
|
||||
assert_eq!(&vals, &[32]);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_shadowing_fast_field() {
|
||||
let mut schema_builder = Schema::builder();
|
||||
let json_field = schema_builder.add_json_field("jsonfield", FAST);
|
||||
let shadowing_json_field = schema_builder.add_json_field("jsonfield.attr", FAST);
|
||||
let schema = schema_builder.build();
|
||||
let index = Index::create_in_ram(schema);
|
||||
let mut index_writer = index.writer_for_tests().unwrap();
|
||||
index_writer
|
||||
.add_document(doc!(json_field=> json!({"attr": {"age": 32}}), shadowing_json_field=>json!({"age": 33})))
|
||||
.unwrap();
|
||||
index_writer.commit().unwrap();
|
||||
let searcher = index.reader().unwrap().searcher();
|
||||
let fast_field_reader = searcher.segment_reader(0u32).fast_fields();
|
||||
let column = fast_field_reader
|
||||
.column_opt::<i64>(&"jsonfield.attr.age")
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
let vals: Vec<i64> = column.values_for_doc(0u32).collect();
|
||||
assert_eq!(&vals, &[33]);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_shadowing_fast_field_with_expand_dots() {
|
||||
let mut schema_builder = Schema::builder();
|
||||
let json_option = JsonObjectOptions::default()
|
||||
.set_fast()
|
||||
.set_expand_dots_enabled();
|
||||
let json_field = schema_builder.add_json_field("jsonfield", json_option.clone());
|
||||
let shadowing_json_field = schema_builder.add_json_field("jsonfield.attr", json_option);
|
||||
let schema = schema_builder.build();
|
||||
let index = Index::create_in_ram(schema);
|
||||
let mut index_writer = index.writer_for_tests().unwrap();
|
||||
index_writer
|
||||
.add_document(doc!(json_field=> json!({"attr.age": 32}), shadowing_json_field=>json!({"age": 33})))
|
||||
.unwrap();
|
||||
index_writer.commit().unwrap();
|
||||
let searcher = index.reader().unwrap().searcher();
|
||||
let fast_field_reader = searcher.segment_reader(0u32).fast_fields();
|
||||
let column = fast_field_reader
|
||||
.column_opt::<i64>(&"jsonfield.attr.age")
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
let vals: Vec<i64> = column.values_for_doc(0u32).collect();
|
||||
assert_eq!(&vals, &[33]);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -7,8 +7,9 @@ use columnar::{
|
||||
DynamicColumnHandle, HasAssociatedColumnType, StrColumn,
|
||||
};
|
||||
|
||||
use crate::core::json_utils::encode_column_name;
|
||||
use crate::directory::FileSlice;
|
||||
use crate::schema::Schema;
|
||||
use crate::schema::{Field, FieldEntry, FieldType, Schema};
|
||||
use crate::space_usage::{FieldUsage, PerFieldSpaceUsage};
|
||||
|
||||
/// Provides access to all of the BitpackedFastFieldReader.
|
||||
@@ -18,16 +19,22 @@ use crate::space_usage::{FieldUsage, PerFieldSpaceUsage};
|
||||
#[derive(Clone)]
|
||||
pub struct FastFieldReaders {
|
||||
columnar: Arc<ColumnarReader>,
|
||||
schema: Schema,
|
||||
}
|
||||
|
||||
impl FastFieldReaders {
|
||||
pub(crate) fn open(fast_field_file: FileSlice) -> io::Result<FastFieldReaders> {
|
||||
pub(crate) fn open(fast_field_file: FileSlice, schema: Schema) -> io::Result<FastFieldReaders> {
|
||||
let columnar = Arc::new(ColumnarReader::open(fast_field_file)?);
|
||||
Ok(FastFieldReaders { columnar })
|
||||
Ok(FastFieldReaders { columnar, schema })
|
||||
}
|
||||
|
||||
pub(crate) fn columnar(&self) -> &ColumnarReader {
|
||||
self.columnar.as_ref()
|
||||
fn resolve_field(&self, column_name: &str) -> Option<String> {
|
||||
let default_field_opt: Option<Field> = if cfg!(feature = "quickwit") {
|
||||
self.schema.get_field("_dynamic").ok()
|
||||
} else {
|
||||
None
|
||||
};
|
||||
self.resolve_column_name_given_default_field(column_name, default_field_opt)
|
||||
}
|
||||
|
||||
pub(crate) fn space_usage(&self, schema: &Schema) -> io::Result<PerFieldSpaceUsage> {
|
||||
@@ -46,6 +53,59 @@ impl FastFieldReaders {
|
||||
Ok(PerFieldSpaceUsage::new(per_field_usages))
|
||||
}
|
||||
|
||||
pub(crate) fn columnar(&self) -> &ColumnarReader {
|
||||
self.columnar.as_ref()
|
||||
}
|
||||
|
||||
/// Transforms a user-supplied fast field name into a column name.
|
||||
///
|
||||
/// A user-supplied fast field name is not necessarily a schema field name
|
||||
/// because we handle fast fields.
|
||||
///
|
||||
/// For instance, if the documents look like `{.., "attributes": {"color": "red"}}` and
|
||||
/// `attributes` is a json fast field, a user could want to run a term aggregation over
|
||||
/// colors, by referring to the field as `attributes.color`.
|
||||
///
|
||||
/// This function transforms `attributes.color` into a column key to be used in the `columnar`.
|
||||
///
|
||||
/// The logic works as follows, first we identify which field is targetted by calling
|
||||
/// `schema.find_field(..)`. This method will attempt to split the user splied fast field
|
||||
/// name by non-escaped dots, and find the longest matching schema field name.
|
||||
/// In our case, it would return the (attribute_field, "color").
|
||||
///
|
||||
/// If no field is found, but a dynamic field is supplied, then we
|
||||
/// will simply assuem the user is targetting the dynamic field. (This feature is used in
|
||||
/// Quickwit.)
|
||||
///
|
||||
/// We then encode the `(field, path)` into the right `columnar_key`.
|
||||
fn resolve_column_name_given_default_field<'a>(
|
||||
&'a self,
|
||||
field_name: &'a str,
|
||||
default_field_opt: Option<Field>,
|
||||
) -> Option<String> {
|
||||
let (field, path): (Field, &str) = self
|
||||
.schema
|
||||
.find_field(field_name)
|
||||
.or_else(|| default_field_opt.map(|default_field| (default_field, field_name)))?;
|
||||
let field_name = self.schema.get_field_name(field);
|
||||
if path.is_empty() {
|
||||
return Some(field_name.to_string());
|
||||
}
|
||||
let field_entry: &FieldEntry = self.schema.get_field_entry(field);
|
||||
let field_type = field_entry.field_type();
|
||||
match (field_type, path) {
|
||||
(FieldType::JsonObject(json_options), path) if !path.is_empty() => {
|
||||
Some(encode_column_name(
|
||||
field_entry.name(),
|
||||
path,
|
||||
json_options.is_expand_dots_enabled(),
|
||||
))
|
||||
}
|
||||
(_, "") => Some(field_entry.name().to_string()),
|
||||
_ => None,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns a typed column associated to a given field name.
|
||||
///
|
||||
/// If no column associated with that field_name exists,
|
||||
@@ -53,11 +113,10 @@ impl FastFieldReaders {
|
||||
/// returns `None`.
|
||||
pub fn column_opt<T>(&self, field_name: &str) -> crate::Result<Option<Column<T>>>
|
||||
where
|
||||
T: PartialOrd + Copy + HasAssociatedColumnType + Send + Sync + 'static,
|
||||
T: HasAssociatedColumnType,
|
||||
DynamicColumn: Into<Option<Column<T>>>,
|
||||
{
|
||||
let column_type = T::column_type();
|
||||
let Some(dynamic_column_handle) = self.dynamic_column_handle(field_name, column_type)?
|
||||
let Some(dynamic_column_handle) = self.dynamic_column_handle(field_name, T::column_type())?
|
||||
else {
|
||||
return Ok(None);
|
||||
};
|
||||
@@ -66,10 +125,15 @@ impl FastFieldReaders {
|
||||
}
|
||||
|
||||
/// Returns the number of `bytes` associated with a column.
|
||||
///
|
||||
/// Returns 0 if the column does not exist.
|
||||
pub fn column_num_bytes(&self, field: &str) -> crate::Result<usize> {
|
||||
let Some(resolved_field_name) = self.resolve_field(field) else {
|
||||
return Ok(0);
|
||||
};
|
||||
Ok(self
|
||||
.columnar
|
||||
.read_columns(field)?
|
||||
.read_columns(&resolved_field_name)?
|
||||
.into_iter()
|
||||
.map(|column_handle| column_handle.num_bytes())
|
||||
.sum())
|
||||
@@ -92,7 +156,7 @@ impl FastFieldReaders {
|
||||
/// Returns a typed column associated to a given field name.
|
||||
///
|
||||
/// Returns an error if no column associated with that field_name exists.
|
||||
pub fn column<T>(&self, field: &str) -> crate::Result<Column<T>>
|
||||
fn column<T>(&self, field: &str) -> crate::Result<Column<T>>
|
||||
where
|
||||
T: PartialOrd + Copy + HasAssociatedColumnType + Send + Sync + 'static,
|
||||
DynamicColumn: Into<Option<Column<T>>>,
|
||||
@@ -152,20 +216,52 @@ impl FastFieldReaders {
|
||||
field_name: &str,
|
||||
column_type: ColumnType,
|
||||
) -> crate::Result<Option<DynamicColumnHandle>> {
|
||||
let Some(resolved_field_name) = self.resolve_field(field_name) else {
|
||||
return Ok(None);
|
||||
};
|
||||
let dynamic_column_handle_opt = self
|
||||
.columnar
|
||||
.read_columns(field_name)?
|
||||
.read_columns(&resolved_field_name)?
|
||||
.into_iter()
|
||||
.find(|column| column.column_type() == column_type);
|
||||
Ok(dynamic_column_handle_opt)
|
||||
}
|
||||
|
||||
#[doc(hidden)]
|
||||
pub async fn list_dynamic_column_handles(
|
||||
&self,
|
||||
field_name: &str,
|
||||
) -> crate::Result<Vec<DynamicColumnHandle>> {
|
||||
let Some(resolved_field_name) = self.resolve_field(field_name) else {
|
||||
return Ok(Vec::new());
|
||||
};
|
||||
let columns = self
|
||||
.columnar
|
||||
.read_columns_async(&resolved_field_name)
|
||||
.await?;
|
||||
Ok(columns)
|
||||
}
|
||||
|
||||
/// Returns the `u64` column used to represent any `u64`-mapped typed (i64, u64, f64, DateTime).
|
||||
#[doc(hidden)]
|
||||
pub fn u64_lenient(&self, field_name: &str) -> crate::Result<Option<Column<u64>>> {
|
||||
for col in self.columnar.read_columns(field_name)? {
|
||||
Ok(self
|
||||
.u64_lenient_with_type(field_name)?
|
||||
.map(|(u64_column, _)| u64_column))
|
||||
}
|
||||
|
||||
/// Returns the `u64` column used to represent any `u64`-mapped typed (i64, u64, f64, DateTime).
|
||||
#[doc(hidden)]
|
||||
pub fn u64_lenient_with_type(
|
||||
&self,
|
||||
field_name: &str,
|
||||
) -> crate::Result<Option<(Column<u64>, ColumnType)>> {
|
||||
let Some(resolved_field_name) = self.resolve_field(field_name) else {
|
||||
return Ok(None);
|
||||
};
|
||||
for col in self.columnar.read_columns(&resolved_field_name)? {
|
||||
if let Some(col_u64) = col.open_u64_lenient()? {
|
||||
return Ok(Some(col_u64));
|
||||
return Ok(Some((col_u64, col.column_type())));
|
||||
}
|
||||
}
|
||||
Ok(None)
|
||||
@@ -192,3 +288,73 @@ impl FastFieldReaders {
|
||||
self.column(field_name)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use crate::schema::{JsonObjectOptions, Schema, FAST};
|
||||
use crate::{Document, Index};
|
||||
|
||||
#[test]
|
||||
fn test_fast_field_reader_resolve_with_dynamic_internal() {
|
||||
let mut schema_builder = Schema::builder();
|
||||
schema_builder.add_i64_field("age", FAST);
|
||||
schema_builder.add_json_field("json_expand_dots_disabled", FAST);
|
||||
schema_builder.add_json_field(
|
||||
"json_expand_dots_enabled",
|
||||
JsonObjectOptions::default()
|
||||
.set_fast()
|
||||
.set_expand_dots_enabled(),
|
||||
);
|
||||
let dynamic_field = schema_builder.add_json_field("_dyna", FAST);
|
||||
let schema = schema_builder.build();
|
||||
let index = Index::create_in_ram(schema);
|
||||
let mut index_writer = index.writer_for_tests().unwrap();
|
||||
index_writer.add_document(Document::default()).unwrap();
|
||||
index_writer.commit().unwrap();
|
||||
let reader = index.reader().unwrap();
|
||||
let searcher = reader.searcher();
|
||||
let reader = searcher.segment_reader(0u32);
|
||||
let fast_field_readers = reader.fast_fields();
|
||||
assert_eq!(
|
||||
fast_field_readers.resolve_column_name_given_default_field("age", None),
|
||||
Some("age".to_string())
|
||||
);
|
||||
assert_eq!(
|
||||
fast_field_readers.resolve_column_name_given_default_field("age", Some(dynamic_field)),
|
||||
Some("age".to_string())
|
||||
);
|
||||
assert_eq!(
|
||||
fast_field_readers.resolve_column_name_given_default_field(
|
||||
"json_expand_dots_disabled.attr.color",
|
||||
None
|
||||
),
|
||||
Some("json_expand_dots_disabled\u{1}attr\u{1}color".to_string())
|
||||
);
|
||||
assert_eq!(
|
||||
fast_field_readers.resolve_column_name_given_default_field(
|
||||
"json_expand_dots_disabled.attr\\.color",
|
||||
Some(dynamic_field)
|
||||
),
|
||||
Some("json_expand_dots_disabled\u{1}attr.color".to_string())
|
||||
);
|
||||
assert_eq!(
|
||||
fast_field_readers.resolve_column_name_given_default_field(
|
||||
"json_expand_dots_enabled.attr\\.color",
|
||||
Some(dynamic_field)
|
||||
),
|
||||
Some("json_expand_dots_enabled\u{1}attr\u{1}color".to_string())
|
||||
);
|
||||
assert_eq!(
|
||||
fast_field_readers
|
||||
.resolve_column_name_given_default_field("notinschema.attr.color", None),
|
||||
None
|
||||
);
|
||||
assert_eq!(
|
||||
fast_field_readers.resolve_column_name_given_default_field(
|
||||
"notinschema.attr.color",
|
||||
Some(dynamic_field)
|
||||
),
|
||||
Some("_dyna\u{1}notinschema\u{1}attr\u{1}color".to_string())
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,64 +1,68 @@
|
||||
use std::io;
|
||||
|
||||
use columnar::{ColumnType, ColumnarWriter, NumericalValue};
|
||||
use columnar::{ColumnarWriter, NumericalValue};
|
||||
use common::replace_in_place;
|
||||
|
||||
use crate::indexer::doc_id_mapping::DocIdMapping;
|
||||
use crate::schema::{Document, FieldType, Schema, Type, Value};
|
||||
use crate::schema::term::{JSON_PATH_SEGMENT_SEP, JSON_PATH_SEGMENT_SEP_STR};
|
||||
use crate::schema::{value_type_to_column_type, Document, FieldType, Schema, Type, Value};
|
||||
use crate::{DatePrecision, DocId};
|
||||
|
||||
/// Only index JSON down to a depth of 20.
|
||||
/// This is mostly to guard us from a stack overflow triggered by malicious input.
|
||||
const JSON_DEPTH_LIMIT: usize = 20;
|
||||
|
||||
/// The `FastFieldsWriter` groups all of the fast field writers.
|
||||
pub struct FastFieldsWriter {
|
||||
columnar_writer: ColumnarWriter,
|
||||
fast_field_names: Vec<Option<String>>, //< TODO see if we can cash the field name hash too.
|
||||
date_precisions: Vec<DatePrecision>,
|
||||
expand_dots: Vec<bool>,
|
||||
num_docs: DocId,
|
||||
// Buffer that we recycle to avoid allocation.
|
||||
json_path_buffer: String,
|
||||
}
|
||||
|
||||
impl FastFieldsWriter {
|
||||
/// Create all `FastFieldWriter` required by the schema.
|
||||
pub fn from_schema(schema: &Schema) -> FastFieldsWriter {
|
||||
let mut columnar_writer = ColumnarWriter::default();
|
||||
let mut fast_fields: Vec<Option<String>> = vec![None; schema.num_fields()];
|
||||
let mut fast_field_names: Vec<Option<String>> = vec![None; schema.num_fields()];
|
||||
let mut date_precisions: Vec<DatePrecision> =
|
||||
std::iter::repeat_with(DatePrecision::default)
|
||||
.take(schema.num_fields())
|
||||
.collect();
|
||||
let mut expand_dots = vec![false; schema.num_fields()];
|
||||
// TODO see other types
|
||||
for (field_id, field_entry) in schema.fields() {
|
||||
if !field_entry.field_type().is_fast() {
|
||||
continue;
|
||||
}
|
||||
fast_fields[field_id.field_id() as usize] = Some(field_entry.name().to_string());
|
||||
fast_field_names[field_id.field_id() as usize] = Some(field_entry.name().to_string());
|
||||
let value_type = field_entry.field_type().value_type();
|
||||
let column_type = match value_type {
|
||||
Type::Str => ColumnType::Str,
|
||||
Type::U64 => ColumnType::U64,
|
||||
Type::I64 => ColumnType::I64,
|
||||
Type::F64 => ColumnType::F64,
|
||||
Type::Bool => ColumnType::Bool,
|
||||
Type::Date => ColumnType::DateTime,
|
||||
Type::Facet => ColumnType::Str,
|
||||
Type::Bytes => ColumnType::Bytes,
|
||||
Type::Json => {
|
||||
continue;
|
||||
}
|
||||
Type::IpAddr => ColumnType::IpAddr,
|
||||
};
|
||||
if let FieldType::Date(date_options) = field_entry.field_type() {
|
||||
date_precisions[field_id.field_id() as usize] = date_options.get_precision();
|
||||
}
|
||||
if let FieldType::JsonObject(json_object_options) = field_entry.field_type() {
|
||||
expand_dots[field_id.field_id() as usize] =
|
||||
json_object_options.is_expand_dots_enabled();
|
||||
}
|
||||
let sort_values_within_row = value_type == Type::Facet;
|
||||
columnar_writer.record_column_type(
|
||||
field_entry.name(),
|
||||
column_type,
|
||||
sort_values_within_row,
|
||||
);
|
||||
if let Some(column_type) = value_type_to_column_type(value_type) {
|
||||
columnar_writer.record_column_type(
|
||||
field_entry.name(),
|
||||
column_type,
|
||||
sort_values_within_row,
|
||||
);
|
||||
}
|
||||
}
|
||||
FastFieldsWriter {
|
||||
columnar_writer,
|
||||
fast_field_names: fast_fields,
|
||||
fast_field_names,
|
||||
num_docs: 0u32,
|
||||
date_precisions,
|
||||
expand_dots,
|
||||
json_path_buffer: String::new(),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -82,7 +86,7 @@ impl FastFieldsWriter {
|
||||
let doc_id = self.num_docs;
|
||||
for field_value in doc.field_values() {
|
||||
if let Some(field_name) =
|
||||
self.fast_field_names[field_value.field().field_id() as usize].as_ref()
|
||||
&self.fast_field_names[field_value.field().field_id() as usize]
|
||||
{
|
||||
match &field_value.value {
|
||||
Value::U64(u64_val) => {
|
||||
@@ -136,7 +140,19 @@ impl FastFieldsWriter {
|
||||
facet.encoded_str(),
|
||||
);
|
||||
}
|
||||
Value::JsonObject(_) => todo!(),
|
||||
Value::JsonObject(json_obj) => {
|
||||
let expand_dots = self.expand_dots[field_value.field().field_id() as usize];
|
||||
self.json_path_buffer.clear();
|
||||
self.json_path_buffer.push_str(field_name);
|
||||
record_json_obj_to_columnar_writer(
|
||||
doc_id,
|
||||
json_obj,
|
||||
expand_dots,
|
||||
JSON_DEPTH_LIMIT,
|
||||
&mut self.json_path_buffer,
|
||||
&mut self.columnar_writer,
|
||||
);
|
||||
}
|
||||
Value::IpAddr(ip_addr) => {
|
||||
self.columnar_writer
|
||||
.record_ip_addr(doc_id, field_name.as_str(), *ip_addr);
|
||||
@@ -163,3 +179,242 @@ impl FastFieldsWriter {
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn columnar_numerical_value(json_number: &serde_json::Number) -> Option<NumericalValue> {
|
||||
if let Some(num_i64) = json_number.as_i64() {
|
||||
return Some(num_i64.into());
|
||||
}
|
||||
if let Some(num_u64) = json_number.as_u64() {
|
||||
return Some(num_u64.into());
|
||||
}
|
||||
if let Some(num_f64) = json_number.as_f64() {
|
||||
return Some(num_f64.into());
|
||||
}
|
||||
// This can happen with arbitrary precision.... but we do not handle it.
|
||||
None
|
||||
}
|
||||
|
||||
fn record_json_obj_to_columnar_writer(
|
||||
doc: DocId,
|
||||
json_obj: &serde_json::Map<String, serde_json::Value>,
|
||||
expand_dots: bool,
|
||||
remaining_depth_limit: usize,
|
||||
json_path_buffer: &mut String,
|
||||
columnar_writer: &mut columnar::ColumnarWriter,
|
||||
) {
|
||||
for (key, child) in json_obj {
|
||||
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()
|
||||
});
|
||||
}
|
||||
record_json_value_to_columnar_writer(
|
||||
doc,
|
||||
child,
|
||||
expand_dots,
|
||||
remaining_depth_limit,
|
||||
json_path_buffer,
|
||||
columnar_writer,
|
||||
);
|
||||
// popping our sub path.
|
||||
json_path_buffer.truncate(len_path);
|
||||
}
|
||||
}
|
||||
|
||||
fn record_json_value_to_columnar_writer(
|
||||
doc: DocId,
|
||||
json_val: &serde_json::Value,
|
||||
expand_dots: bool,
|
||||
mut remaining_depth_limit: usize,
|
||||
json_path_writer: &mut String,
|
||||
columnar_writer: &mut columnar::ColumnarWriter,
|
||||
) {
|
||||
if remaining_depth_limit == 0 {
|
||||
return;
|
||||
}
|
||||
remaining_depth_limit -= 1;
|
||||
match json_val {
|
||||
serde_json::Value::Null => {
|
||||
// TODO handle null
|
||||
}
|
||||
serde_json::Value::Bool(bool_val) => {
|
||||
columnar_writer.record_bool(doc, json_path_writer, *bool_val);
|
||||
}
|
||||
serde_json::Value::Number(json_number) => {
|
||||
if let Some(numerical_value) = columnar_numerical_value(json_number) {
|
||||
columnar_writer.record_numerical(doc, json_path_writer.as_str(), numerical_value);
|
||||
}
|
||||
}
|
||||
serde_json::Value::String(text) => {
|
||||
columnar_writer.record_str(doc, json_path_writer.as_str(), text);
|
||||
}
|
||||
serde_json::Value::Array(arr) => {
|
||||
for el in arr {
|
||||
record_json_value_to_columnar_writer(
|
||||
doc,
|
||||
el,
|
||||
expand_dots,
|
||||
remaining_depth_limit,
|
||||
json_path_writer,
|
||||
columnar_writer,
|
||||
);
|
||||
}
|
||||
}
|
||||
serde_json::Value::Object(json_obj) => {
|
||||
record_json_obj_to_columnar_writer(
|
||||
doc,
|
||||
json_obj,
|
||||
expand_dots,
|
||||
remaining_depth_limit,
|
||||
json_path_writer,
|
||||
columnar_writer,
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use columnar::{Column, ColumnarReader, ColumnarWriter, StrColumn};
|
||||
|
||||
use super::record_json_value_to_columnar_writer;
|
||||
use crate::fastfield::writer::JSON_DEPTH_LIMIT;
|
||||
use crate::DocId;
|
||||
|
||||
fn test_columnar_from_jsons_aux(
|
||||
json_docs: &[serde_json::Value],
|
||||
expand_dots: bool,
|
||||
) -> ColumnarReader {
|
||||
let mut columnar_writer = ColumnarWriter::default();
|
||||
let mut json_path = String::new();
|
||||
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,
|
||||
);
|
||||
}
|
||||
let mut buffer = Vec::new();
|
||||
columnar_writer
|
||||
.serialize(json_docs.len() as DocId, None, &mut buffer)
|
||||
.unwrap();
|
||||
ColumnarReader::open(buffer).unwrap()
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_json_fastfield_record_simple() {
|
||||
let json_doc = serde_json::json!({
|
||||
"float": 1.02,
|
||||
"text": "hello happy tax payer",
|
||||
"nested": {"child": 3, "child2": 5},
|
||||
"arr": ["hello", "happy", "tax", "payer"]
|
||||
});
|
||||
let columnar_reader = test_columnar_from_jsons_aux(&[json_doc], false);
|
||||
let columns = columnar_reader.list_columns().unwrap();
|
||||
{
|
||||
assert_eq!(columns[0].0, "arr");
|
||||
let column_arr_opt: Option<StrColumn> = columns[0].1.open().unwrap().into();
|
||||
assert!(column_arr_opt
|
||||
.unwrap()
|
||||
.term_ords(0)
|
||||
.eq([1, 0, 3, 2].into_iter()));
|
||||
}
|
||||
{
|
||||
assert_eq!(columns[1].0, "float");
|
||||
let column_float_opt: Option<Column<f64>> = columns[1].1.open().unwrap().into();
|
||||
assert!(column_float_opt
|
||||
.unwrap()
|
||||
.values_for_doc(0)
|
||||
.eq([1.02f64].into_iter()));
|
||||
}
|
||||
{
|
||||
assert_eq!(columns[2].0, "nested\u{1}child");
|
||||
let column_nest_child_opt: Option<Column<i64>> = columns[2].1.open().unwrap().into();
|
||||
assert!(column_nest_child_opt
|
||||
.unwrap()
|
||||
.values_for_doc(0)
|
||||
.eq([3].into_iter()));
|
||||
}
|
||||
{
|
||||
assert_eq!(columns[3].0, "nested\u{1}child2");
|
||||
let column_nest_child2_opt: Option<Column<i64>> = columns[3].1.open().unwrap().into();
|
||||
assert!(column_nest_child2_opt
|
||||
.unwrap()
|
||||
.values_for_doc(0)
|
||||
.eq([5].into_iter()));
|
||||
}
|
||||
{
|
||||
assert_eq!(columns[4].0, "text");
|
||||
let column_text_opt: Option<StrColumn> = columns[4].1.open().unwrap().into();
|
||||
assert!(column_text_opt.unwrap().term_ords(0).eq([0].into_iter()));
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_json_fastfield_deep_obj() {
|
||||
let json_doc = serde_json::json!(
|
||||
{"a": {"a": {"a": {"a": {"a":
|
||||
{"a": {"a": {"a": {"a": {"a":
|
||||
{"a": {"a": {"a": {"a": {"a":
|
||||
{"a": {"a": {"a": {"depth_accepted": 19, "a": { "depth_truncated": 20}
|
||||
}}}}}}}}}}}}}}}}}}});
|
||||
let columnar_reader = test_columnar_from_jsons_aux(&[json_doc], false);
|
||||
let columns = columnar_reader.list_columns().unwrap();
|
||||
assert_eq!(columns.len(), 1);
|
||||
assert!(columns[0].0.ends_with("a\u{1}a\u{1}a\u{1}depth_accepted"));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_json_fastfield_deep_arr() {
|
||||
let json_doc = json!(
|
||||
{"obj":
|
||||
[[[[[,
|
||||
[[[[[,
|
||||
[[[[[,
|
||||
[[18, [19, //< within limits
|
||||
[20]]]]]]]]]]]]]]]]]]]});
|
||||
let columnar_reader = test_columnar_from_jsons_aux(&[json_doc], false);
|
||||
let columns = columnar_reader.list_columns().unwrap();
|
||||
assert_eq!(columns.len(), 1);
|
||||
assert_eq!(columns[0].0, "obj");
|
||||
let dynamic_column = columns[0].1.open().unwrap();
|
||||
let col: Option<Column<i64>> = dynamic_column.into();
|
||||
let vals: Vec<i64> = col.unwrap().values_for_doc(0).collect();
|
||||
assert_eq!(&vals, &[18, 19])
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_json_fast_field_do_not_expand_dots() {
|
||||
let json_doc = json!({"field.with.dots": {"child.with.dot": "hello"}});
|
||||
let columnar_reader = test_columnar_from_jsons_aux(&[json_doc], false);
|
||||
let columns = columnar_reader.list_columns().unwrap();
|
||||
assert_eq!(columns.len(), 1);
|
||||
assert_eq!(columns[0].0, "field.with.dots\u{1}child.with.dot");
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_json_fast_field_expand_dots() {
|
||||
let json_doc = json!({"field.with.dots": {"child.with.dot": "hello"}});
|
||||
let columnar_reader = test_columnar_from_jsons_aux(&[json_doc], true);
|
||||
let columns = columnar_reader.list_columns().unwrap();
|
||||
assert_eq!(columns.len(), 1);
|
||||
assert_eq!(
|
||||
columns[0].0,
|
||||
"field\u{1}with\u{1}dots\u{1}child\u{1}with\u{1}dot"
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -112,7 +112,7 @@ mod tests {
|
||||
Term::from_field_text(text, "hello"),
|
||||
IndexRecordOption::WithFreqs,
|
||||
);
|
||||
let weight = query.weight(EnableScoring::Enabled(&searcher))?;
|
||||
let weight = query.weight(EnableScoring::enabled_from_searcher(&searcher))?;
|
||||
let mut scorer = weight.scorer(searcher.segment_reader(0), 1.0f32)?;
|
||||
assert_eq!(scorer.doc(), 0);
|
||||
assert!((scorer.score() - 0.22920431).abs() < 0.001f32);
|
||||
@@ -141,7 +141,7 @@ mod tests {
|
||||
Term::from_field_text(text, "hello"),
|
||||
IndexRecordOption::WithFreqs,
|
||||
);
|
||||
let weight = query.weight(EnableScoring::Enabled(&searcher))?;
|
||||
let weight = query.weight(EnableScoring::enabled_from_searcher(&searcher))?;
|
||||
let mut scorer = weight.scorer(searcher.segment_reader(0), 1.0f32)?;
|
||||
assert_eq!(scorer.doc(), 0);
|
||||
assert!((scorer.score() - 0.22920431).abs() < 0.001f32);
|
||||
|
||||
@@ -1490,7 +1490,7 @@ mod tests {
|
||||
|
||||
let in_order_alive_ids: Vec<u64> = segment_reader
|
||||
.doc_ids_alive()
|
||||
.flat_map(|doc| fast_field_reader.values(doc))
|
||||
.flat_map(|doc| fast_field_reader.values_for_doc(doc))
|
||||
.collect();
|
||||
assert_eq!(&in_order_alive_ids[..], &[9, 8, 7, 6, 5, 4, 1, 0]);
|
||||
Ok(())
|
||||
@@ -1550,7 +1550,7 @@ mod tests {
|
||||
let fast_field_reader = segment_reader.fast_fields().u64("id")?;
|
||||
let in_order_alive_ids: Vec<u64> = segment_reader
|
||||
.doc_ids_alive()
|
||||
.flat_map(|doc| fast_field_reader.values(doc))
|
||||
.flat_map(|doc| fast_field_reader.values_for_doc(doc))
|
||||
.collect();
|
||||
assert_eq!(&in_order_alive_ids[..], &[9, 8, 7, 6, 5, 4, 2, 0]);
|
||||
Ok(())
|
||||
@@ -1795,7 +1795,7 @@ mod tests {
|
||||
let ff_reader = segment_reader.fast_fields().u64("id").unwrap();
|
||||
segment_reader
|
||||
.doc_ids_alive()
|
||||
.flat_map(move |doc| ff_reader.values(doc).collect_vec().into_iter())
|
||||
.flat_map(move |doc| ff_reader.values_for_doc(doc).collect_vec().into_iter())
|
||||
})
|
||||
.collect();
|
||||
|
||||
@@ -1806,7 +1806,7 @@ mod tests {
|
||||
let ff_reader = segment_reader.fast_fields().u64("id").unwrap();
|
||||
segment_reader
|
||||
.doc_ids_alive()
|
||||
.flat_map(move |doc| ff_reader.values(doc).collect_vec().into_iter())
|
||||
.flat_map(move |doc| ff_reader.values_for_doc(doc).collect_vec().into_iter())
|
||||
})
|
||||
.collect();
|
||||
|
||||
@@ -1824,7 +1824,7 @@ mod tests {
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
for doc in segment_reader.doc_ids_alive() {
|
||||
all_ips.extend(ip_reader.values(doc));
|
||||
all_ips.extend(ip_reader.values_for_doc(doc));
|
||||
}
|
||||
num_ips += ip_reader.values.num_vals();
|
||||
}
|
||||
@@ -1883,7 +1883,7 @@ mod tests {
|
||||
for reader in searcher.segment_readers() {
|
||||
if let Some(ff_reader) = reader.fast_fields().column_opt::<Ipv6Addr>("ips").unwrap() {
|
||||
for doc in reader.doc_ids_alive() {
|
||||
ips.extend(ff_reader.values(doc));
|
||||
ips.extend(ff_reader.values_for_doc(doc));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1915,7 +1915,7 @@ mod tests {
|
||||
for reader in searcher.segment_readers() {
|
||||
if let Some(ff_reader) = reader.fast_fields().column_opt::<Ipv6Addr>("ips").unwrap() {
|
||||
for doc in reader.doc_ids_alive() {
|
||||
ips.extend(ff_reader.values(doc));
|
||||
ips.extend(ff_reader.values_for_doc(doc));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1935,12 +1935,12 @@ mod tests {
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
for doc in segment_reader.doc_ids_alive() {
|
||||
let vals: Vec<u64> = ff_reader.values(doc).collect();
|
||||
let vals: Vec<u64> = ff_reader.values_for_doc(doc).collect();
|
||||
assert_eq!(vals.len(), 2);
|
||||
assert_eq!(vals[0], vals[1]);
|
||||
assert_eq!(id_reader.first(doc), Some(vals[0]));
|
||||
|
||||
let bool_vals: Vec<bool> = bool_ff_reader.values(doc).collect();
|
||||
let bool_vals: Vec<bool> = bool_ff_reader.values_for_doc(doc).collect();
|
||||
assert_eq!(bool_vals.len(), 2);
|
||||
assert_ne!(bool_vals[0], bool_vals[1]);
|
||||
|
||||
@@ -2236,8 +2236,8 @@ mod tests {
|
||||
assert_eq!(val_col.get_cardinality(), Cardinality::Multivalued);
|
||||
assert_eq!(id_col.first(0u32), Some(1u64));
|
||||
assert_eq!(id_col.first(1u32), Some(2u64));
|
||||
assert!(val_col.values(0u32).eq([1u64, 1u64].into_iter()));
|
||||
assert!(val_col.values(1u32).eq([2u64, 2u64].into_iter()));
|
||||
assert!(val_col.values_for_doc(0u32).eq([1u64, 1u64].into_iter()));
|
||||
assert!(val_col.values_for_doc(1u32).eq([2u64, 2u64].into_iter()));
|
||||
}
|
||||
|
||||
#[test]
|
||||
|
||||
@@ -1,7 +1,8 @@
|
||||
use std::sync::Arc;
|
||||
|
||||
use columnar::{
|
||||
ColumnValues, ColumnarReader, MergeRowOrder, RowAddr, ShuffleMergeOrder, StackMergeOrder,
|
||||
ColumnType, ColumnValues, ColumnarReader, MergeRowOrder, RowAddr, ShuffleMergeOrder,
|
||||
StackMergeOrder,
|
||||
};
|
||||
use common::ReadOnlyBitSet;
|
||||
use itertools::Itertools;
|
||||
@@ -16,7 +17,7 @@ use crate::fieldnorm::{FieldNormReader, FieldNormReaders, FieldNormsSerializer,
|
||||
use crate::indexer::doc_id_mapping::{MappingType, SegmentDocIdMapping};
|
||||
use crate::indexer::SegmentSerializer;
|
||||
use crate::postings::{InvertedIndexSerializer, Postings, SegmentPostings};
|
||||
use crate::schema::{Field, FieldType, Schema};
|
||||
use crate::schema::{value_type_to_column_type, Field, FieldType, Schema};
|
||||
use crate::store::StoreWriter;
|
||||
use crate::termdict::{TermMerger, TermOrdinal};
|
||||
use crate::{
|
||||
@@ -135,6 +136,19 @@ fn convert_to_merge_order(
|
||||
}
|
||||
}
|
||||
|
||||
fn extract_fast_field_required_columns(schema: &Schema) -> Vec<(String, ColumnType)> {
|
||||
schema
|
||||
.fields()
|
||||
.map(|(_, field_entry)| field_entry)
|
||||
.filter(|field_entry| field_entry.is_fast())
|
||||
.filter_map(|field_entry| {
|
||||
let column_name = field_entry.name().to_string();
|
||||
let column_type = value_type_to_column_type(field_entry.field_type().value_type())?;
|
||||
Some((column_name, column_type))
|
||||
})
|
||||
.collect()
|
||||
}
|
||||
|
||||
impl IndexMerger {
|
||||
pub fn open(
|
||||
schema: Schema,
|
||||
@@ -248,13 +262,19 @@ impl IndexMerger {
|
||||
doc_id_mapping: SegmentDocIdMapping,
|
||||
) -> crate::Result<()> {
|
||||
debug_time!("write-fast-fields");
|
||||
let required_columns = extract_fast_field_required_columns(&self.schema);
|
||||
let columnars: Vec<&ColumnarReader> = self
|
||||
.readers
|
||||
.iter()
|
||||
.map(|reader| reader.fast_fields().columnar())
|
||||
.collect();
|
||||
let merge_row_order = convert_to_merge_order(&columnars[..], doc_id_mapping);
|
||||
columnar::merge_columnar(&columnars[..], merge_row_order, fast_field_wrt)?;
|
||||
columnar::merge_columnar(
|
||||
&columnars[..],
|
||||
&required_columns,
|
||||
merge_row_order,
|
||||
fast_field_wrt,
|
||||
)?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@@ -1477,7 +1497,7 @@ mod tests {
|
||||
let mut vals: Vec<u64> = Vec::new();
|
||||
let mut test_vals = move |col: &Column<u64>, doc: DocId, expected: &[u64]| {
|
||||
vals.clear();
|
||||
vals.extend(col.values(doc));
|
||||
vals.extend(col.values_for_doc(doc));
|
||||
assert_eq!(&vals[..], expected);
|
||||
};
|
||||
|
||||
@@ -1628,7 +1648,7 @@ mod tests {
|
||||
let reader = index.reader()?;
|
||||
let searcher = reader.searcher();
|
||||
let mut term_scorer = term_query
|
||||
.specialized_weight(EnableScoring::Enabled(&searcher))?
|
||||
.specialized_weight(EnableScoring::enabled_from_searcher(&searcher))?
|
||||
.specialized_scorer(searcher.segment_reader(0u32), 1.0)?;
|
||||
assert_eq!(term_scorer.doc(), 0);
|
||||
assert_nearly_equals!(term_scorer.block_max_score(), 0.0079681855);
|
||||
@@ -1643,7 +1663,7 @@ mod tests {
|
||||
assert_eq!(searcher.segment_readers().len(), 2);
|
||||
for segment_reader in searcher.segment_readers() {
|
||||
let mut term_scorer = term_query
|
||||
.specialized_weight(EnableScoring::Enabled(&searcher))?
|
||||
.specialized_weight(EnableScoring::enabled_from_searcher(&searcher))?
|
||||
.specialized_scorer(segment_reader, 1.0)?;
|
||||
// the difference compared to before is intrinsic to the bm25 formula. no worries
|
||||
// there.
|
||||
@@ -1668,7 +1688,7 @@ mod tests {
|
||||
|
||||
let segment_reader = searcher.segment_reader(0u32);
|
||||
let mut term_scorer = term_query
|
||||
.specialized_weight(EnableScoring::Enabled(&searcher))?
|
||||
.specialized_weight(EnableScoring::enabled_from_searcher(&searcher))?
|
||||
.specialized_scorer(segment_reader, 1.0)?;
|
||||
// the difference compared to before is intrinsic to the bm25 formula. no worries there.
|
||||
for doc in segment_reader.doc_ids_alive() {
|
||||
|
||||
@@ -475,8 +475,6 @@ mod tests {
|
||||
#[cfg(all(test, feature = "unstable"))]
|
||||
mod bench_sorted_index_merge {
|
||||
|
||||
use std::sync::Arc;
|
||||
|
||||
use test::{self, Bencher};
|
||||
|
||||
use crate::core::Index;
|
||||
|
||||
@@ -5,7 +5,6 @@ mod doc_opstamp_mapping;
|
||||
mod flat_map_with_buffer;
|
||||
pub mod index_writer;
|
||||
mod index_writer_status;
|
||||
mod json_term_writer;
|
||||
mod log_merge_policy;
|
||||
mod merge_operation;
|
||||
pub mod merge_policy;
|
||||
@@ -25,9 +24,6 @@ use crossbeam_channel as channel;
|
||||
use smallvec::SmallVec;
|
||||
|
||||
pub use self::index_writer::IndexWriter;
|
||||
pub(crate) use self::json_term_writer::{
|
||||
convert_to_fast_value_and_get_term, set_string_and_get_terms, JsonTermWriter,
|
||||
};
|
||||
pub use self::log_merge_policy::LogMergePolicy;
|
||||
pub use self::merge_operation::MergeOperation;
|
||||
pub use self::merge_policy::{MergeCandidate, MergePolicy, NoMergePolicy};
|
||||
@@ -56,8 +52,9 @@ type AddBatchReceiver = channel::Receiver<AddBatch>;
|
||||
#[cfg(test)]
|
||||
mod tests_mmap {
|
||||
|
||||
// use crate::query::QueryParser;
|
||||
use crate::schema::{Schema, TEXT};
|
||||
use crate::collector::Count;
|
||||
use crate::query::QueryParser;
|
||||
use crate::schema::{JsonObjectOptions, Schema, TEXT};
|
||||
use crate::{Index, Term};
|
||||
|
||||
#[test]
|
||||
@@ -77,45 +74,63 @@ mod tests_mmap {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
// #[test]
|
||||
// fn test_json_field_expand_dots_disabled_dot_escaped_required() {
|
||||
// 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();
|
||||
// let json = serde_json::json!({"k8s.container.name": "prometheus", "val": "hello"});
|
||||
// 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(), 1);
|
||||
// let parse_query = QueryParser::for_index(&index, Vec::new());
|
||||
// let query = parse_query
|
||||
// .parse_query(r#"json.k8s\.container\.name:prometheus"#)
|
||||
// .unwrap();
|
||||
// let num_docs = searcher.search(&query, &Count).unwrap();
|
||||
// assert_eq!(num_docs, 1);
|
||||
// }
|
||||
#[test]
|
||||
fn test_json_field_expand_dots_disabled_dot_escaped_required() {
|
||||
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();
|
||||
let json = serde_json::json!({"k8s.container.name": "prometheus", "val": "hello"});
|
||||
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(), 1);
|
||||
let parse_query = QueryParser::for_index(&index, Vec::new());
|
||||
{
|
||||
let query = parse_query
|
||||
.parse_query(r#"json.k8s\.container\.name:prometheus"#)
|
||||
.unwrap();
|
||||
let num_docs = searcher.search(&query, &Count).unwrap();
|
||||
assert_eq!(num_docs, 1);
|
||||
}
|
||||
{
|
||||
let query = parse_query
|
||||
.parse_query(r#"json.k8s.container.name:prometheus"#)
|
||||
.unwrap();
|
||||
let num_docs = searcher.search(&query, &Count).unwrap();
|
||||
assert_eq!(num_docs, 0);
|
||||
}
|
||||
}
|
||||
|
||||
// #[test]
|
||||
// fn test_json_field_expand_dots_enabled_dot_escape_not_required() {
|
||||
// let mut schema_builder = Schema::builder();
|
||||
// let json_options: JsonObjectOptions =
|
||||
// JsonObjectOptions::from(TEXT).set_expand_dots_enabled();
|
||||
// let json_field = schema_builder.add_json_field("json", json_options);
|
||||
// let index = Index::create_in_ram(schema_builder.build());
|
||||
// let mut index_writer = index.writer_for_tests().unwrap();
|
||||
// let json = serde_json::json!({"k8s.container.name": "prometheus", "val": "hello"});
|
||||
// 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(), 1);
|
||||
// let parse_query = QueryParser::for_index(&index, Vec::new());
|
||||
// let query = parse_query
|
||||
// .parse_query(r#"json.k8s.container.name:prometheus"#)
|
||||
// .unwrap();
|
||||
// let num_docs = searcher.search(&query, &Count).unwrap();
|
||||
// assert_eq!(num_docs, 1);
|
||||
// }
|
||||
#[test]
|
||||
fn test_json_field_expand_dots_enabled_dot_escape_not_required() {
|
||||
let mut schema_builder = Schema::builder();
|
||||
let json_options: JsonObjectOptions =
|
||||
JsonObjectOptions::from(TEXT).set_expand_dots_enabled();
|
||||
let json_field = schema_builder.add_json_field("json", json_options);
|
||||
let index = Index::create_in_ram(schema_builder.build());
|
||||
let mut index_writer = index.writer_for_tests().unwrap();
|
||||
let json = serde_json::json!({"k8s.container.name": "prometheus", "val": "hello"});
|
||||
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(), 1);
|
||||
let parse_query = QueryParser::for_index(&index, Vec::new());
|
||||
{
|
||||
let query = parse_query
|
||||
.parse_query(r#"json.k8s.container.name:prometheus"#)
|
||||
.unwrap();
|
||||
let num_docs = searcher.search(&query, &Count).unwrap();
|
||||
assert_eq!(num_docs, 1);
|
||||
}
|
||||
{
|
||||
let query = parse_query
|
||||
.parse_query(r#"json.k8s\.container\.name:prometheus"#)
|
||||
.unwrap();
|
||||
let num_docs = searcher.search(&query, &Count).unwrap();
|
||||
assert_eq!(num_docs, 1);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -3,10 +3,10 @@ use itertools::Itertools;
|
||||
|
||||
use super::doc_id_mapping::{get_doc_id_mapping_from_field, DocIdMapping};
|
||||
use super::operation::AddOperation;
|
||||
use crate::core::json_utils::index_json_values;
|
||||
use crate::core::Segment;
|
||||
use crate::fastfield::FastFieldsWriter;
|
||||
use crate::fieldnorm::{FieldNormReaders, FieldNormsWriter};
|
||||
use crate::indexer::json_term_writer::index_json_values;
|
||||
use crate::indexer::segment_serializer::SegmentSerializer;
|
||||
use crate::postings::{
|
||||
compute_table_size, serialize_postings, IndexingContext, IndexingPosition,
|
||||
@@ -435,8 +435,8 @@ mod tests {
|
||||
|
||||
use super::compute_initial_table_size;
|
||||
use crate::collector::Count;
|
||||
use crate::core::json_utils::JsonTermWriter;
|
||||
use crate::directory::RamDirectory;
|
||||
use crate::indexer::json_term_writer::JsonTermWriter;
|
||||
use crate::postings::TermInfo;
|
||||
use crate::query::PhraseQuery;
|
||||
use crate::schema::{IndexRecordOption, Schema, Type, STORED, STRING, TEXT};
|
||||
|
||||
@@ -123,11 +123,9 @@ mod functional_test;
|
||||
mod macros;
|
||||
mod future_result;
|
||||
|
||||
// Re-exports
|
||||
pub use common::DateTime;
|
||||
/// Re-export of the `time` crate
|
||||
///
|
||||
/// Tantivy uses [`time`](https://crates.io/crates/time) for dates.
|
||||
pub use time;
|
||||
pub use {columnar, query_grammar, time};
|
||||
|
||||
pub use crate::error::TantivyError;
|
||||
pub use crate::future_result::FutureResult;
|
||||
|
||||
@@ -2,11 +2,53 @@ use serde::{Deserialize, Serialize};
|
||||
|
||||
use crate::fieldnorm::FieldNormReader;
|
||||
use crate::query::Explanation;
|
||||
use crate::schema::Field;
|
||||
use crate::{Score, Searcher, Term};
|
||||
|
||||
const K1: Score = 1.2;
|
||||
const B: Score = 0.75;
|
||||
|
||||
/// An interface to compute the statistics needed in BM25 scoring.
|
||||
///
|
||||
/// The standard implementation is a [Searcher] but you can also
|
||||
/// create your own to adjust the statistics.
|
||||
pub trait Bm25StatisticsProvider {
|
||||
/// The total number of tokens in a given field across all documents in
|
||||
/// the index.
|
||||
fn total_num_tokens(&self, field: Field) -> crate::Result<u64>;
|
||||
|
||||
/// The total number of documents in the index.
|
||||
fn total_num_docs(&self) -> crate::Result<u64>;
|
||||
|
||||
/// The number of documents containing the given term.
|
||||
fn doc_freq(&self, term: &Term) -> crate::Result<u64>;
|
||||
}
|
||||
|
||||
impl Bm25StatisticsProvider for Searcher {
|
||||
fn total_num_tokens(&self, field: Field) -> crate::Result<u64> {
|
||||
let mut total_num_tokens = 0u64;
|
||||
|
||||
for segment_reader in self.segment_readers() {
|
||||
let inverted_index = segment_reader.inverted_index(field)?;
|
||||
total_num_tokens += inverted_index.total_num_tokens();
|
||||
}
|
||||
Ok(total_num_tokens)
|
||||
}
|
||||
|
||||
fn total_num_docs(&self) -> crate::Result<u64> {
|
||||
let mut total_num_docs = 0u64;
|
||||
|
||||
for segment_reader in self.segment_readers() {
|
||||
total_num_docs += u64::from(segment_reader.max_doc());
|
||||
}
|
||||
Ok(total_num_docs)
|
||||
}
|
||||
|
||||
fn doc_freq(&self, term: &Term) -> crate::Result<u64> {
|
||||
self.doc_freq(term)
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn idf(doc_freq: u64, doc_count: u64) -> Score {
|
||||
assert!(doc_count >= doc_freq, "{} >= {}", doc_count, doc_freq);
|
||||
let x = ((doc_count - doc_freq) as Score + 0.5) / (doc_freq as Score + 0.5);
|
||||
@@ -32,6 +74,7 @@ pub struct Bm25Params {
|
||||
pub avg_fieldnorm: Score,
|
||||
}
|
||||
|
||||
/// A struct used for computing BM25 scores.
|
||||
#[derive(Clone)]
|
||||
pub struct Bm25Weight {
|
||||
idf_explain: Explanation,
|
||||
@@ -41,6 +84,7 @@ pub struct Bm25Weight {
|
||||
}
|
||||
|
||||
impl Bm25Weight {
|
||||
/// Increase the weight by a multiplicative factor.
|
||||
pub fn boost_by(&self, boost: Score) -> Bm25Weight {
|
||||
Bm25Weight {
|
||||
idf_explain: self.idf_explain.clone(),
|
||||
@@ -50,7 +94,11 @@ impl Bm25Weight {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn for_terms(searcher: &Searcher, terms: &[Term]) -> crate::Result<Bm25Weight> {
|
||||
/// Construct a [Bm25Weight] for a phrase of terms.
|
||||
pub fn for_terms(
|
||||
statistics: &dyn Bm25StatisticsProvider,
|
||||
terms: &[Term],
|
||||
) -> crate::Result<Bm25Weight> {
|
||||
assert!(!terms.is_empty(), "Bm25 requires at least one term");
|
||||
let field = terms[0].field();
|
||||
for term in &terms[1..] {
|
||||
@@ -61,17 +109,12 @@ impl Bm25Weight {
|
||||
);
|
||||
}
|
||||
|
||||
let mut total_num_tokens = 0u64;
|
||||
let mut total_num_docs = 0u64;
|
||||
for segment_reader in searcher.segment_readers() {
|
||||
let inverted_index = segment_reader.inverted_index(field)?;
|
||||
total_num_tokens += inverted_index.total_num_tokens();
|
||||
total_num_docs += u64::from(segment_reader.max_doc());
|
||||
}
|
||||
let total_num_tokens = statistics.total_num_tokens(field)?;
|
||||
let total_num_docs = statistics.total_num_docs()?;
|
||||
let average_fieldnorm = total_num_tokens as Score / total_num_docs as Score;
|
||||
|
||||
if terms.len() == 1 {
|
||||
let term_doc_freq = searcher.doc_freq(&terms[0])?;
|
||||
let term_doc_freq = statistics.doc_freq(&terms[0])?;
|
||||
Ok(Bm25Weight::for_one_term(
|
||||
term_doc_freq,
|
||||
total_num_docs,
|
||||
@@ -80,7 +123,7 @@ impl Bm25Weight {
|
||||
} else {
|
||||
let mut idf_sum: Score = 0.0;
|
||||
for term in terms {
|
||||
let term_doc_freq = searcher.doc_freq(term)?;
|
||||
let term_doc_freq = statistics.doc_freq(term)?;
|
||||
idf_sum += idf(term_doc_freq, total_num_docs);
|
||||
}
|
||||
let idf_explain = Explanation::new("idf", idf_sum);
|
||||
@@ -88,6 +131,7 @@ impl Bm25Weight {
|
||||
}
|
||||
}
|
||||
|
||||
/// Construct a [Bm25Weight] for a single term.
|
||||
pub fn for_one_term(
|
||||
term_doc_freq: u64,
|
||||
total_num_docs: u64,
|
||||
@@ -114,11 +158,13 @@ impl Bm25Weight {
|
||||
}
|
||||
}
|
||||
|
||||
/// Compute the BM25 score of a single document.
|
||||
#[inline]
|
||||
pub fn score(&self, fieldnorm_id: u8, term_freq: u32) -> Score {
|
||||
self.weight * self.tf_factor(fieldnorm_id, term_freq)
|
||||
}
|
||||
|
||||
/// Compute the maximum possible BM25 score given this weight.
|
||||
pub fn max_score(&self) -> Score {
|
||||
self.score(255u8, 2_013_265_944)
|
||||
}
|
||||
@@ -130,6 +176,7 @@ impl Bm25Weight {
|
||||
term_freq / (term_freq + norm)
|
||||
}
|
||||
|
||||
/// Produce an [Explanation] of a BM25 score.
|
||||
pub fn explain(&self, fieldnorm_id: u8, term_freq: u32) -> Explanation {
|
||||
// The explain format is directly copied from Lucene's.
|
||||
// (So, Kudos to Lucene)
|
||||
|
||||
@@ -55,7 +55,7 @@ mod tests {
|
||||
let query_parser = QueryParser::for_index(&index, vec![text_field]);
|
||||
let query = query_parser.parse_query("+a")?;
|
||||
let searcher = index.reader()?.searcher();
|
||||
let weight = query.weight(EnableScoring::Enabled(&searcher))?;
|
||||
let weight = query.weight(EnableScoring::enabled_from_searcher(&searcher))?;
|
||||
let scorer = weight.scorer(searcher.segment_reader(0u32), 1.0)?;
|
||||
assert!(scorer.is::<TermScorer>());
|
||||
Ok(())
|
||||
@@ -68,13 +68,13 @@ mod tests {
|
||||
let searcher = index.reader()?.searcher();
|
||||
{
|
||||
let query = query_parser.parse_query("+a +b +c")?;
|
||||
let weight = query.weight(EnableScoring::Enabled(&searcher))?;
|
||||
let weight = query.weight(EnableScoring::enabled_from_searcher(&searcher))?;
|
||||
let scorer = weight.scorer(searcher.segment_reader(0u32), 1.0)?;
|
||||
assert!(scorer.is::<Intersection<TermScorer>>());
|
||||
}
|
||||
{
|
||||
let query = query_parser.parse_query("+a +(b c)")?;
|
||||
let weight = query.weight(EnableScoring::Enabled(&searcher))?;
|
||||
let weight = query.weight(EnableScoring::enabled_from_searcher(&searcher))?;
|
||||
let scorer = weight.scorer(searcher.segment_reader(0u32), 1.0)?;
|
||||
assert!(scorer.is::<Intersection<Box<dyn Scorer>>>());
|
||||
}
|
||||
@@ -88,7 +88,7 @@ mod tests {
|
||||
let searcher = index.reader()?.searcher();
|
||||
{
|
||||
let query = query_parser.parse_query("+a b")?;
|
||||
let weight = query.weight(EnableScoring::Enabled(&searcher))?;
|
||||
let weight = query.weight(EnableScoring::enabled_from_searcher(&searcher))?;
|
||||
let scorer = weight.scorer(searcher.segment_reader(0u32), 1.0)?;
|
||||
assert!(scorer.is::<RequiredOptionalScorer<
|
||||
Box<dyn Scorer>,
|
||||
@@ -243,7 +243,7 @@ mod tests {
|
||||
let boolean_query =
|
||||
BooleanQuery::new(vec![(Occur::Should, term_a), (Occur::Should, term_b)]);
|
||||
let boolean_weight = boolean_query
|
||||
.weight(EnableScoring::Enabled(&searcher))
|
||||
.weight(EnableScoring::enabled_from_searcher(&searcher))
|
||||
.unwrap();
|
||||
{
|
||||
let mut boolean_scorer = boolean_weight.scorer(searcher.segment_reader(0u32), 1.0)?;
|
||||
|
||||
@@ -1,6 +1,4 @@
|
||||
use tantivy_query_grammar::Occur;
|
||||
|
||||
use crate::query::{BooleanWeight, DisjunctionMaxCombiner, EnableScoring, Query, Weight};
|
||||
use crate::query::{BooleanWeight, DisjunctionMaxCombiner, EnableScoring, Occur, Query, Weight};
|
||||
use crate::{Score, Term};
|
||||
|
||||
/// The disjunction max query returns documents matching one or more wrapped queries,
|
||||
|
||||
@@ -12,6 +12,7 @@ mod explanation;
|
||||
mod fuzzy_query;
|
||||
mod intersection;
|
||||
mod more_like_this;
|
||||
mod phrase_prefix_query;
|
||||
mod phrase_query;
|
||||
mod query;
|
||||
mod query_parser;
|
||||
@@ -28,12 +29,12 @@ mod weight;
|
||||
mod vec_docset;
|
||||
|
||||
pub(crate) mod score_combiner;
|
||||
pub use tantivy_query_grammar::Occur;
|
||||
pub use query_grammar::Occur;
|
||||
|
||||
pub use self::all_query::{AllQuery, AllScorer, AllWeight};
|
||||
pub use self::automaton_weight::AutomatonWeight;
|
||||
pub use self::bitset::BitSetDocSet;
|
||||
pub(crate) use self::bm25::Bm25Weight;
|
||||
pub use self::bm25::{Bm25StatisticsProvider, Bm25Weight};
|
||||
pub use self::boolean_query::BooleanQuery;
|
||||
pub(crate) use self::boolean_query::BooleanWeight;
|
||||
pub use self::boost_query::BoostQuery;
|
||||
@@ -47,6 +48,7 @@ pub(crate) use self::fuzzy_query::DfaWrapper;
|
||||
pub use self::fuzzy_query::FuzzyTermQuery;
|
||||
pub use self::intersection::{intersect_scorers, Intersection};
|
||||
pub use self::more_like_this::{MoreLikeThisQuery, MoreLikeThisQueryBuilder};
|
||||
pub use self::phrase_prefix_query::PhrasePrefixQuery;
|
||||
pub use self::phrase_query::PhraseQuery;
|
||||
pub use self::query::{EnableScoring, Query, QueryClone};
|
||||
pub use self::query_parser::{QueryParser, QueryParserError};
|
||||
@@ -63,6 +65,7 @@ pub use self::union::Union;
|
||||
#[cfg(test)]
|
||||
pub use self::vec_docset::VecDocSet;
|
||||
pub use self::weight::Weight;
|
||||
pub(crate) use self::weight::{for_each_docset, for_each_pruning_scorer, for_each_scorer};
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
|
||||
@@ -44,7 +44,7 @@ impl MoreLikeThisQuery {
|
||||
impl Query for MoreLikeThisQuery {
|
||||
fn weight(&self, enable_scoring: EnableScoring<'_>) -> crate::Result<Box<dyn Weight>> {
|
||||
let searcher = match enable_scoring {
|
||||
EnableScoring::Enabled(searcher) => searcher,
|
||||
EnableScoring::Enabled { searcher, .. } => searcher,
|
||||
EnableScoring::Disabled { .. } => {
|
||||
let err = "MoreLikeThisQuery requires to enable scoring.".to_string();
|
||||
return Err(crate::TantivyError::InvalidArgument(err));
|
||||
|
||||
34
src/query/phrase_prefix_query/mod.rs
Normal file
34
src/query/phrase_prefix_query/mod.rs
Normal file
@@ -0,0 +1,34 @@
|
||||
mod phrase_prefix_query;
|
||||
mod phrase_prefix_scorer;
|
||||
mod phrase_prefix_weight;
|
||||
|
||||
pub use phrase_prefix_query::PhrasePrefixQuery;
|
||||
pub use phrase_prefix_scorer::PhrasePrefixScorer;
|
||||
pub use phrase_prefix_weight::PhrasePrefixWeight;
|
||||
|
||||
fn prefix_end(prefix_start: &[u8]) -> Option<Vec<u8>> {
|
||||
let mut res = prefix_start.to_owned();
|
||||
while !res.is_empty() {
|
||||
let end = res.len() - 1;
|
||||
if res[end] == u8::MAX {
|
||||
res.pop();
|
||||
} else {
|
||||
res[end] += 1;
|
||||
return Some(res);
|
||||
}
|
||||
}
|
||||
None
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
fn test_prefix_end() {
|
||||
assert_eq!(prefix_end(b"aaa"), Some(b"aab".to_vec()));
|
||||
assert_eq!(prefix_end(b"aa\xff"), Some(b"ab".to_vec()));
|
||||
assert_eq!(prefix_end(b"a\xff\xff"), Some(b"b".to_vec()));
|
||||
assert_eq!(prefix_end(b"\xff\xff\xff"), None);
|
||||
}
|
||||
}
|
||||
169
src/query/phrase_prefix_query/phrase_prefix_query.rs
Normal file
169
src/query/phrase_prefix_query/phrase_prefix_query.rs
Normal file
@@ -0,0 +1,169 @@
|
||||
use std::ops::Bound;
|
||||
|
||||
use super::{prefix_end, PhrasePrefixWeight};
|
||||
use crate::query::bm25::Bm25Weight;
|
||||
use crate::query::{EnableScoring, Query, RangeQuery, Weight};
|
||||
use crate::schema::{Field, IndexRecordOption, Term};
|
||||
|
||||
const DEFAULT_MAX_EXPANSIONS: u32 = 50;
|
||||
|
||||
/// `PhrasePrefixQuery` matches a specific sequence of words followed by term of which only a
|
||||
/// prefix is known.
|
||||
///
|
||||
/// For instance the phrase prefix query for `"part t"` will match
|
||||
/// the sentence
|
||||
///
|
||||
/// **Alan just got a part time job.**
|
||||
///
|
||||
/// On the other hand it will not match the sentence.
|
||||
///
|
||||
/// **This is my favorite part of the job.**
|
||||
///
|
||||
/// Using a `PhrasePrefixQuery` on a field requires positions
|
||||
/// to be indexed for this field.
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct PhrasePrefixQuery {
|
||||
field: Field,
|
||||
phrase_terms: Vec<(usize, Term)>,
|
||||
prefix: (usize, Term),
|
||||
max_expansions: u32,
|
||||
}
|
||||
|
||||
impl PhrasePrefixQuery {
|
||||
/// Creates a new `PhrasePrefixQuery` given a list of terms.
|
||||
///
|
||||
/// There must be at least two terms, and all terms
|
||||
/// must belong to the same field.
|
||||
/// Offset for each term will be same as index in the Vector
|
||||
/// The last Term is a prefix and not a full value
|
||||
pub fn new(terms: Vec<Term>) -> PhrasePrefixQuery {
|
||||
let terms_with_offset = terms.into_iter().enumerate().collect();
|
||||
PhrasePrefixQuery::new_with_offset(terms_with_offset)
|
||||
}
|
||||
|
||||
/// Creates a new `PhrasePrefixQuery` given a list of terms and their offsets.
|
||||
///
|
||||
/// Can be used to provide custom offset for each term.
|
||||
pub fn new_with_offset(mut terms: Vec<(usize, Term)>) -> PhrasePrefixQuery {
|
||||
assert!(
|
||||
!terms.is_empty(),
|
||||
"A phrase prefix query is required to have at least one term."
|
||||
);
|
||||
terms.sort_by_key(|&(offset, _)| offset);
|
||||
let field = terms[0].1.field();
|
||||
assert!(
|
||||
terms[1..].iter().all(|term| term.1.field() == field),
|
||||
"All terms from a phrase query must belong to the same field"
|
||||
);
|
||||
PhrasePrefixQuery {
|
||||
field,
|
||||
prefix: terms.pop().unwrap(),
|
||||
phrase_terms: terms,
|
||||
max_expansions: DEFAULT_MAX_EXPANSIONS,
|
||||
}
|
||||
}
|
||||
|
||||
/// Maximum number of terms to which the last provided term will expand.
|
||||
pub fn set_max_expansions(&mut self, value: u32) {
|
||||
self.max_expansions = value;
|
||||
}
|
||||
|
||||
/// The [`Field`] this `PhrasePrefixQuery` is targeting.
|
||||
pub fn field(&self) -> Field {
|
||||
self.field
|
||||
}
|
||||
|
||||
/// `Term`s in the phrase without the associated offsets.
|
||||
pub fn phrase_terms(&self) -> Vec<Term> {
|
||||
// TODO should we include the last term too?
|
||||
self.phrase_terms
|
||||
.iter()
|
||||
.map(|(_, term)| term.clone())
|
||||
.collect::<Vec<Term>>()
|
||||
}
|
||||
|
||||
/// Returns the [`PhrasePrefixWeight`] for the given phrase query given a specific `searcher`.
|
||||
///
|
||||
/// This function is the same as [`Query::weight()`] except it returns
|
||||
/// a specialized type [`PhraseQueryWeight`] instead of a Boxed trait.
|
||||
/// If the query was only one term long, this returns `None` wherease [`Query::weight`]
|
||||
/// returns a boxed [`RangeWeight`]
|
||||
///
|
||||
/// Returns `None`, if phrase_terms is empty, which happens if the phrase prefix query was
|
||||
/// built with a single term.
|
||||
pub(crate) fn phrase_prefix_query_weight(
|
||||
&self,
|
||||
enable_scoring: EnableScoring<'_>,
|
||||
) -> crate::Result<Option<PhrasePrefixWeight>> {
|
||||
if self.phrase_terms.is_empty() {
|
||||
return Ok(None);
|
||||
}
|
||||
let schema = enable_scoring.schema();
|
||||
let field_entry = schema.get_field_entry(self.field);
|
||||
let has_positions = field_entry
|
||||
.field_type()
|
||||
.get_index_record_option()
|
||||
.map(IndexRecordOption::has_positions)
|
||||
.unwrap_or(false);
|
||||
if !has_positions {
|
||||
let field_name = field_entry.name();
|
||||
return Err(crate::TantivyError::SchemaError(format!(
|
||||
"Applied phrase query on field {:?}, which does not have positions indexed",
|
||||
field_name
|
||||
)));
|
||||
}
|
||||
let terms = self.phrase_terms();
|
||||
let bm25_weight_opt = match enable_scoring {
|
||||
EnableScoring::Enabled { searcher, .. } => {
|
||||
Some(Bm25Weight::for_terms(searcher, &terms)?)
|
||||
}
|
||||
EnableScoring::Disabled { .. } => None,
|
||||
};
|
||||
let weight = PhrasePrefixWeight::new(
|
||||
self.phrase_terms.clone(),
|
||||
self.prefix.clone(),
|
||||
bm25_weight_opt,
|
||||
self.max_expansions,
|
||||
);
|
||||
Ok(Some(weight))
|
||||
}
|
||||
}
|
||||
|
||||
impl Query for PhrasePrefixQuery {
|
||||
/// Create the weight associated with a query.
|
||||
///
|
||||
/// See [`Weight`].
|
||||
fn weight(&self, enable_scoring: EnableScoring<'_>) -> crate::Result<Box<dyn Weight>> {
|
||||
if let Some(phrase_weight) = self.phrase_prefix_query_weight(enable_scoring)? {
|
||||
Ok(Box::new(phrase_weight))
|
||||
} else {
|
||||
// There are no prefix. Let's just match the suffix.
|
||||
let end_term = if let Some(end_value) = prefix_end(self.prefix.1.value_bytes()) {
|
||||
let mut end_term = Term::with_capacity(end_value.len());
|
||||
end_term.set_field_and_type(self.field, self.prefix.1.typ());
|
||||
end_term.append_bytes(&end_value);
|
||||
Bound::Excluded(end_term)
|
||||
} else {
|
||||
Bound::Unbounded
|
||||
};
|
||||
|
||||
let mut range_query = RangeQuery::new_term_bounds(
|
||||
enable_scoring
|
||||
.schema()
|
||||
.get_field_name(self.field)
|
||||
.to_owned(),
|
||||
self.prefix.1.typ(),
|
||||
&Bound::Included(self.prefix.1.clone()),
|
||||
&end_term,
|
||||
);
|
||||
range_query.limit(self.max_expansions as u64);
|
||||
range_query.weight(enable_scoring)
|
||||
}
|
||||
}
|
||||
|
||||
fn query_terms<'a>(&'a self, visitor: &mut dyn FnMut(&'a Term, bool)) {
|
||||
for (_, term) in &self.phrase_terms {
|
||||
visitor(term, true);
|
||||
}
|
||||
}
|
||||
}
|
||||
207
src/query/phrase_prefix_query/phrase_prefix_scorer.rs
Normal file
207
src/query/phrase_prefix_query/phrase_prefix_scorer.rs
Normal file
@@ -0,0 +1,207 @@
|
||||
use crate::docset::{DocSet, TERMINATED};
|
||||
use crate::fieldnorm::FieldNormReader;
|
||||
use crate::postings::Postings;
|
||||
use crate::query::bm25::Bm25Weight;
|
||||
use crate::query::phrase_query::{intersection_count, PhraseScorer};
|
||||
use crate::query::Scorer;
|
||||
use crate::{DocId, Score};
|
||||
|
||||
enum PhraseKind<TPostings: Postings> {
|
||||
SinglePrefix {
|
||||
position_offset: u32,
|
||||
postings: TPostings,
|
||||
positions: Vec<u32>,
|
||||
},
|
||||
MultiPrefix(PhraseScorer<TPostings>),
|
||||
}
|
||||
|
||||
impl<TPostings: Postings> PhraseKind<TPostings> {
|
||||
fn get_intersection(&mut self) -> &[u32] {
|
||||
match self {
|
||||
PhraseKind::SinglePrefix {
|
||||
position_offset,
|
||||
postings,
|
||||
positions,
|
||||
} => {
|
||||
if positions.is_empty() {
|
||||
postings.positions_with_offset(*position_offset, positions);
|
||||
}
|
||||
positions
|
||||
}
|
||||
PhraseKind::MultiPrefix(postings) => postings.get_intersection(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<TPostings: Postings> DocSet for PhraseKind<TPostings> {
|
||||
fn advance(&mut self) -> DocId {
|
||||
match self {
|
||||
PhraseKind::SinglePrefix {
|
||||
postings,
|
||||
positions,
|
||||
..
|
||||
} => {
|
||||
positions.clear();
|
||||
postings.advance()
|
||||
}
|
||||
PhraseKind::MultiPrefix(postings) => postings.advance(),
|
||||
}
|
||||
}
|
||||
|
||||
fn doc(&self) -> DocId {
|
||||
match self {
|
||||
PhraseKind::SinglePrefix { postings, .. } => postings.doc(),
|
||||
PhraseKind::MultiPrefix(postings) => postings.doc(),
|
||||
}
|
||||
}
|
||||
|
||||
fn size_hint(&self) -> u32 {
|
||||
match self {
|
||||
PhraseKind::SinglePrefix { postings, .. } => postings.size_hint(),
|
||||
PhraseKind::MultiPrefix(postings) => postings.size_hint(),
|
||||
}
|
||||
}
|
||||
|
||||
fn seek(&mut self, target: DocId) -> DocId {
|
||||
match self {
|
||||
PhraseKind::SinglePrefix {
|
||||
postings,
|
||||
positions,
|
||||
..
|
||||
} => {
|
||||
positions.clear();
|
||||
postings.seek(target)
|
||||
}
|
||||
PhraseKind::MultiPrefix(postings) => postings.seek(target),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<TPostings: Postings> Scorer for PhraseKind<TPostings> {
|
||||
fn score(&mut self) -> Score {
|
||||
match self {
|
||||
PhraseKind::SinglePrefix { positions, .. } => {
|
||||
if positions.is_empty() {
|
||||
0.0
|
||||
} else {
|
||||
1.0
|
||||
}
|
||||
}
|
||||
PhraseKind::MultiPrefix(postings) => postings.score(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub struct PhrasePrefixScorer<TPostings: Postings> {
|
||||
phrase_scorer: PhraseKind<TPostings>,
|
||||
suffixes: Vec<TPostings>,
|
||||
suffix_offset: u32,
|
||||
phrase_count: u32,
|
||||
}
|
||||
|
||||
impl<TPostings: Postings> PhrasePrefixScorer<TPostings> {
|
||||
// If similarity_weight is None, then scoring is disabled.
|
||||
pub fn new(
|
||||
mut term_postings: Vec<(usize, TPostings)>,
|
||||
similarity_weight_opt: Option<Bm25Weight>,
|
||||
fieldnorm_reader: FieldNormReader,
|
||||
suffixes: Vec<TPostings>,
|
||||
suffix_pos: usize,
|
||||
) -> PhrasePrefixScorer<TPostings> {
|
||||
// correct indices so we can merge with our suffix term the PhraseScorer doesn't know about
|
||||
let max_offset = term_postings
|
||||
.iter()
|
||||
.map(|(pos, _)| *pos)
|
||||
.chain(std::iter::once(suffix_pos))
|
||||
.max()
|
||||
.unwrap();
|
||||
|
||||
let phrase_scorer = if term_postings.len() > 1 {
|
||||
PhraseKind::MultiPrefix(PhraseScorer::new_with_offset(
|
||||
term_postings,
|
||||
similarity_weight_opt,
|
||||
fieldnorm_reader,
|
||||
0,
|
||||
1,
|
||||
))
|
||||
} else {
|
||||
let (pos, postings) = term_postings
|
||||
.pop()
|
||||
.expect("PhrasePrefixScorer must have at least two terms");
|
||||
let offset = suffix_pos - pos;
|
||||
PhraseKind::SinglePrefix {
|
||||
position_offset: offset as u32,
|
||||
postings,
|
||||
positions: Vec::with_capacity(100),
|
||||
}
|
||||
};
|
||||
let mut phrase_prefix_scorer = PhrasePrefixScorer {
|
||||
phrase_scorer,
|
||||
suffixes,
|
||||
suffix_offset: (max_offset - suffix_pos) as u32,
|
||||
phrase_count: 0,
|
||||
};
|
||||
if !phrase_prefix_scorer.matches_prefix() {
|
||||
phrase_prefix_scorer.advance();
|
||||
}
|
||||
phrase_prefix_scorer
|
||||
}
|
||||
|
||||
pub fn phrase_count(&self) -> u32 {
|
||||
self.phrase_count
|
||||
}
|
||||
|
||||
fn matches_prefix(&mut self) -> bool {
|
||||
let mut count = 0;
|
||||
let mut positions = Vec::new();
|
||||
let current_doc = self.doc();
|
||||
let pos_matching = self.phrase_scorer.get_intersection();
|
||||
for suffix in &mut self.suffixes {
|
||||
if suffix.doc() > current_doc {
|
||||
continue;
|
||||
}
|
||||
let doc = suffix.seek(current_doc);
|
||||
if doc == current_doc {
|
||||
suffix.positions_with_offset(self.suffix_offset, &mut positions);
|
||||
count += intersection_count(pos_matching, &positions);
|
||||
}
|
||||
}
|
||||
self.phrase_count = count as u32;
|
||||
count != 0
|
||||
}
|
||||
}
|
||||
|
||||
impl<TPostings: Postings> DocSet for PhrasePrefixScorer<TPostings> {
|
||||
fn advance(&mut self) -> DocId {
|
||||
loop {
|
||||
let doc = self.phrase_scorer.advance();
|
||||
if doc == TERMINATED || self.matches_prefix() {
|
||||
return doc;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn seek(&mut self, target: DocId) -> DocId {
|
||||
self.phrase_scorer.seek(target);
|
||||
let doc = self.phrase_scorer.seek(target);
|
||||
if doc == TERMINATED || self.matches_prefix() {
|
||||
return doc;
|
||||
}
|
||||
self.advance()
|
||||
}
|
||||
|
||||
fn doc(&self) -> DocId {
|
||||
self.phrase_scorer.doc()
|
||||
}
|
||||
|
||||
fn size_hint(&self) -> u32 {
|
||||
self.phrase_scorer.size_hint()
|
||||
}
|
||||
}
|
||||
|
||||
impl<TPostings: Postings> Scorer for PhrasePrefixScorer<TPostings> {
|
||||
fn score(&mut self) -> Score {
|
||||
// TODO modify score??
|
||||
self.phrase_scorer.score()
|
||||
}
|
||||
}
|
||||
258
src/query/phrase_prefix_query/phrase_prefix_weight.rs
Normal file
258
src/query/phrase_prefix_query/phrase_prefix_weight.rs
Normal file
@@ -0,0 +1,258 @@
|
||||
use super::{prefix_end, PhrasePrefixScorer};
|
||||
use crate::core::SegmentReader;
|
||||
use crate::fieldnorm::FieldNormReader;
|
||||
use crate::postings::SegmentPostings;
|
||||
use crate::query::bm25::Bm25Weight;
|
||||
use crate::query::explanation::does_not_match;
|
||||
use crate::query::{EmptyScorer, Explanation, Scorer, Weight};
|
||||
use crate::schema::{IndexRecordOption, Term};
|
||||
use crate::{DocId, DocSet, Score};
|
||||
|
||||
pub struct PhrasePrefixWeight {
|
||||
phrase_terms: Vec<(usize, Term)>,
|
||||
prefix: (usize, Term),
|
||||
similarity_weight_opt: Option<Bm25Weight>,
|
||||
max_expansions: u32,
|
||||
}
|
||||
|
||||
impl PhrasePrefixWeight {
|
||||
/// Creates a new phrase weight.
|
||||
/// If `similarity_weight_opt` is None, then scoring is disabled
|
||||
pub fn new(
|
||||
phrase_terms: Vec<(usize, Term)>,
|
||||
prefix: (usize, Term),
|
||||
similarity_weight_opt: Option<Bm25Weight>,
|
||||
max_expansions: u32,
|
||||
) -> PhrasePrefixWeight {
|
||||
PhrasePrefixWeight {
|
||||
phrase_terms,
|
||||
prefix,
|
||||
similarity_weight_opt,
|
||||
max_expansions,
|
||||
}
|
||||
}
|
||||
|
||||
fn fieldnorm_reader(&self, reader: &SegmentReader) -> crate::Result<FieldNormReader> {
|
||||
let field = self.phrase_terms[0].1.field();
|
||||
if self.similarity_weight_opt.is_some() {
|
||||
if let Some(fieldnorm_reader) = reader.fieldnorms_readers().get_field(field)? {
|
||||
return Ok(fieldnorm_reader);
|
||||
}
|
||||
}
|
||||
Ok(FieldNormReader::constant(reader.max_doc(), 1))
|
||||
}
|
||||
|
||||
pub(crate) fn phrase_scorer(
|
||||
&self,
|
||||
reader: &SegmentReader,
|
||||
boost: Score,
|
||||
) -> crate::Result<Option<PhrasePrefixScorer<SegmentPostings>>> {
|
||||
let similarity_weight_opt = self
|
||||
.similarity_weight_opt
|
||||
.as_ref()
|
||||
.map(|similarity_weight| similarity_weight.boost_by(boost));
|
||||
let fieldnorm_reader = self.fieldnorm_reader(reader)?;
|
||||
let mut term_postings_list = Vec::new();
|
||||
if reader.has_deletes() {
|
||||
for &(offset, ref term) in &self.phrase_terms {
|
||||
if let Some(postings) = reader
|
||||
.inverted_index(term.field())?
|
||||
.read_postings(term, IndexRecordOption::WithFreqsAndPositions)?
|
||||
{
|
||||
term_postings_list.push((offset, postings));
|
||||
} else {
|
||||
return Ok(None);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
for &(offset, ref term) in &self.phrase_terms {
|
||||
if let Some(postings) = reader
|
||||
.inverted_index(term.field())?
|
||||
.read_postings_no_deletes(term, IndexRecordOption::WithFreqsAndPositions)?
|
||||
{
|
||||
term_postings_list.push((offset, postings));
|
||||
} else {
|
||||
return Ok(None);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
let inv_index = reader.inverted_index(self.prefix.1.field())?;
|
||||
let mut stream = inv_index.terms().range().ge(self.prefix.1.value_bytes());
|
||||
if let Some(end) = prefix_end(self.prefix.1.value_bytes()) {
|
||||
stream = stream.lt(&end);
|
||||
}
|
||||
|
||||
#[cfg(feature = "quickwit")]
|
||||
{
|
||||
// We don't have this on the fst, hence we end up needing a feature flag.
|
||||
//
|
||||
// This is not a problem however as we enforce the limit below too.
|
||||
// The point of `stream.limit` is to limit the number of term dictionary
|
||||
// blocks being downloaded.
|
||||
stream = stream.limit(self.max_expansions as u64);
|
||||
}
|
||||
|
||||
let mut stream = stream.into_stream()?;
|
||||
|
||||
let mut suffixes = Vec::with_capacity(self.max_expansions as usize);
|
||||
let mut new_term = self.prefix.1.clone();
|
||||
while stream.advance() && (suffixes.len() as u32) < self.max_expansions {
|
||||
new_term.clear_with_type(new_term.typ());
|
||||
new_term.append_bytes(stream.key());
|
||||
if reader.has_deletes() {
|
||||
if let Some(postings) =
|
||||
inv_index.read_postings(&new_term, IndexRecordOption::WithFreqsAndPositions)?
|
||||
{
|
||||
suffixes.push(postings);
|
||||
}
|
||||
} else if let Some(postings) = inv_index
|
||||
.read_postings_no_deletes(&new_term, IndexRecordOption::WithFreqsAndPositions)?
|
||||
{
|
||||
suffixes.push(postings);
|
||||
}
|
||||
}
|
||||
|
||||
Ok(Some(PhrasePrefixScorer::new(
|
||||
term_postings_list,
|
||||
similarity_weight_opt,
|
||||
fieldnorm_reader,
|
||||
suffixes,
|
||||
self.prefix.0,
|
||||
)))
|
||||
}
|
||||
}
|
||||
|
||||
impl Weight for PhrasePrefixWeight {
|
||||
fn scorer(&self, reader: &SegmentReader, boost: Score) -> crate::Result<Box<dyn Scorer>> {
|
||||
if let Some(scorer) = self.phrase_scorer(reader, boost)? {
|
||||
Ok(Box::new(scorer))
|
||||
} else {
|
||||
Ok(Box::new(EmptyScorer))
|
||||
}
|
||||
}
|
||||
|
||||
fn explain(&self, reader: &SegmentReader, doc: DocId) -> crate::Result<Explanation> {
|
||||
let scorer_opt = self.phrase_scorer(reader, 1.0)?;
|
||||
if scorer_opt.is_none() {
|
||||
return Err(does_not_match(doc));
|
||||
}
|
||||
let mut scorer = scorer_opt.unwrap();
|
||||
if scorer.seek(doc) != doc {
|
||||
return Err(does_not_match(doc));
|
||||
}
|
||||
let fieldnorm_reader = self.fieldnorm_reader(reader)?;
|
||||
let fieldnorm_id = fieldnorm_reader.fieldnorm_id(doc);
|
||||
let phrase_count = scorer.phrase_count();
|
||||
let mut explanation = Explanation::new("Phrase Prefix Scorer", scorer.score());
|
||||
if let Some(similarity_weight) = self.similarity_weight_opt.as_ref() {
|
||||
explanation.add_detail(similarity_weight.explain(fieldnorm_id, phrase_count));
|
||||
}
|
||||
Ok(explanation)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use crate::core::Index;
|
||||
use crate::docset::TERMINATED;
|
||||
use crate::query::{EnableScoring, PhrasePrefixQuery, Query};
|
||||
use crate::schema::{Schema, TEXT};
|
||||
use crate::{DocSet, Term};
|
||||
|
||||
pub fn create_index(texts: &[&'static str]) -> crate::Result<Index> {
|
||||
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_for_tests()?;
|
||||
for &text in texts {
|
||||
let doc = doc!(text_field=>text);
|
||||
index_writer.add_document(doc)?;
|
||||
}
|
||||
index_writer.commit()?;
|
||||
}
|
||||
Ok(index)
|
||||
}
|
||||
|
||||
#[test]
|
||||
pub fn test_phrase_count_long() -> crate::Result<()> {
|
||||
let index = create_index(&[
|
||||
"aa bb dd cc",
|
||||
"aa aa bb c dd aa bb cc aa bb dc",
|
||||
" aa bb cd",
|
||||
])?;
|
||||
let schema = index.schema();
|
||||
let text_field = schema.get_field("text").unwrap();
|
||||
let searcher = index.reader()?.searcher();
|
||||
let phrase_query = PhrasePrefixQuery::new(vec![
|
||||
Term::from_field_text(text_field, "aa"),
|
||||
Term::from_field_text(text_field, "bb"),
|
||||
Term::from_field_text(text_field, "c"),
|
||||
]);
|
||||
let enable_scoring = EnableScoring::enabled_from_searcher(&searcher);
|
||||
let phrase_weight = phrase_query
|
||||
.phrase_prefix_query_weight(enable_scoring)
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
let mut phrase_scorer = phrase_weight
|
||||
.phrase_scorer(searcher.segment_reader(0u32), 1.0)?
|
||||
.unwrap();
|
||||
assert_eq!(phrase_scorer.doc(), 1);
|
||||
assert_eq!(phrase_scorer.phrase_count(), 2);
|
||||
assert_eq!(phrase_scorer.advance(), 2);
|
||||
assert_eq!(phrase_scorer.doc(), 2);
|
||||
assert_eq!(phrase_scorer.phrase_count(), 1);
|
||||
assert_eq!(phrase_scorer.advance(), TERMINATED);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test]
|
||||
pub fn test_phrase_count_mid() -> crate::Result<()> {
|
||||
let index = create_index(&["aa dd cc", "aa aa bb c dd aa bb cc aa dc", " aa bb cd"])?;
|
||||
let schema = index.schema();
|
||||
let text_field = schema.get_field("text").unwrap();
|
||||
let searcher = index.reader()?.searcher();
|
||||
let phrase_query = PhrasePrefixQuery::new(vec![
|
||||
Term::from_field_text(text_field, "aa"),
|
||||
Term::from_field_text(text_field, "b"),
|
||||
]);
|
||||
let enable_scoring = EnableScoring::enabled_from_searcher(&searcher);
|
||||
let phrase_weight = phrase_query
|
||||
.phrase_prefix_query_weight(enable_scoring)
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
let mut phrase_scorer = phrase_weight
|
||||
.phrase_scorer(searcher.segment_reader(0u32), 1.0)?
|
||||
.unwrap();
|
||||
assert_eq!(phrase_scorer.doc(), 1);
|
||||
assert_eq!(phrase_scorer.phrase_count(), 2);
|
||||
assert_eq!(phrase_scorer.advance(), 2);
|
||||
assert_eq!(phrase_scorer.doc(), 2);
|
||||
assert_eq!(phrase_scorer.phrase_count(), 1);
|
||||
assert_eq!(phrase_scorer.advance(), TERMINATED);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[test]
|
||||
pub fn test_phrase_count_short() -> crate::Result<()> {
|
||||
let index = create_index(&["aa dd", "aa aa bb c dd aa bb cc aa dc", " aa bb cd"])?;
|
||||
let schema = index.schema();
|
||||
let text_field = schema.get_field("text").unwrap();
|
||||
let searcher = index.reader()?.searcher();
|
||||
let phrase_query = PhrasePrefixQuery::new(vec![Term::from_field_text(text_field, "c")]);
|
||||
let enable_scoring = EnableScoring::enabled_from_searcher(&searcher);
|
||||
assert!(phrase_query
|
||||
.phrase_prefix_query_weight(enable_scoring)
|
||||
.unwrap()
|
||||
.is_none());
|
||||
let weight = phrase_query.weight(enable_scoring).unwrap();
|
||||
let mut phrase_scorer = weight.scorer(searcher.segment_reader(0u32), 1.0)?;
|
||||
assert_eq!(phrase_scorer.doc(), 1);
|
||||
assert_eq!(phrase_scorer.advance(), 2);
|
||||
assert_eq!(phrase_scorer.doc(), 2);
|
||||
assert_eq!(phrase_scorer.advance(), TERMINATED);
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
@@ -3,6 +3,7 @@ mod phrase_scorer;
|
||||
mod phrase_weight;
|
||||
|
||||
pub use self::phrase_query::PhraseQuery;
|
||||
pub(crate) use self::phrase_scorer::intersection_count;
|
||||
pub use self::phrase_scorer::PhraseScorer;
|
||||
pub use self::phrase_weight::PhraseWeight;
|
||||
|
||||
|
||||
@@ -108,7 +108,10 @@ impl PhraseQuery {
|
||||
}
|
||||
let terms = self.phrase_terms();
|
||||
let bm25_weight_opt = match enable_scoring {
|
||||
EnableScoring::Enabled(searcher) => Some(Bm25Weight::for_terms(searcher, &terms)?),
|
||||
EnableScoring::Enabled {
|
||||
statistics_provider,
|
||||
..
|
||||
} => Some(Bm25Weight::for_terms(statistics_provider, &terms)?),
|
||||
EnableScoring::Disabled { .. } => None,
|
||||
};
|
||||
let mut weight = PhraseWeight::new(self.phrase_terms.clone(), bm25_weight_opt);
|
||||
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user