Compare commits

...

3 Commits

Author SHA1 Message Date
Paul Masurel
c8c80d21cb Removing the need for column on Multivalued blabla 2022-11-14 11:46:15 +09:00
Pascal Seitz
a1c1f6a764 Improve position_to_docid, refactor, add tests 2022-11-10 13:56:01 +08:00
Pascal Seitz
605456027d add support for ip range query on multivalue fastfields 2022-11-08 17:45:15 +08:00
9 changed files with 490 additions and 183 deletions

View File

@@ -113,7 +113,7 @@ mod tests {
b.iter(|| {
let mut positions = Vec::new();
column.get_positions_for_value_range(
column.get_docids_for_value_range(
major_item..=major_item,
0..data.len() as u32,
&mut positions,
@@ -129,7 +129,7 @@ mod tests {
b.iter(|| {
let mut positions = Vec::new();
column.get_positions_for_value_range(
column.get_docids_for_value_range(
minor_item..=minor_item,
0..data.len() as u32,
&mut positions,
@@ -145,11 +145,7 @@ mod tests {
b.iter(|| {
let mut positions = Vec::new();
column.get_positions_for_value_range(
0..=u128::MAX,
0..data.len() as u32,
&mut positions,
);
column.get_docids_for_value_range(0..=u128::MAX, 0..data.len() as u32, &mut positions);
positions
});
}

View File

@@ -35,7 +35,7 @@ pub trait Column<T: PartialOrd = u64>: Send + Sync {
///
/// Note that position == docid for single value fast fields
#[inline]
fn get_positions_for_value_range(
fn get_docids_for_value_range(
&self,
value_range: RangeInclusive<T>,
doc_id_range: Range<u32>,
@@ -70,6 +70,11 @@ pub trait Column<T: PartialOrd = u64>: Send + Sync {
/// The number of values in the column.
fn num_vals(&self) -> u32;
/// The number of docs in the column. For single value columns this equals num_vals.
fn num_docs(&self) -> u32 {
self.num_vals()
}
/// 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)))
@@ -222,13 +227,13 @@ where
)
}
fn get_positions_for_value_range(
fn get_docids_for_value_range(
&self,
range: RangeInclusive<Output>,
doc_id_range: Range<u32>,
positions: &mut Vec<u32>,
) {
self.from_column.get_positions_for_value_range(
self.from_column.get_docids_for_value_range(
self.monotonic_mapping.inverse(range.start().clone())
..=self.monotonic_mapping.inverse(range.end().clone()),
doc_id_range,

View File

@@ -306,13 +306,13 @@ impl Column<u128> for CompactSpaceDecompressor {
}
#[inline]
fn get_positions_for_value_range(
fn get_docids_for_value_range(
&self,
value_range: RangeInclusive<u128>,
doc_id_range: Range<u32>,
positions_range: Range<u32>,
positions: &mut Vec<u32>,
) {
self.get_positions_for_value_range(value_range, doc_id_range, positions)
self.get_positions_for_value_range(value_range, positions_range, positions)
}
}
@@ -351,13 +351,13 @@ impl CompactSpaceDecompressor {
pub fn get_positions_for_value_range(
&self,
value_range: RangeInclusive<u128>,
doc_id_range: Range<u32>,
position_range: Range<u32>,
positions: &mut Vec<u32>,
) {
if value_range.start() > value_range.end() {
return;
}
let doc_id_range = doc_id_range.start..doc_id_range.end.min(self.num_vals());
let position_range = position_range.start..position_range.end.min(self.num_vals());
let from_value = *value_range.start();
let to_value = *value_range.end();
assert!(to_value >= from_value);
@@ -390,10 +390,10 @@ impl CompactSpaceDecompressor {
let range = compact_from..=compact_to;
let scan_num_docs = doc_id_range.end - doc_id_range.start;
let scan_num_docs = position_range.end - position_range.start;
let step_size = 4;
let cutoff = doc_id_range.start + scan_num_docs - scan_num_docs % step_size;
let cutoff = position_range.start + scan_num_docs - scan_num_docs % step_size;
let mut push_if_in_range = |idx, val| {
if range.contains(&val) {
@@ -402,7 +402,7 @@ impl CompactSpaceDecompressor {
};
let get_val = |idx| self.params.bit_unpacker.get(idx, &self.data);
// unrolled loop
for idx in (doc_id_range.start..cutoff).step_by(step_size as usize) {
for idx in (position_range.start..cutoff).step_by(step_size as usize) {
let idx1 = idx;
let idx2 = idx + 1;
let idx3 = idx + 2;
@@ -418,7 +418,7 @@ impl CompactSpaceDecompressor {
}
// handle rest
for idx in cutoff..doc_id_range.end {
for idx in cutoff..position_range.end {
push_if_in_range(idx, get_val(idx as u32));
}
}
@@ -704,7 +704,7 @@ mod tests {
doc_id_range: Range<u32>,
) -> Vec<u32> {
let mut positions = Vec::new();
column.get_positions_for_value_range(value_range, doc_id_range, &mut positions);
column.get_docids_for_value_range(value_range, doc_id_range, &mut positions);
positions
}

View File

@@ -218,7 +218,7 @@ mod tests {
.map(|(pos, _)| pos as u32)
.collect();
let mut positions = Vec::new();
reader.get_positions_for_value_range(
reader.get_docids_for_value_range(
data[test_rand_idx]..=data[test_rand_idx],
0..data.len() as u32,
&mut positions,

View File

@@ -119,7 +119,7 @@ fn bench_ip() {
for value in dataset.iter().take(1110).skip(1100).cloned() {
doc_values.clear();
print_time!("get range");
decompressor.get_positions_for_value_range(
decompressor.get_docids_for_value_range(
value..=value,
0..decompressor.num_vals(),
&mut doc_values,

View File

@@ -26,6 +26,22 @@ impl MultiValueIndex {
start..end
}
/// Returns `[start, end)`, such that the values associated with
/// the given documents are `start..end`.
///
/// The passed end range is allowed to be out of bounds.
#[inline]
pub(crate) fn docid_range_to_position_range(&self, range: Range<DocId>) -> Range<u32> {
let end_docid = range.end.min(self.num_docs() - 1) + 1;
let start_docid = range.start.min(end_docid);
let start = self.idx.get_val(start_docid) as u32;
let end = self.idx.get_val(end_docid) as u32;
assert!(start <= end);
start..end
}
/// returns the num of values associated with a doc_id
pub(crate) fn num_vals_for_doc(&self, doc: DocId) -> u32 {
let range = self.range(doc);
@@ -45,6 +61,7 @@ impl MultiValueIndex {
}
/// Converts a list of positions of values in a 1:n index to the corresponding list of DocIds.
/// Positions are converted inplace to docids.
///
/// Since there is no index for value pos -> docid, but docid -> value pos range, we scan the
/// index.
@@ -52,42 +69,55 @@ impl MultiValueIndex {
/// Correctness: positions needs to be sorted. idx_reader needs to contain monotonically
/// increasing positions.
///
///
/// TODO: Instead of a linear scan we can employ a exponential search into binary search to
/// match a docid to its value position.
pub(crate) fn positions_to_docids(&self, docid_start: u32, positions: &[u32]) -> Vec<DocId> {
let mut docs = vec![];
let mut cur_doc = docid_start;
pub(crate) fn positions_to_docids(&self, doc_id_range: Range<u32>, positions: &mut Vec<u32>) {
if positions.is_empty() {
return;
}
let mut cur_doc = doc_id_range.start;
let mut last_doc = None;
for pos in positions {
assert!(self.idx.get_val(doc_id_range.start) as u32 <= positions[0]);
let mut write_doc_pos = 0;
for i in 0..positions.len() {
let pos = positions[i];
loop {
let end = self.idx.get_val(cur_doc + 1) as u32;
if end > *pos {
// avoid duplicates
if Some(cur_doc) == last_doc {
break;
}
docs.push(cur_doc);
if end > pos {
positions[write_doc_pos] = cur_doc;
write_doc_pos += if last_doc == Some(cur_doc) { 0 } else { 1 };
last_doc = Some(cur_doc);
break;
}
cur_doc += 1;
}
}
docs
positions.truncate(write_doc_pos);
}
}
#[cfg(test)]
mod tests {
use std::ops::Range;
use std::sync::Arc;
use fastfield_codecs::IterColumn;
use crate::fastfield::MultiValueIndex;
fn index_to_pos_helper(
index: &MultiValueIndex,
doc_id_range: Range<u32>,
positions: &[u32],
) -> Vec<u32> {
let mut positions = positions.to_vec();
index.positions_to_docids(doc_id_range, &mut positions);
positions
}
#[test]
fn test_positions_to_docid() {
let offsets = vec![0, 10, 12, 15, 22, 23]; // docid values are [0..10, 10..12, 12..15, etc.]
@@ -97,14 +127,14 @@ mod tests {
{
let positions = vec![10u32, 11, 15, 20, 21, 22];
assert_eq!(index.positions_to_docids(0, &positions), vec![1, 3, 4]);
assert_eq!(index.positions_to_docids(1, &positions), vec![1, 3, 4]);
assert_eq!(index.positions_to_docids(0, &[9]), vec![0]);
assert_eq!(index.positions_to_docids(1, &[10]), vec![1]);
assert_eq!(index.positions_to_docids(1, &[11]), vec![1]);
assert_eq!(index.positions_to_docids(2, &[12]), vec![2]);
assert_eq!(index.positions_to_docids(2, &[12, 14]), vec![2]);
assert_eq!(index.positions_to_docids(2, &[12, 14, 15]), vec![2, 3]);
assert_eq!(index_to_pos_helper(&index, 0..5, &positions), vec![1, 3, 4]);
assert_eq!(index_to_pos_helper(&index, 1..5, &positions), vec![1, 3, 4]);
assert_eq!(index_to_pos_helper(&index, 0..5, &[9]), vec![0]);
assert_eq!(index_to_pos_helper(&index, 1..5, &[10]), vec![1]);
assert_eq!(index_to_pos_helper(&index, 1..5, &[11]), vec![1]);
assert_eq!(index_to_pos_helper(&index, 2..5, &[12]), vec![2]);
assert_eq!(index_to_pos_helper(&index, 2..5, &[12, 14]), vec![2]);
assert_eq!(index_to_pos_helper(&index, 2..5, &[12, 14, 15]), vec![2, 3]);
}
}
}

View File

@@ -14,6 +14,7 @@ use crate::DocId;
/// The `vals_reader` will access the concatenated list of all
/// values for all reader.
/// The `idx_reader` associated, for each document, the index of its first value.
/// Stores the start position for each document.
#[derive(Clone)]
pub struct MultiValuedFastFieldReader<Item: FastValue> {
idx_reader: MultiValueIndex,
@@ -107,6 +108,22 @@ impl<T: MonotonicallyMappableToU128> MultiValuedU128FastFieldReader<T> {
}
}
#[inline]
fn get_docids_for_value_range(
&self,
value_range: RangeInclusive<T>,
doc_id_range: Range<u32>,
positions: &mut Vec<u32>,
) {
let position_range = self
.get_index_reader()
.docid_range_to_position_range(doc_id_range.clone());
self.vals_reader
.get_docids_for_value_range(value_range, position_range, positions);
self.idx_reader.positions_to_docids(doc_id_range, positions);
}
/// Returns the array of values associated to the given `doc`.
#[inline]
pub fn get_first_val(&self, doc: DocId) -> Option<T> {
@@ -138,19 +155,6 @@ impl<T: MonotonicallyMappableToU128> MultiValuedU128FastFieldReader<T> {
self.get_vals_for_range(range, vals);
}
/// Returns all docids which are in the provided value range
pub fn get_positions_for_value_range(
&self,
value_range: RangeInclusive<T>,
doc_id_range: Range<u32>,
) -> Vec<DocId> {
let mut positions = Vec::new();
self.vals_reader
.get_positions_for_value_range(value_range, doc_id_range, &mut positions);
self.idx_reader.positions_to_docids(0, &positions)
}
/// Iterates over all elements in the fast field
pub fn iter(&self) -> impl Iterator<Item = T> + '_ {
self.vals_reader.iter()
@@ -180,7 +184,7 @@ impl<T: MonotonicallyMappableToU128> MultiValuedU128FastFieldReader<T> {
self.idx_reader.num_vals_for_doc(doc)
}
/// Returns the overall number of values in this field.
/// Returns the overall number of values in this field. It does not include deletes.
#[inline]
pub fn total_num_vals(&self) -> u64 {
assert_eq!(

View File

@@ -805,7 +805,7 @@ mod tests {
use std::collections::{HashMap, HashSet};
use std::net::Ipv6Addr;
use fastfield_codecs::MonotonicallyMappableToU128;
use fastfield_codecs::{Column, MonotonicallyMappableToU128};
use proptest::prelude::*;
use proptest::prop_oneof;
use proptest::strategy::Strategy;
@@ -1591,6 +1591,25 @@ mod tests {
(existing_ids, deleted_ids)
}
fn get_id_list(ops: &[IndexingOp]) -> Vec<u64> {
let mut id_list = Vec::new();
for &op in ops {
match op {
IndexingOp::AddDoc { id } => {
id_list.push(id);
}
IndexingOp::DeleteDoc { id } => {
id_list.retain(|el| *el != id);
}
IndexingOp::DeleteDocQuery { id } => {
id_list.retain(|el| *el != id);
}
_ => {}
}
}
id_list
}
fn test_operation_strategy(
ops: &[IndexingOp],
sort_index: bool,
@@ -1600,7 +1619,9 @@ mod tests {
let ip_field = schema_builder.add_ip_addr_field("ip", FAST | INDEXED | STORED);
let ips_field = schema_builder.add_ip_addr_field(
"ips",
IpAddrOptions::default().set_fast(Cardinality::MultiValues),
IpAddrOptions::default()
.set_fast(Cardinality::MultiValues)
.set_indexed(),
);
let id_field = schema_builder.add_u64_field("id", FAST | INDEXED | STORED);
let i64_field = schema_builder.add_i64_field("i64", INDEXED);
@@ -1665,11 +1686,13 @@ mod tests {
// rotate right
let multi_text_field_text3 = "test3 test1 test2 test3 test1 test2";
let ip_from_id = |id| Ipv6Addr::from_u128(id as u128);
for &op in ops {
match op {
IndexingOp::AddDoc { id } => {
let facet = Facet::from(&("/cola/".to_string() + &id.to_string()));
let ip_from_id = Ipv6Addr::from_u128(id as u128);
let ip = ip_from_id(id);
if !ip_exists(id) {
// every 3rd doc has no ip field
@@ -1693,9 +1716,9 @@ mod tests {
} else {
index_writer.add_document(doc!(id_field=>id,
bytes_field => id.to_le_bytes().as_slice(),
ip_field => ip_from_id,
ips_field => ip_from_id,
ips_field => ip_from_id,
ip_field => ip,
ips_field => ip,
ips_field => ip,
multi_numbers=> id,
multi_numbers => id,
bool_field => (id % 2u64) != 0,
@@ -1738,6 +1761,7 @@ mod tests {
index_writer.commit()?;
let searcher = index.reader()?.searcher();
let num_segments_before_merge = searcher.segment_readers().len();
if force_end_merge {
index_writer.wait_merging_threads()?;
let mut index_writer = index.writer_for_tests()?;
@@ -1749,6 +1773,7 @@ mod tests {
assert!(index_writer.wait_merging_threads().is_ok());
}
}
let num_segments_after_merge = searcher.segment_readers().len();
old_reader.reload()?;
let old_searcher = old_reader.searcher();
@@ -1776,6 +1801,22 @@ mod tests {
.collect();
let (expected_ids_and_num_occurrences, deleted_ids) = expected_ids(ops);
let id_list = get_id_list(ops);
// multivalue fast field content
let mut all_ips = Vec::new();
let mut num_ips = 0;
for segment_reader in searcher.segment_readers().iter() {
let ip_reader = segment_reader.fast_fields().ip_addrs(ips_field).unwrap();
for doc in segment_reader.doc_ids_alive() {
let mut vals = vec![];
ip_reader.get_vals(doc, &mut vals);
all_ips.extend_from_slice(&vals);
}
num_ips += ip_reader.total_num_vals();
}
let num_docs_expected = expected_ids_and_num_occurrences
.iter()
.map(|(_, id_occurrences)| *id_occurrences as usize)
@@ -1797,6 +1838,30 @@ mod tests {
.collect::<HashSet<_>>()
);
if force_end_merge && num_segments_before_merge > 1 && num_segments_after_merge == 1 {
let mut expected_multi_ips: Vec<_> = id_list
.iter()
.filter(|id| ip_exists(**id))
.flat_map(|id| vec![ip_from_id(*id), ip_from_id(*id)])
.collect();
assert_eq!(num_ips, expected_multi_ips.len() as u64);
expected_multi_ips.sort();
all_ips.sort();
assert_eq!(expected_multi_ips, all_ips);
// Test fastfield num_docs
let num_docs: usize = searcher
.segment_readers()
.iter()
.map(|segment_reader| {
let ff_reader = segment_reader.fast_fields().ip_addrs(ips_field).unwrap();
ff_reader.num_docs() as usize
})
.sum();
assert_eq!(num_docs, num_docs_expected);
}
// Load all ips addr
let ips: HashSet<Ipv6Addr> = searcher
.segment_readers()
@@ -2000,6 +2065,51 @@ mod tests {
assert_eq!(do_search_ip_field(&format!("\"{}\"", ip_addr)), count);
}
}
// assert data is like expected
//
for (existing_id, count) in expected_ids_and_num_occurrences.iter().take(10) {
let (existing_id, count) = (*existing_id, *count);
if !ip_exists(existing_id) {
continue;
}
let gen_query_inclusive = |field: &str, from: Ipv6Addr, to: Ipv6Addr| {
format!("{}:[{} TO {}]", field, &from.to_string(), &to.to_string())
};
let ip = ip_from_id(existing_id);
let do_search_ip_field = |term: &str| do_search(term, ip_field).len() as u64;
// Range query on single value field
// let query = gen_query_inclusive("ip", ip, ip);
// assert_eq!(do_search_ip_field(&query), count);
// Range query on multi value field
let query = gen_query_inclusive("ips", ip, ip);
assert_eq!(do_search_ip_field(&query), count);
}
// ip range query on fast field
//
for (existing_id, count) in expected_ids_and_num_occurrences.iter().take(10) {
let (existing_id, count) = (*existing_id, *count);
if !ip_exists(existing_id) {
continue;
}
let gen_query_inclusive = |field: &str, from: Ipv6Addr, to: Ipv6Addr| {
format!("{}:[{} TO {}]", field, &from.to_string(), &to.to_string())
};
let ip = ip_from_id(existing_id);
let do_search_ip_field = |term: &str| do_search(term, ip_field).len() as u64;
// Range query on single value field
// let query = gen_query_inclusive("ip", ip, ip);
// assert_eq!(do_search_ip_field(&query), count);
// Range query on multi value field
let query = gen_query_inclusive("ips", ip, ip);
assert_eq!(do_search_ip_field(&query), count);
}
// test facets
for segment_reader in searcher.segment_readers().iter() {
let mut facet_reader = segment_reader.facet_reader(facet_field).unwrap();
@@ -2021,6 +2131,40 @@ mod tests {
Ok(())
}
#[test]
fn test_ip_range_query_multivalue_bug() {
assert!(test_operation_strategy(
&[
IndexingOp::AddDoc { id: 2 },
IndexingOp::Commit,
IndexingOp::AddDoc { id: 1 },
IndexingOp::AddDoc { id: 1 },
IndexingOp::Commit,
IndexingOp::Merge
],
true,
false
)
.is_ok());
}
#[test]
fn test_ff_num_ips_regression() {
assert!(test_operation_strategy(
&[
IndexingOp::AddDoc { id: 13 },
IndexingOp::AddDoc { id: 1 },
IndexingOp::Commit,
IndexingOp::DeleteDocQuery { id: 13 },
IndexingOp::AddDoc { id: 1 },
IndexingOp::Commit,
],
false,
true
)
.is_ok());
}
#[test]
fn test_minimal() {
assert!(test_operation_strategy(
@@ -2030,7 +2174,7 @@ mod tests {
IndexingOp::DeleteDoc { id: 13 }
],
true,
false
true
)
.is_ok());

View File

@@ -11,6 +11,7 @@ use fastfield_codecs::{Column, MonotonicallyMappableToU128};
use super::range_query::map_bound;
use super::{ConstScorer, Explanation, Scorer, Weight};
use crate::fastfield::MultiValuedU128FastFieldReader;
use crate::schema::{Cardinality, Field};
use crate::{DocId, DocSet, Score, SegmentReader, TantivyError, TERMINATED};
@@ -43,16 +44,29 @@ impl Weight for IPFastFieldRangeWeight {
let field_type = reader.schema().get_field_entry(self.field).field_type();
match field_type.fastfield_cardinality().unwrap() {
Cardinality::SingleValue => {
let ip_addr_fast_field = reader.fast_fields().ip_addr(self.field)?;
let ip_addr_fast_field: Arc<dyn Column<Ipv6Addr>> =
reader.fast_fields().ip_addr(self.field)?;
let value_range = bound_to_value_range(
&self.left_bound,
&self.right_bound,
ip_addr_fast_field.as_ref(),
ip_addr_fast_field.min_value(),
ip_addr_fast_field.max_value(),
);
let docset = IpRangeDocSet::new(value_range, ip_addr_fast_field);
Ok(Box::new(ConstScorer::new(docset, boost)))
}
Cardinality::MultiValues => unimplemented!(),
Cardinality::MultiValues => {
let ip_addr_fast_field: MultiValuedU128FastFieldReader<Ipv6Addr> =
reader.fast_fields().ip_addrs(self.field)?;
let value_range = bound_to_value_range(
&self.left_bound,
&self.right_bound,
ip_addr_fast_field.min_value(),
ip_addr_fast_field.max_value(),
);
let docset = IpRangeDocSet::new(value_range, Arc::new(ip_addr_fast_field));
Ok(Box::new(ConstScorer::new(docset, boost)))
}
}
}
@@ -73,18 +87,19 @@ impl Weight for IPFastFieldRangeWeight {
fn bound_to_value_range(
left_bound: &Bound<Ipv6Addr>,
right_bound: &Bound<Ipv6Addr>,
column: &dyn Column<Ipv6Addr>,
min_value: Ipv6Addr,
max_value: Ipv6Addr,
) -> RangeInclusive<Ipv6Addr> {
let start_value = match left_bound {
Bound::Included(ip_addr) => *ip_addr,
Bound::Excluded(ip_addr) => Ipv6Addr::from(ip_addr.to_u128() + 1),
Bound::Unbounded => column.min_value(),
Bound::Unbounded => min_value,
};
let end_value = match right_bound {
Bound::Included(ip_addr) => *ip_addr,
Bound::Excluded(ip_addr) => Ipv6Addr::from(ip_addr.to_u128() - 1),
Bound::Unbounded => column.max_value(),
Bound::Unbounded => max_value,
};
start_value..=end_value
}
@@ -109,22 +124,23 @@ impl VecCursor {
fn current(&self) -> Option<u32> {
self.docs.get(self.current_pos).map(|el| *el as u32)
}
fn get_cleared_data(&mut self) -> &mut Vec<u32> {
self.docs.clear();
self.current_pos = 0;
&mut self.docs
}
fn last_value(&self) -> Option<u32> {
self.docs.iter().last().cloned()
}
fn is_empty(&self) -> bool {
self.current_pos >= self.docs.len()
}
}
struct IpRangeDocSet {
struct IpRangeDocSet<T> {
/// The range filter on the values.
value_range: RangeInclusive<Ipv6Addr>,
ip_addr_fast_field: Arc<dyn Column<Ipv6Addr>>,
ip_addrs: T,
/// The next docid start range to fetch (inclusive).
next_fetch_start: u32,
/// Number of docs range checked in a batch.
@@ -141,18 +157,17 @@ struct IpRangeDocSet {
last_seek_pos_opt: Option<u32>,
}
const DEFALT_FETCH_HORIZON: u32 = 128;
impl IpRangeDocSet {
fn new(
value_range: RangeInclusive<Ipv6Addr>,
ip_addr_fast_field: Arc<dyn Column<Ipv6Addr>>,
) -> Self {
const DEFAULT_FETCH_HORIZON: u32 = 128;
impl<T> IpRangeDocSet<T>
where Self: SingleOrMultivalued
{
fn new(value_range: RangeInclusive<Ipv6Addr>, ip_addrs: T) -> Self {
let mut ip_range_docset = Self {
value_range,
ip_addr_fast_field,
ip_addrs,
loaded_docs: VecCursor::new(),
next_fetch_start: 0,
fetch_horizon: DEFALT_FETCH_HORIZON,
fetch_horizon: DEFAULT_FETCH_HORIZON,
last_seek_pos_opt: None,
};
ip_range_docset.reset_fetch_range();
@@ -161,7 +176,7 @@ impl IpRangeDocSet {
}
fn reset_fetch_range(&mut self) {
self.fetch_horizon = DEFALT_FETCH_HORIZON;
self.fetch_horizon = DEFAULT_FETCH_HORIZON;
}
/// Returns true if more data could be fetched
@@ -185,36 +200,72 @@ impl IpRangeDocSet {
true
}
}
}
/// Fetches a block for docid range [next_fetch_start .. next_fetch_start + HORIZON]
trait SingleOrMultivalued {
fn num_docs(&self) -> u32;
fn fetch_horizon(&mut self, horizon: u32) -> bool {
let mut finished_to_end = false;
// Have different implem for single value and multivalue
todo!();
// let mut finished_to_end = false;
let limit = self.ip_addr_fast_field.num_vals();
let mut end = self.next_fetch_start + horizon;
if end >= limit {
end = limit;
finished_to_end = true;
}
// let limit = self.num_docs();
// let mut end = self.next_fetch_start + horizon;
// if end >= limit {
// end = limit;
// finished_to_end = true;
// }
let data = self.loaded_docs.get_cleared_data();
self.ip_addr_fast_field.get_positions_for_value_range(
self.value_range.clone(),
self.next_fetch_start..end,
data,
);
self.next_fetch_start = end;
finished_to_end
// let last_loaded_docs_val = self
// .is_multivalue
// .then(|| self.loaded_docs.last_value())
// .flatten();
// let last_loaded_docs_val =
// if self.is_multivalue {
// self.loaded_docs.last_value()
// } else {
// None
// };
// let loaded_docs_data = self.loaded_docs.get_cleared_data();
// self.ip_addr_fast_field.get_docids_for_value_range(
// self.value_range.clone(),
// self.next_fetch_start..end,
// loaded_docs_data,
// );
// // In case of multivalues, we may have an overlap of the same docid between fetching
// blocks if let Some(last_value) = last_loaded_docs_val {
// while self.loaded_docs.current() == Some(last_value) {
// self.loaded_docs.next();
// }
// }
// self.next_fetch_start = end;
// finished_to_end
}
}
impl DocSet for IpRangeDocSet {
impl SingleOrMultivalued for IpRangeDocSet<Arc<dyn Column<Ipv6Addr>>> {
fn num_docs(&self) -> u32 {
self.ip_addrs.num_docs()
}
}
impl SingleOrMultivalued for IpRangeDocSet<Arc<MultiValuedU128FastFieldReader<Ipv6Addr>>> {
fn num_docs(&self) -> u32 {
self.ip_addrs.get_index_reader().num_docs()
}
}
impl<T: Send> DocSet for IpRangeDocSet<T>
where Self: SingleOrMultivalued
{
#[inline]
fn advance(&mut self) -> DocId {
if let Some(docid) = self.loaded_docs.next() {
docid as u32
} else {
if self.next_fetch_start >= self.ip_addr_fast_field.num_vals() as u32 {
if self.next_fetch_start >= self.num_docs() as u32 {
return TERMINATED;
}
self.fetch_block();
@@ -269,7 +320,7 @@ mod tests {
use super::*;
use crate::collector::Count;
use crate::query::QueryParser;
use crate::schema::{Schema, FAST, INDEXED, STORED, STRING};
use crate::schema::{IpAddrOptions, Schema, FAST, INDEXED, STORED, STRING};
use crate::Index;
#[derive(Clone, Debug)]
@@ -280,12 +331,13 @@ mod tests {
fn operation_strategy() -> impl Strategy<Value = Doc> {
prop_oneof![
(0u64..100u64).prop_map(doc_from_id_1),
(1u64..100u64).prop_map(doc_from_id_2),
(0u64..10_000u64).prop_map(doc_from_id_1),
(1u64..10_000u64).prop_map(doc_from_id_2),
]
}
pub fn doc_from_id_1(id: u64) -> Doc {
let id = id * 1000;
Doc {
// ip != id
id: id.to_string(),
@@ -293,6 +345,7 @@ mod tests {
}
}
fn doc_from_id_2(id: u64) -> Doc {
let id = id * 1000;
Doc {
// ip != id
id: (id - 1).to_string(),
@@ -310,6 +363,12 @@ mod tests {
#[test]
fn ip_range_regression1_test() {
let ops = vec![doc_from_id_1(0)];
assert!(test_ip_range_for_docs(ops).is_ok());
}
#[test]
fn ip_range_regression2_test() {
let ops = vec![
doc_from_id_1(52),
doc_from_id_1(63),
@@ -321,14 +380,20 @@ mod tests {
}
#[test]
fn ip_range_regression2_test() {
let ops = vec![doc_from_id_1(0)];
fn ip_range_regression3_test() {
let ops = vec![doc_from_id_1(1), doc_from_id_1(2), doc_from_id_1(3)];
assert!(test_ip_range_for_docs(ops).is_ok());
}
pub fn create_index_from_docs(docs: &[Doc]) -> Index {
let mut schema_builder = Schema::builder();
let ip_field = schema_builder.add_ip_addr_field("ip", INDEXED | STORED | FAST);
let ips_field = schema_builder.add_ip_addr_field(
"ips",
IpAddrOptions::default()
.set_fast(Cardinality::MultiValues)
.set_indexed(),
);
let text_field = schema_builder.add_text_field("id", STRING | STORED);
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
@@ -338,6 +403,8 @@ mod tests {
for doc in docs.iter() {
index_writer
.add_document(doc!(
ips_field => doc.ip,
ips_field => doc.ip,
ip_field => doc.ip,
text_field => doc.id.to_string(),
))
@@ -361,8 +428,8 @@ mod tests {
.unwrap()
};
let gen_query_inclusive = |from: Ipv6Addr, to: Ipv6Addr| {
format!("ip:[{} TO {}]", &from.to_string(), &to.to_string())
let gen_query_inclusive = |field: &str, from: Ipv6Addr, to: Ipv6Addr| {
format!("{}:[{} TO {}]", field, &from.to_string(), &to.to_string())
};
let test_sample = |sample_docs: Vec<Doc>| {
@@ -373,7 +440,10 @@ mod tests {
.filter(|doc| (ips[0]..=ips[1]).contains(&doc.ip))
.count();
let query = gen_query_inclusive(ips[0], ips[1]);
let query = gen_query_inclusive("ip", ips[0], ips[1]);
assert_eq!(get_num_hits(query_from_text(&query)), expected_num_hits);
let query = gen_query_inclusive("ips", ips[0], ips[1]);
assert_eq!(get_num_hits(query_from_text(&query)), expected_num_hits);
// Intersection search
@@ -382,7 +452,20 @@ mod tests {
.iter()
.filter(|doc| (ips[0]..=ips[1]).contains(&doc.ip) && doc.id == id_filter)
.count();
let query = format!("{} AND id:{}", query, &id_filter);
let query = format!(
"{} AND id:{}",
gen_query_inclusive("ip", ips[0], ips[1]),
&id_filter
);
assert_eq!(get_num_hits(query_from_text(&query)), expected_num_hits);
// Intersection search on multivalue ip field
let id_filter = sample_docs[0].id.to_string();
let query = format!(
"{} AND id:{}",
gen_query_inclusive("ips", ips[0], ips[1]),
&id_filter
);
assert_eq!(get_num_hits(query_from_text(&query)), expected_num_hits);
};
@@ -402,7 +485,8 @@ mod tests {
#[cfg(all(test, feature = "unstable"))]
mod bench {
use rand::{thread_rng, Rng};
use rand::rngs::StdRng;
use rand::{Rng, SeedableRng};
use test::Bencher;
use super::tests::*;
@@ -412,7 +496,7 @@ mod bench {
use crate::Index;
fn get_index_0_to_100() -> Index {
let mut rng = thread_rng();
let mut rng = StdRng::from_seed([1u8; 32]);
let num_vals = 100_000;
let docs: Vec<_> = (0..num_vals)
.map(|_i| {
@@ -424,8 +508,10 @@ mod bench {
"many".to_string() // 90%
};
Doc {
id: id,
id,
// Multiply by 1000, so that we create many buckets in the compact space
// The benches depend on this range to select n-percent of elements with the
// methods below.
ip: Ipv6Addr::from_u128(rng.gen_range(0..100) * 1000),
}
})
@@ -434,22 +520,42 @@ mod bench {
let index = create_index_from_docs(&docs);
index
}
fn get_90_percent() -> RangeInclusive<Ipv6Addr> {
let start = Ipv6Addr::from_u128(0);
let end = Ipv6Addr::from_u128(90 * 1000);
start..=end
}
fn get_10_percent() -> RangeInclusive<Ipv6Addr> {
let start = Ipv6Addr::from_u128(0);
let end = Ipv6Addr::from_u128(10 * 1000);
start..=end
}
fn get_1_percent() -> RangeInclusive<Ipv6Addr> {
let start = Ipv6Addr::from_u128(10 * 1000);
let end = Ipv6Addr::from_u128(10 * 1000);
start..=end
}
fn excute_query(
start_inclusive: Ipv6Addr,
end_inclusive: Ipv6Addr,
field: &str,
ip_range: RangeInclusive<Ipv6Addr>,
suffix: &str,
index: &Index,
) -> usize {
let gen_query_inclusive = |from: Ipv6Addr, to: Ipv6Addr| {
let gen_query_inclusive = |from: &Ipv6Addr, to: &Ipv6Addr| {
format!(
"ip:[{} TO {}] {}",
"{}:[{} TO {}] {}",
field,
&from.to_string(),
&to.to_string(),
suffix
)
};
let query = gen_query_inclusive(start_inclusive, end_inclusive);
let query = gen_query_inclusive(ip_range.start(), ip_range.end());
let query_from_text = |text: &str| {
QueryParser::for_index(&index, vec![])
.parse_query(text)
@@ -465,131 +571,153 @@ mod bench {
fn bench_ip_range_hit_90_percent(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| {
let start = Ipv6Addr::from_u128(0);
let end = Ipv6Addr::from_u128(90 * 1000);
excute_query(start, end, "", &index)
});
bench.iter(|| excute_query("ip", get_90_percent(), "", &index));
}
#[bench]
fn bench_ip_range_hit_10_percent(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| {
let start = Ipv6Addr::from_u128(0);
let end = Ipv6Addr::from_u128(10 * 1000);
excute_query(start, end, "", &index)
});
bench.iter(|| excute_query("ip", get_10_percent(), "", &index));
}
#[bench]
fn bench_ip_range_hit_1_percent(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| {
let start = Ipv6Addr::from_u128(10 * 1000);
let end = Ipv6Addr::from_u128(10 * 1000);
excute_query(start, end, "", &index)
});
bench.iter(|| excute_query("ip", get_1_percent(), "", &index));
}
#[bench]
fn bench_ip_range_hit_10_percent_intersect_with_10_percent(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| {
let start = Ipv6Addr::from_u128(0);
let end = Ipv6Addr::from_u128(10 * 1000);
excute_query(start, end, "AND id:few", &index)
});
bench.iter(|| excute_query("ip", get_10_percent(), "AND id:few", &index));
}
#[bench]
fn bench_ip_range_hit_1_percent_intersect_with_10_percent(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| {
let start = Ipv6Addr::from_u128(10 * 1000);
let end = Ipv6Addr::from_u128(10 * 1000);
excute_query(start, end, "AND id:few", &index)
});
bench.iter(|| excute_query("ip", get_1_percent(), "AND id:few", &index));
}
#[bench]
fn bench_ip_range_hit_1_percent_intersect_with_90_percent(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| {
let start = Ipv6Addr::from_u128(10 * 1000);
let end = Ipv6Addr::from_u128(10 * 1000);
excute_query(start, end, "AND id:many", &index)
});
bench.iter(|| excute_query("ip", get_1_percent(), "AND id:many", &index));
}
#[bench]
fn bench_ip_range_hit_1_percent_intersect_with_1_percent(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| {
let start = Ipv6Addr::from_u128(10 * 1000);
let end = Ipv6Addr::from_u128(10 * 1000);
excute_query(start, end, "AND id:veryfew", &index)
});
bench.iter(|| excute_query("ip", get_1_percent(), "AND id:veryfew", &index));
}
#[bench]
fn bench_ip_range_hit_10_percent_intersect_with_90_percent(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| {
let start = Ipv6Addr::from_u128(0);
let end = Ipv6Addr::from_u128(10 * 1000);
excute_query(start, end, "AND id:many", &index)
});
bench.iter(|| excute_query("ip", get_10_percent(), "AND id:many", &index));
}
#[bench]
fn bench_ip_range_hit_90_percent_intersect_with_90_percent(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| {
let start = Ipv6Addr::from_u128(0);
let end = Ipv6Addr::from_u128(90 * 1000);
excute_query(start, end, "AND id:many", &index)
});
bench.iter(|| excute_query("ip", get_90_percent(), "AND id:many", &index));
}
#[bench]
fn bench_ip_range_hit_90_percent_intersect_with_10_percent(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| {
let start = Ipv6Addr::from_u128(0);
let end = Ipv6Addr::from_u128(90 * 1000);
excute_query(start, end, "AND id:few", &index)
});
bench.iter(|| excute_query("ip", get_90_percent(), "AND id:few", &index));
}
#[bench]
fn bench_ip_range_hit_90_percent_intersect_with_1_percent(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| {
let start = Ipv6Addr::from_u128(0);
let end = Ipv6Addr::from_u128(90 * 1000);
bench.iter(|| excute_query("ip", get_90_percent(), "AND id:veryfew", &index));
}
excute_query(start, end, "AND id:veryfew", &index)
});
#[bench]
fn bench_ip_range_hit_90_percent_multi(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| excute_query("ips", get_90_percent(), "", &index));
}
#[bench]
fn bench_ip_range_hit_10_percent_multi(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| excute_query("ips", get_10_percent(), "", &index));
}
#[bench]
fn bench_ip_range_hit_1_percent_multi(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| excute_query("ips", get_1_percent(), "", &index));
}
#[bench]
fn bench_ip_range_hit_10_percent_intersect_with_10_percent_multi(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| excute_query("ips", get_10_percent(), "AND id:few", &index));
}
#[bench]
fn bench_ip_range_hit_1_percent_intersect_with_10_percent_multi(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| excute_query("ips", get_1_percent(), "AND id:few", &index));
}
#[bench]
fn bench_ip_range_hit_1_percent_intersect_with_90_percent_multi(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| excute_query("ips", get_1_percent(), "AND id:many", &index));
}
#[bench]
fn bench_ip_range_hit_1_percent_intersect_with_1_percent_multi(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| excute_query("ips", get_1_percent(), "AND id:veryfew", &index));
}
#[bench]
fn bench_ip_range_hit_10_percent_intersect_with_90_percent_multi(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| excute_query("ips", get_10_percent(), "AND id:many", &index));
}
#[bench]
fn bench_ip_range_hit_90_percent_intersect_with_90_percent_multi(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| excute_query("ips", get_90_percent(), "AND id:many", &index));
}
#[bench]
fn bench_ip_range_hit_90_percent_intersect_with_10_percent_multi(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| excute_query("ips", get_90_percent(), "AND id:few", &index));
}
#[bench]
fn bench_ip_range_hit_90_percent_intersect_with_1_percent_multi(bench: &mut Bencher) {
let index = get_index_0_to_100();
bench.iter(|| excute_query("ips", get_90_percent(), "AND id:veryfew", &index));
}
}