Change the project to become a workspace with milli as a default-member

This commit is contained in:
Clément Renault
2021-02-12 16:15:09 +01:00
parent d450b971f9
commit e8639517da
56 changed files with 1053 additions and 2617 deletions

View File

@ -0,0 +1,756 @@
use std::collections::HashMap;
use std::fmt::Debug;
use std::ops::Bound::{self, Included, Excluded};
use std::str::FromStr;
use anyhow::Context;
use either::Either;
use heed::types::{ByteSlice, DecodeIgnore};
use log::debug;
use num_traits::Bounded;
use pest::error::{Error as PestError, ErrorVariant};
use pest::iterators::{Pair, Pairs};
use pest::Parser;
use roaring::RoaringBitmap;
use crate::facet::FacetType;
use crate::heed_codec::facet::FacetValueStringCodec;
use crate::heed_codec::facet::{FacetLevelValueI64Codec, FacetLevelValueF64Codec};
use crate::{Index, FieldId, FieldsIdsMap, CboRoaringBitmapCodec};
use super::FacetRange;
use super::parser::Rule;
use super::parser::{PREC_CLIMBER, FilterParser};
use self::FacetCondition::*;
use self::FacetNumberOperator::*;
#[derive(Debug, Copy, Clone, PartialEq)]
pub enum FacetNumberOperator<T> {
GreaterThan(T),
GreaterThanOrEqual(T),
Equal(T),
NotEqual(T),
LowerThan(T),
LowerThanOrEqual(T),
Between(T, T),
}
impl<T> FacetNumberOperator<T> {
/// This method can return two operations in case it must express
/// an OR operation for the between case (i.e. `TO`).
fn negate(self) -> (Self, Option<Self>) {
match self {
GreaterThan(x) => (LowerThanOrEqual(x), None),
GreaterThanOrEqual(x) => (LowerThan(x), None),
Equal(x) => (NotEqual(x), None),
NotEqual(x) => (Equal(x), None),
LowerThan(x) => (GreaterThanOrEqual(x), None),
LowerThanOrEqual(x) => (GreaterThan(x), None),
Between(x, y) => (LowerThan(x), Some(GreaterThan(y))),
}
}
}
#[derive(Debug, Clone, PartialEq)]
pub enum FacetStringOperator {
Equal(String),
NotEqual(String),
}
impl FacetStringOperator {
fn equal(s: &str) -> Self {
FacetStringOperator::Equal(s.to_lowercase())
}
#[allow(dead_code)]
fn not_equal(s: &str) -> Self {
FacetStringOperator::equal(s).negate()
}
fn negate(self) -> Self {
match self {
FacetStringOperator::Equal(x) => FacetStringOperator::NotEqual(x),
FacetStringOperator::NotEqual(x) => FacetStringOperator::Equal(x),
}
}
}
#[derive(Debug, Clone, PartialEq)]
pub enum FacetCondition {
OperatorI64(FieldId, FacetNumberOperator<i64>),
OperatorF64(FieldId, FacetNumberOperator<f64>),
OperatorString(FieldId, FacetStringOperator),
Or(Box<Self>, Box<Self>),
And(Box<Self>, Box<Self>),
}
fn get_field_id_facet_type<'a>(
fields_ids_map: &FieldsIdsMap,
faceted_fields: &HashMap<FieldId, FacetType>,
items: &mut Pairs<'a, Rule>,
) -> Result<(FieldId, FacetType), PestError<Rule>>
{
// lexing ensures that we at least have a key
let key = items.next().unwrap();
let field_id = fields_ids_map
.id(key.as_str())
.ok_or_else(|| {
PestError::new_from_span(
ErrorVariant::CustomError {
message: format!(
"attribute `{}` not found, available attributes are: {}",
key.as_str(),
fields_ids_map.iter().map(|(_, n)| n).collect::<Vec<_>>().join(", ")
),
},
key.as_span(),
)
})?;
let facet_type = faceted_fields
.get(&field_id)
.copied()
.ok_or_else(|| {
PestError::new_from_span(
ErrorVariant::CustomError {
message: format!(
"attribute `{}` is not faceted, available faceted attributes are: {}",
key.as_str(),
faceted_fields.keys().flat_map(|id| fields_ids_map.name(*id)).collect::<Vec<_>>().join(", ")
),
},
key.as_span(),
)
})?;
Ok((field_id, facet_type))
}
fn pest_parse<T>(pair: Pair<Rule>) -> Result<T, pest::error::Error<Rule>>
where T: FromStr,
T::Err: ToString,
{
match pair.as_str().parse() {
Ok(value) => Ok(value),
Err(e) => {
Err(PestError::<Rule>::new_from_span(
ErrorVariant::CustomError { message: e.to_string() },
pair.as_span(),
))
}
}
}
impl FacetCondition {
pub fn from_array<I, J, A, B>(
rtxn: &heed::RoTxn,
index: &Index,
array: I,
) -> anyhow::Result<Option<FacetCondition>>
where I: IntoIterator<Item=Either<J, B>>,
J: IntoIterator<Item=A>,
A: AsRef<str>,
B: AsRef<str>,
{
fn facet_condition(
fields_ids_map: &FieldsIdsMap,
faceted_fields: &HashMap<String, FacetType>,
key: &str,
value: &str,
) -> anyhow::Result<FacetCondition>
{
let fid = fields_ids_map.id(key).with_context(|| {
format!("{:?} isn't present in the fields ids map", key)
})?;
let ftype = faceted_fields.get(key).copied().with_context(|| {
format!("{:?} isn't a faceted field", key)
})?;
let (neg, value) = match value.trim().strip_prefix('-') {
Some(value) => (true, value.trim()),
None => (false, value.trim()),
};
let operator = match ftype {
FacetType::String => OperatorString(fid, FacetStringOperator::equal(value)),
FacetType::Float => OperatorF64(fid, FacetNumberOperator::Equal(value.parse()?)),
FacetType::Integer => OperatorI64(fid, FacetNumberOperator::Equal(value.parse()?)),
};
if neg { Ok(operator.negate()) } else { Ok(operator) }
}
let fields_ids_map = index.fields_ids_map(rtxn)?;
let faceted_fields = index.faceted_fields(rtxn)?;
let mut ands = None;
for either in array {
match either {
Either::Left(array) => {
let mut ors = None;
for rule in array {
let mut iter = rule.as_ref().splitn(2, ':');
let key = iter.next().context("missing facet condition key")?;
let value = iter.next().context("missing facet condition value")?;
let condition = facet_condition(&fields_ids_map, &faceted_fields, key, value)?;
ors = match ors.take() {
Some(ors) => Some(Or(Box::new(ors), Box::new(condition))),
None => Some(condition),
};
}
if let Some(rule) = ors {
ands = match ands.take() {
Some(ands) => Some(And(Box::new(ands), Box::new(rule))),
None => Some(rule),
};
}
},
Either::Right(rule) => {
let mut iter = rule.as_ref().splitn(2, ':');
let key = iter.next().context("missing facet condition key")?;
let value = iter.next().context("missing facet condition value")?;
let condition = facet_condition(&fields_ids_map, &faceted_fields, key, value)?;
ands = match ands.take() {
Some(ands) => Some(And(Box::new(ands), Box::new(condition))),
None => Some(condition),
};
}
}
}
Ok(ands)
}
pub fn from_str(
rtxn: &heed::RoTxn,
index: &Index,
expression: &str,
) -> anyhow::Result<FacetCondition>
{
let fields_ids_map = index.fields_ids_map(rtxn)?;
let faceted_fields = index.faceted_fields_ids(rtxn)?;
let lexed = FilterParser::parse(Rule::prgm, expression)?;
FacetCondition::from_pairs(&fields_ids_map, &faceted_fields, lexed)
}
fn from_pairs(
fim: &FieldsIdsMap,
ff: &HashMap<FieldId, FacetType>,
expression: Pairs<Rule>,
) -> anyhow::Result<Self>
{
PREC_CLIMBER.climb(
expression,
|pair: Pair<Rule>| match pair.as_rule() {
Rule::greater => Ok(Self::greater_than(fim, ff, pair)?),
Rule::geq => Ok(Self::greater_than_or_equal(fim, ff, pair)?),
Rule::eq => Ok(Self::equal(fim, ff, pair)?),
Rule::neq => Ok(Self::equal(fim, ff, pair)?.negate()),
Rule::leq => Ok(Self::lower_than_or_equal(fim, ff, pair)?),
Rule::less => Ok(Self::lower_than(fim, ff, pair)?),
Rule::between => Ok(Self::between(fim, ff, pair)?),
Rule::not => Ok(Self::from_pairs(fim, ff, pair.into_inner())?.negate()),
Rule::prgm => Self::from_pairs(fim, ff, pair.into_inner()),
Rule::term => Self::from_pairs(fim, ff, pair.into_inner()),
_ => unreachable!(),
},
|lhs: anyhow::Result<Self>, op: Pair<Rule>, rhs: anyhow::Result<Self>| {
match op.as_rule() {
Rule::or => Ok(Or(Box::new(lhs?), Box::new(rhs?))),
Rule::and => Ok(And(Box::new(lhs?), Box::new(rhs?))),
_ => unreachable!(),
}
},
)
}
fn negate(self) -> FacetCondition {
match self {
OperatorI64(fid, op) => match op.negate() {
(op, None) => OperatorI64(fid, op),
(a, Some(b)) => Or(Box::new(OperatorI64(fid, a)), Box::new(OperatorI64(fid, b))),
},
OperatorF64(fid, op) => match op.negate() {
(op, None) => OperatorF64(fid, op),
(a, Some(b)) => Or(Box::new(OperatorF64(fid, a)), Box::new(OperatorF64(fid, b))),
},
OperatorString(fid, op) => OperatorString(fid, op.negate()),
Or(a, b) => And(Box::new(a.negate()), Box::new(b.negate())),
And(a, b) => Or(Box::new(a.negate()), Box::new(b.negate())),
}
}
fn between(
fields_ids_map: &FieldsIdsMap,
faceted_fields: &HashMap<FieldId, FacetType>,
item: Pair<Rule>,
) -> anyhow::Result<FacetCondition>
{
let item_span = item.as_span();
let mut items = item.into_inner();
let (fid, ftype) = get_field_id_facet_type(fields_ids_map, faceted_fields, &mut items)?;
let lvalue = items.next().unwrap();
let rvalue = items.next().unwrap();
match ftype {
FacetType::Integer => {
let lvalue = pest_parse(lvalue)?;
let rvalue = pest_parse(rvalue)?;
Ok(OperatorI64(fid, Between(lvalue, rvalue)))
},
FacetType::Float => {
let lvalue = pest_parse(lvalue)?;
let rvalue = pest_parse(rvalue)?;
Ok(OperatorF64(fid, Between(lvalue, rvalue)))
},
FacetType::String => {
Err(PestError::<Rule>::new_from_span(
ErrorVariant::CustomError {
message: "invalid operator on a faceted string".to_string(),
},
item_span,
).into())
},
}
}
fn equal(
fields_ids_map: &FieldsIdsMap,
faceted_fields: &HashMap<FieldId, FacetType>,
item: Pair<Rule>,
) -> anyhow::Result<FacetCondition>
{
let mut items = item.into_inner();
let (fid, ftype) = get_field_id_facet_type(fields_ids_map, faceted_fields, &mut items)?;
let value = items.next().unwrap();
match ftype {
FacetType::Integer => Ok(OperatorI64(fid, Equal(pest_parse(value)?))),
FacetType::Float => Ok(OperatorF64(fid, Equal(pest_parse(value)?))),
FacetType::String => Ok(OperatorString(fid, FacetStringOperator::equal(value.as_str()))),
}
}
fn greater_than(
fields_ids_map: &FieldsIdsMap,
faceted_fields: &HashMap<FieldId, FacetType>,
item: Pair<Rule>,
) -> anyhow::Result<FacetCondition>
{
let item_span = item.as_span();
let mut items = item.into_inner();
let (fid, ftype) = get_field_id_facet_type(fields_ids_map, faceted_fields, &mut items)?;
let value = items.next().unwrap();
match ftype {
FacetType::Integer => Ok(OperatorI64(fid, GreaterThan(pest_parse(value)?))),
FacetType::Float => Ok(OperatorF64(fid, GreaterThan(pest_parse(value)?))),
FacetType::String => {
Err(PestError::<Rule>::new_from_span(
ErrorVariant::CustomError {
message: "invalid operator on a faceted string".to_string(),
},
item_span,
).into())
},
}
}
fn greater_than_or_equal(
fields_ids_map: &FieldsIdsMap,
faceted_fields: &HashMap<FieldId, FacetType>,
item: Pair<Rule>,
) -> anyhow::Result<FacetCondition>
{
let item_span = item.as_span();
let mut items = item.into_inner();
let (fid, ftype) = get_field_id_facet_type(fields_ids_map, faceted_fields, &mut items)?;
let value = items.next().unwrap();
match ftype {
FacetType::Integer => Ok(OperatorI64(fid, GreaterThanOrEqual(pest_parse(value)?))),
FacetType::Float => Ok(OperatorF64(fid, GreaterThanOrEqual(pest_parse(value)?))),
FacetType::String => {
Err(PestError::<Rule>::new_from_span(
ErrorVariant::CustomError {
message: "invalid operator on a faceted string".to_string(),
},
item_span,
).into())
},
}
}
fn lower_than(
fields_ids_map: &FieldsIdsMap,
faceted_fields: &HashMap<FieldId, FacetType>,
item: Pair<Rule>,
) -> anyhow::Result<FacetCondition>
{
let item_span = item.as_span();
let mut items = item.into_inner();
let (fid, ftype) = get_field_id_facet_type(fields_ids_map, faceted_fields, &mut items)?;
let value = items.next().unwrap();
match ftype {
FacetType::Integer => Ok(OperatorI64(fid, LowerThan(pest_parse(value)?))),
FacetType::Float => Ok(OperatorF64(fid, LowerThan(pest_parse(value)?))),
FacetType::String => {
Err(PestError::<Rule>::new_from_span(
ErrorVariant::CustomError {
message: "invalid operator on a faceted string".to_string(),
},
item_span,
).into())
},
}
}
fn lower_than_or_equal(
fields_ids_map: &FieldsIdsMap,
faceted_fields: &HashMap<FieldId, FacetType>,
item: Pair<Rule>,
) -> anyhow::Result<FacetCondition>
{
let item_span = item.as_span();
let mut items = item.into_inner();
let (fid, ftype) = get_field_id_facet_type(fields_ids_map, faceted_fields, &mut items)?;
let value = items.next().unwrap();
match ftype {
FacetType::Integer => Ok(OperatorI64(fid, LowerThanOrEqual(pest_parse(value)?))),
FacetType::Float => Ok(OperatorF64(fid, LowerThanOrEqual(pest_parse(value)?))),
FacetType::String => {
Err(PestError::<Rule>::new_from_span(
ErrorVariant::CustomError {
message: "invalid operator on a faceted string".to_string(),
},
item_span,
).into())
},
}
}
}
impl FacetCondition {
/// Aggregates the documents ids that are part of the specified range automatically
/// going deeper through the levels.
fn explore_facet_levels<'t, T: 't, KC>(
rtxn: &'t heed::RoTxn,
db: heed::Database<ByteSlice, CboRoaringBitmapCodec>,
field_id: FieldId,
level: u8,
left: Bound<T>,
right: Bound<T>,
output: &mut RoaringBitmap,
) -> anyhow::Result<()>
where
T: Copy + PartialEq + PartialOrd + Bounded + Debug,
KC: heed::BytesDecode<'t, DItem = (u8, u8, T, T)>,
KC: for<'x> heed::BytesEncode<'x, EItem = (u8, u8, T, T)>,
{
match (left, right) {
// If the request is an exact value we must go directly to the deepest level.
(Included(l), Included(r)) if l == r && level > 0 => {
return Self::explore_facet_levels::<T, KC>(rtxn, db, field_id, 0, left, right, output);
},
// lower TO upper when lower > upper must return no result
(Included(l), Included(r)) if l > r => return Ok(()),
(Included(l), Excluded(r)) if l >= r => return Ok(()),
(Excluded(l), Excluded(r)) if l >= r => return Ok(()),
(Excluded(l), Included(r)) if l >= r => return Ok(()),
(_, _) => (),
}
let mut left_found = None;
let mut right_found = None;
// We must create a custom iterator to be able to iterate over the
// requested range as the range iterator cannot express some conditions.
let iter = FacetRange::new(rtxn, db.remap_key_type::<KC>(), field_id, level, left, right)?;
debug!("Iterating between {:?} and {:?} (level {})", left, right, level);
for (i, result) in iter.enumerate() {
let ((_fid, level, l, r), docids) = result?;
debug!("{:?} to {:?} (level {}) found {} documents", l, r, level, docids.len());
output.union_with(&docids);
// We save the leftest and rightest bounds we actually found at this level.
if i == 0 { left_found = Some(l); }
right_found = Some(r);
}
// Can we go deeper?
let deeper_level = match level.checked_sub(1) {
Some(level) => level,
None => return Ok(()),
};
// We must refine the left and right bounds of this range by retrieving the
// missing part in a deeper level.
match left_found.zip(right_found) {
Some((left_found, right_found)) => {
// If the bound is satisfied we avoid calling this function again.
if !matches!(left, Included(l) if l == left_found) {
let sub_right = Excluded(left_found);
debug!("calling left with {:?} to {:?} (level {})", left, sub_right, deeper_level);
Self::explore_facet_levels::<T, KC>(rtxn, db, field_id, deeper_level, left, sub_right, output)?;
}
if !matches!(right, Included(r) if r == right_found) {
let sub_left = Excluded(right_found);
debug!("calling right with {:?} to {:?} (level {})", sub_left, right, deeper_level);
Self::explore_facet_levels::<T, KC>(rtxn, db, field_id, deeper_level, sub_left, right, output)?;
}
},
None => {
// If we found nothing at this level it means that we must find
// the same bounds but at a deeper, more precise level.
Self::explore_facet_levels::<T, KC>(rtxn, db, field_id, deeper_level, left, right, output)?;
},
}
Ok(())
}
fn evaluate_number_operator<'t, T: 't, KC>(
rtxn: &'t heed::RoTxn,
index: &Index,
db: heed::Database<ByteSlice, CboRoaringBitmapCodec>,
field_id: FieldId,
operator: FacetNumberOperator<T>,
) -> anyhow::Result<RoaringBitmap>
where
T: Copy + PartialEq + PartialOrd + Bounded + Debug,
KC: heed::BytesDecode<'t, DItem = (u8, u8, T, T)>,
KC: for<'x> heed::BytesEncode<'x, EItem = (u8, u8, T, T)>,
{
// Make sure we always bound the ranges with the field id and the level,
// as the facets values are all in the same database and prefixed by the
// field id and the level.
let (left, right) = match operator {
GreaterThan(val) => (Excluded(val), Included(T::max_value())),
GreaterThanOrEqual(val) => (Included(val), Included(T::max_value())),
Equal(val) => (Included(val), Included(val)),
NotEqual(val) => {
let all_documents_ids = index.faceted_documents_ids(rtxn, field_id)?;
let docids = Self::evaluate_number_operator::<T, KC>(rtxn, index, db, field_id, Equal(val))?;
return Ok(all_documents_ids - docids);
},
LowerThan(val) => (Included(T::min_value()), Excluded(val)),
LowerThanOrEqual(val) => (Included(T::min_value()), Included(val)),
Between(left, right) => (Included(left), Included(right)),
};
// Ask for the biggest value that can exist for this specific field, if it exists
// that's fine if it don't, the value just before will be returned instead.
let biggest_level = db
.remap_types::<KC, DecodeIgnore>()
.get_lower_than_or_equal_to(rtxn, &(field_id, u8::MAX, T::max_value(), T::max_value()))?
.and_then(|((id, level, _, _), _)| if id == field_id { Some(level) } else { None });
match biggest_level {
Some(level) => {
let mut output = RoaringBitmap::new();
Self::explore_facet_levels::<T, KC>(rtxn, db, field_id, level, left, right, &mut output)?;
Ok(output)
},
None => Ok(RoaringBitmap::new()),
}
}
fn evaluate_string_operator(
rtxn: &heed::RoTxn,
index: &Index,
db: heed::Database<FacetValueStringCodec, CboRoaringBitmapCodec>,
field_id: FieldId,
operator: &FacetStringOperator,
) -> anyhow::Result<RoaringBitmap>
{
match operator {
FacetStringOperator::Equal(string) => {
match db.get(rtxn, &(field_id, string))? {
Some(docids) => Ok(docids),
None => Ok(RoaringBitmap::new())
}
},
FacetStringOperator::NotEqual(string) => {
let all_documents_ids = index.faceted_documents_ids(rtxn, field_id)?;
let op = FacetStringOperator::Equal(string.clone());
let docids = Self::evaluate_string_operator(rtxn, index, db, field_id, &op)?;
Ok(all_documents_ids - docids)
},
}
}
pub fn evaluate(
&self,
rtxn: &heed::RoTxn,
index: &Index,
) -> anyhow::Result<RoaringBitmap>
{
let db = index.facet_field_id_value_docids;
match self {
OperatorI64(fid, op) => {
Self::evaluate_number_operator::<i64, FacetLevelValueI64Codec>(rtxn, index, db, *fid, *op)
},
OperatorF64(fid, op) => {
Self::evaluate_number_operator::<f64, FacetLevelValueF64Codec>(rtxn, index, db, *fid, *op)
},
OperatorString(fid, op) => {
let db = db.remap_key_type::<FacetValueStringCodec>();
Self::evaluate_string_operator(rtxn, index, db, *fid, op)
},
Or(lhs, rhs) => {
let lhs = lhs.evaluate(rtxn, index)?;
let rhs = rhs.evaluate(rtxn, index)?;
Ok(lhs | rhs)
},
And(lhs, rhs) => {
let lhs = lhs.evaluate(rtxn, index)?;
let rhs = rhs.evaluate(rtxn, index)?;
Ok(lhs & rhs)
},
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::update::Settings;
use heed::EnvOpenOptions;
use maplit::hashmap;
#[test]
fn string() {
let path = tempfile::tempdir().unwrap();
let mut options = EnvOpenOptions::new();
options.map_size(10 * 1024 * 1024); // 10 MB
let index = Index::new(options, &path).unwrap();
// Set the faceted fields to be the channel.
let mut wtxn = index.write_txn().unwrap();
let mut builder = Settings::new(&mut wtxn, &index, 0);
builder.set_faceted_fields(hashmap!{ "channel".into() => "string".into() });
builder.execute(|_, _| ()).unwrap();
wtxn.commit().unwrap();
// Test that the facet condition is correctly generated.
let rtxn = index.read_txn().unwrap();
let condition = FacetCondition::from_str(&rtxn, &index, "channel = ponce").unwrap();
let expected = OperatorString(0, FacetStringOperator::equal("Ponce"));
assert_eq!(condition, expected);
let condition = FacetCondition::from_str(&rtxn, &index, "channel != ponce").unwrap();
let expected = OperatorString(0, FacetStringOperator::not_equal("ponce"));
assert_eq!(condition, expected);
let condition = FacetCondition::from_str(&rtxn, &index, "NOT channel = ponce").unwrap();
let expected = OperatorString(0, FacetStringOperator::not_equal("ponce"));
assert_eq!(condition, expected);
}
#[test]
fn i64() {
let path = tempfile::tempdir().unwrap();
let mut options = EnvOpenOptions::new();
options.map_size(10 * 1024 * 1024); // 10 MB
let index = Index::new(options, &path).unwrap();
// Set the faceted fields to be the channel.
let mut wtxn = index.write_txn().unwrap();
let mut builder = Settings::new(&mut wtxn, &index, 0);
builder.set_faceted_fields(hashmap!{ "timestamp".into() => "integer".into() });
builder.execute(|_, _| ()).unwrap();
wtxn.commit().unwrap();
// Test that the facet condition is correctly generated.
let rtxn = index.read_txn().unwrap();
let condition = FacetCondition::from_str(&rtxn, &index, "timestamp 22 TO 44").unwrap();
let expected = OperatorI64(0, Between(22, 44));
assert_eq!(condition, expected);
let condition = FacetCondition::from_str(&rtxn, &index, "NOT timestamp 22 TO 44").unwrap();
let expected = Or(
Box::new(OperatorI64(0, LowerThan(22))),
Box::new(OperatorI64(0, GreaterThan(44))),
);
assert_eq!(condition, expected);
}
#[test]
fn parentheses() {
let path = tempfile::tempdir().unwrap();
let mut options = EnvOpenOptions::new();
options.map_size(10 * 1024 * 1024); // 10 MB
let index = Index::new(options, &path).unwrap();
// Set the faceted fields to be the channel.
let mut wtxn = index.write_txn().unwrap();
let mut builder = Settings::new(&mut wtxn, &index, 0);
builder.set_searchable_fields(vec!["channel".into(), "timestamp".into()]); // to keep the fields order
builder.set_faceted_fields(hashmap!{
"channel".into() => "string".into(),
"timestamp".into() => "integer".into(),
});
builder.execute(|_, _| ()).unwrap();
wtxn.commit().unwrap();
// Test that the facet condition is correctly generated.
let rtxn = index.read_txn().unwrap();
let condition = FacetCondition::from_str(
&rtxn, &index,
"channel = gotaga OR (timestamp 22 TO 44 AND channel != ponce)",
).unwrap();
let expected = Or(
Box::new(OperatorString(0, FacetStringOperator::equal("gotaga"))),
Box::new(And(
Box::new(OperatorI64(1, Between(22, 44))),
Box::new(OperatorString(0, FacetStringOperator::not_equal("ponce"))),
))
);
assert_eq!(condition, expected);
let condition = FacetCondition::from_str(
&rtxn, &index,
"channel = gotaga OR NOT (timestamp 22 TO 44 AND channel != ponce)",
).unwrap();
let expected = Or(
Box::new(OperatorString(0, FacetStringOperator::equal("gotaga"))),
Box::new(Or(
Box::new(Or(
Box::new(OperatorI64(1, LowerThan(22))),
Box::new(OperatorI64(1, GreaterThan(44))),
)),
Box::new(OperatorString(0, FacetStringOperator::equal("ponce"))),
)),
);
assert_eq!(condition, expected);
}
#[test]
fn from_array() {
let path = tempfile::tempdir().unwrap();
let mut options = EnvOpenOptions::new();
options.map_size(10 * 1024 * 1024); // 10 MB
let index = Index::new(options, &path).unwrap();
// Set the faceted fields to be the channel.
let mut wtxn = index.write_txn().unwrap();
let mut builder = Settings::new(&mut wtxn, &index, 0);
builder.set_searchable_fields(vec!["channel".into(), "timestamp".into()]); // to keep the fields order
builder.set_faceted_fields(hashmap!{
"channel".into() => "string".into(),
"timestamp".into() => "integer".into(),
});
builder.execute(|_, _| ()).unwrap();
wtxn.commit().unwrap();
// Test that the facet condition is correctly generated.
let rtxn = index.read_txn().unwrap();
let condition = FacetCondition::from_array(
&rtxn, &index,
vec![Either::Right("channel:gotaga"), Either::Left(vec!["timestamp:44", "channel:-ponce"])],
).unwrap().unwrap();
let expected = FacetCondition::from_str(
&rtxn, &index,
"channel = gotaga AND (timestamp = 44 OR channel != ponce)",
).unwrap();
assert_eq!(condition, expected);
}
}

View File

@ -0,0 +1,260 @@
use std::collections::{HashSet, BTreeMap};
use std::ops::Bound::Unbounded;
use std::{cmp, fmt};
use anyhow::Context;
use heed::BytesDecode;
use roaring::RoaringBitmap;
use crate::facet::{FacetType, FacetValue};
use crate::heed_codec::facet::{FacetValueStringCodec, FacetLevelValueF64Codec, FacetLevelValueI64Codec};
use crate::heed_codec::facet::{FieldDocIdFacetStringCodec, FieldDocIdFacetF64Codec, FieldDocIdFacetI64Codec};
use crate::search::facet::{FacetIter, FacetRange};
use crate::{Index, FieldId, DocumentId};
/// The default number of values by facets that will
/// be fetched from the key-value store.
const DEFAULT_VALUES_BY_FACET: usize = 100;
/// The hard limit in the number of values by facets that will be fetched from
/// the key-value store. Searching for more values could slow down the engine.
const MAX_VALUES_BY_FACET: usize = 1000;
/// Threshold on the number of candidates that will make
/// the system to choose between one algorithm or another.
const CANDIDATES_THRESHOLD: u64 = 1000;
pub struct FacetDistribution<'a> {
facets: Option<HashSet<String>>,
candidates: Option<RoaringBitmap>,
max_values_by_facet: usize,
rtxn: &'a heed::RoTxn<'a>,
index: &'a Index,
}
impl<'a> FacetDistribution<'a> {
pub fn new(rtxn: &'a heed::RoTxn, index: &'a Index) -> FacetDistribution<'a> {
FacetDistribution {
facets: None,
candidates: None,
max_values_by_facet: DEFAULT_VALUES_BY_FACET,
rtxn,
index,
}
}
pub fn facets<I: IntoIterator<Item=A>, A: AsRef<str>>(&mut self, names: I) -> &mut Self {
self.facets = Some(names.into_iter().map(|s| s.as_ref().to_string()).collect());
self
}
pub fn candidates(&mut self, candidates: RoaringBitmap) -> &mut Self {
self.candidates = Some(candidates);
self
}
pub fn max_values_by_facet(&mut self, max: usize) -> &mut Self {
self.max_values_by_facet = cmp::min(max, MAX_VALUES_BY_FACET);
self
}
/// There is a small amount of candidates OR we ask for facet string values so we
/// decide to iterate over the facet values of each one of them, one by one.
fn facet_values_from_documents(
&self,
field_id: FieldId,
facet_type: FacetType,
candidates: &RoaringBitmap,
) -> heed::Result<BTreeMap<FacetValue, u64>>
{
fn fetch_facet_values<'t, KC, K: 't>(
index: &Index,
rtxn: &'t heed::RoTxn,
field_id: FieldId,
candidates: &RoaringBitmap,
) -> heed::Result<BTreeMap<FacetValue, u64>>
where
KC: BytesDecode<'t, DItem = (FieldId, DocumentId, K)>,
K: Into<FacetValue>,
{
let mut facet_values = BTreeMap::new();
let mut key_buffer = vec![field_id];
for docid in candidates.into_iter().take(CANDIDATES_THRESHOLD as usize) {
key_buffer.truncate(1);
key_buffer.extend_from_slice(&docid.to_be_bytes());
let iter = index.field_id_docid_facet_values
.prefix_iter(rtxn, &key_buffer)?
.remap_key_type::<KC>();
for result in iter {
let ((_, _, value), ()) = result?;
*facet_values.entry(value.into()).or_insert(0) += 1;
}
}
Ok(facet_values)
}
let index = self.index;
let rtxn = self.rtxn;
match facet_type {
FacetType::String => {
fetch_facet_values::<FieldDocIdFacetStringCodec, _>(index, rtxn, field_id, candidates)
},
FacetType::Float => {
fetch_facet_values::<FieldDocIdFacetF64Codec, _>(index, rtxn, field_id, candidates)
},
FacetType::Integer => {
fetch_facet_values::<FieldDocIdFacetI64Codec, _>(index, rtxn, field_id, candidates)
},
}
}
/// There is too much documents, we use the facet levels to move throught
/// the facet values, to find the candidates and values associated.
fn facet_values_from_facet_levels(
&self,
field_id: FieldId,
facet_type: FacetType,
candidates: &RoaringBitmap,
) -> heed::Result<BTreeMap<FacetValue, u64>>
{
let iter = match facet_type {
FacetType::String => unreachable!(),
FacetType::Float => {
let iter = FacetIter::<f64, FacetLevelValueF64Codec>::new_non_reducing(
self.rtxn, self.index, field_id, candidates.clone(),
)?;
let iter = iter.map(|r| r.map(|(v, docids)| (FacetValue::from(v), docids)));
Box::new(iter) as Box::<dyn Iterator<Item=_>>
},
FacetType::Integer => {
let iter = FacetIter::<i64, FacetLevelValueI64Codec>::new_non_reducing(
self.rtxn, self.index, field_id, candidates.clone(),
)?;
Box::new(iter.map(|r| r.map(|(v, docids)| (FacetValue::from(v), docids))))
},
};
let mut facet_values = BTreeMap::new();
for result in iter {
let (value, mut docids) = result?;
docids.intersect_with(candidates);
if !docids.is_empty() {
facet_values.insert(value, docids.len());
}
if facet_values.len() == self.max_values_by_facet {
break;
}
}
Ok(facet_values)
}
/// Placeholder search, a.k.a. no candidates were specified. We iterate throught the
/// facet values one by one and iterate on the facet level 0 for numbers.
fn facet_values_from_raw_facet_database(
&self,
field_id: FieldId,
facet_type: FacetType,
) -> heed::Result<BTreeMap<FacetValue, u64>>
{
let db = self.index.facet_field_id_value_docids;
let level = 0;
let iter = match facet_type {
FacetType::String => {
let iter = db
.prefix_iter(self.rtxn, &[field_id])?
.remap_key_type::<FacetValueStringCodec>()
.map(|r| r.map(|((_, v), docids)| (FacetValue::from(v), docids)));
Box::new(iter) as Box::<dyn Iterator<Item=_>>
},
FacetType::Float => {
let db = db.remap_key_type::<FacetLevelValueF64Codec>();
let range = FacetRange::<f64, _>::new(
self.rtxn, db, field_id, level, Unbounded, Unbounded,
)?;
Box::new(range.map(|r| r.map(|((_, _, v, _), docids)| (FacetValue::from(v), docids))))
},
FacetType::Integer => {
let db = db.remap_key_type::<FacetLevelValueI64Codec>();
let range = FacetRange::<i64, _>::new(
self.rtxn, db, field_id, level, Unbounded, Unbounded,
)?;
Box::new(range.map(|r| r.map(|((_, _, v, _), docids)| (FacetValue::from(v), docids))))
},
};
let mut facet_values = BTreeMap::new();
for result in iter {
let (value, docids) = result?;
facet_values.insert(value, docids.len());
if facet_values.len() == self.max_values_by_facet {
break;
}
}
Ok(facet_values)
}
fn facet_values(
&self,
field_id: FieldId,
facet_type: FacetType,
) -> heed::Result<BTreeMap<FacetValue, u64>>
{
if let Some(candidates) = self.candidates.as_ref() {
// Classic search, candidates were specified, we must return facet values only related
// to those candidates. We also enter here for facet strings for performance reasons.
if candidates.len() <= CANDIDATES_THRESHOLD || facet_type == FacetType::String {
self.facet_values_from_documents(field_id, facet_type, candidates)
} else {
self.facet_values_from_facet_levels(field_id, facet_type, candidates)
}
} else {
self.facet_values_from_raw_facet_database(field_id, facet_type)
}
}
pub fn execute(&self) -> anyhow::Result<BTreeMap<String, BTreeMap<FacetValue, u64>>> {
let fields_ids_map = self.index.fields_ids_map(self.rtxn)?;
let faceted_fields = self.index.faceted_fields(self.rtxn)?;
let fields_ids: Vec<_> = match &self.facets {
Some(names) => names
.iter()
.filter_map(|n| faceted_fields.get(n).map(|t| (n.to_string(), *t)))
.collect(),
None => faceted_fields.into_iter().collect(),
};
let mut facets_values = BTreeMap::new();
for (name, ftype) in fields_ids {
let fid = fields_ids_map.id(&name).with_context(|| {
format!("missing field name {:?} from the fields id map", name)
})?;
let values = self.facet_values(fid, ftype)?;
facets_values.insert(name, values);
}
Ok(facets_values)
}
}
impl fmt::Debug for FacetDistribution<'_> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
let FacetDistribution {
facets,
candidates,
max_values_by_facet,
rtxn: _,
index: _,
} = self;
f.debug_struct("FacetDistribution")
.field("facets", facets)
.field("candidates", candidates)
.field("max_values_by_facet", max_values_by_facet)
.finish()
}
}

View File

@ -0,0 +1,29 @@
key = _{quoted | word}
value = _{quoted | word}
quoted = _{ (PUSH("'") | PUSH("\"")) ~ string ~ POP }
string = {char*}
word = ${(LETTER | NUMBER | "_" | "-" | ".")+}
char = _{ !(PEEK | "\\") ~ ANY
| "\\" ~ (PEEK | "\\" | "/" | "b" | "f" | "n" | "r" | "t")
| "\\" ~ ("u" ~ ASCII_HEX_DIGIT{4})}
condition = _{between | eq | greater | less | geq | leq | neq}
between = {key ~ value ~ "TO" ~ value}
geq = {key ~ ">=" ~ value}
leq = {key ~ "<=" ~ value}
neq = {key ~ "!=" ~ value}
eq = {key ~ "=" ~ value}
greater = {key ~ ">" ~ value}
less = {key ~ "<" ~ value}
prgm = {SOI ~ expr ~ EOI}
expr = _{ ( term ~ (operation ~ term)* ) }
term = { ("(" ~ expr ~ ")") | condition | not }
operation = _{ and | or }
and = {"AND"}
or = {"OR"}
not = {"NOT" ~ term}
WHITESPACE = _{ " " }

View File

@ -0,0 +1,286 @@
use std::fmt::Debug;
use std::ops::Bound::{self, Included, Excluded, Unbounded};
use either::Either::{self, Left, Right};
use heed::types::{DecodeIgnore, ByteSlice};
use heed::{BytesEncode, BytesDecode};
use heed::{Database, RoRange, RoRevRange, LazyDecode};
use log::debug;
use num_traits::Bounded;
use roaring::RoaringBitmap;
use crate::heed_codec::CboRoaringBitmapCodec;
use crate::{Index, FieldId};
pub use self::facet_condition::{FacetCondition, FacetNumberOperator, FacetStringOperator};
pub use self::facet_distribution::FacetDistribution;
mod facet_condition;
mod facet_distribution;
mod parser;
pub struct FacetRange<'t, T: 't, KC> {
iter: RoRange<'t, KC, LazyDecode<CboRoaringBitmapCodec>>,
end: Bound<T>,
}
impl<'t, T: 't, KC> FacetRange<'t, T, KC>
where
KC: for<'a> BytesEncode<'a, EItem = (FieldId, u8, T, T)>,
T: PartialOrd + Copy + Bounded,
{
pub fn new(
rtxn: &'t heed::RoTxn,
db: Database<KC, CboRoaringBitmapCodec>,
field_id: FieldId,
level: u8,
left: Bound<T>,
right: Bound<T>,
) -> heed::Result<FacetRange<'t, T, KC>>
{
let left_bound = match left {
Included(left) => Included((field_id, level, left, T::min_value())),
Excluded(left) => Excluded((field_id, level, left, T::min_value())),
Unbounded => Included((field_id, level, T::min_value(), T::min_value())),
};
let right_bound = Included((field_id, level, T::max_value(), T::max_value()));
let iter = db.lazily_decode_data().range(rtxn, &(left_bound, right_bound))?;
Ok(FacetRange { iter, end: right })
}
}
impl<'t, T, KC> Iterator for FacetRange<'t, T, KC>
where
KC: for<'a> BytesEncode<'a, EItem = (FieldId, u8, T, T)>,
KC: BytesDecode<'t, DItem = (FieldId, u8, T, T)>,
T: PartialOrd + Copy,
{
type Item = heed::Result<((FieldId, u8, T, T), RoaringBitmap)>;
fn next(&mut self) -> Option<Self::Item> {
match self.iter.next() {
Some(Ok(((fid, level, left, right), docids))) => {
let must_be_returned = match self.end {
Included(end) => right <= end,
Excluded(end) => right < end,
Unbounded => true,
};
if must_be_returned {
match docids.decode() {
Ok(docids) => Some(Ok(((fid, level, left, right), docids))),
Err(e) => Some(Err(e)),
}
} else {
None
}
},
Some(Err(e)) => Some(Err(e)),
None => None,
}
}
}
pub struct FacetRevRange<'t, T: 't, KC> {
iter: RoRevRange<'t, KC, LazyDecode<CboRoaringBitmapCodec>>,
end: Bound<T>,
}
impl<'t, T: 't, KC> FacetRevRange<'t, T, KC>
where
KC: for<'a> BytesEncode<'a, EItem = (FieldId, u8, T, T)>,
T: PartialOrd + Copy + Bounded,
{
pub fn new(
rtxn: &'t heed::RoTxn,
db: Database<KC, CboRoaringBitmapCodec>,
field_id: FieldId,
level: u8,
left: Bound<T>,
right: Bound<T>,
) -> heed::Result<FacetRevRange<'t, T, KC>>
{
let left_bound = match left {
Included(left) => Included((field_id, level, left, T::min_value())),
Excluded(left) => Excluded((field_id, level, left, T::min_value())),
Unbounded => Included((field_id, level, T::min_value(), T::min_value())),
};
let right_bound = Included((field_id, level, T::max_value(), T::max_value()));
let iter = db.lazily_decode_data().rev_range(rtxn, &(left_bound, right_bound))?;
Ok(FacetRevRange { iter, end: right })
}
}
impl<'t, T, KC> Iterator for FacetRevRange<'t, T, KC>
where
KC: for<'a> BytesEncode<'a, EItem = (FieldId, u8, T, T)>,
KC: BytesDecode<'t, DItem = (FieldId, u8, T, T)>,
T: PartialOrd + Copy,
{
type Item = heed::Result<((FieldId, u8, T, T), RoaringBitmap)>;
fn next(&mut self) -> Option<Self::Item> {
loop {
match self.iter.next() {
Some(Ok(((fid, level, left, right), docids))) => {
let must_be_returned = match self.end {
Included(end) => right <= end,
Excluded(end) => right < end,
Unbounded => true,
};
if must_be_returned {
match docids.decode() {
Ok(docids) => return Some(Ok(((fid, level, left, right), docids))),
Err(e) => return Some(Err(e)),
}
}
continue;
},
Some(Err(e)) => return Some(Err(e)),
None => return None,
}
}
}
}
pub struct FacetIter<'t, T: 't, KC> {
rtxn: &'t heed::RoTxn<'t>,
db: Database<KC, CboRoaringBitmapCodec>,
field_id: FieldId,
level_iters: Vec<(RoaringBitmap, Either<FacetRange<'t, T, KC>, FacetRevRange<'t, T, KC>>)>,
must_reduce: bool,
}
impl<'t, T, KC> FacetIter<'t, T, KC>
where
KC: heed::BytesDecode<'t, DItem = (FieldId, u8, T, T)>,
KC: for<'a> BytesEncode<'a, EItem = (FieldId, u8, T, T)>,
T: PartialOrd + Copy + Bounded,
{
/// Create a `FacetIter` that will iterate on the different facet entries
/// (facet value + documents ids) and that will reduce the given documents ids
/// while iterating on the different facet levels.
pub fn new_reducing(
rtxn: &'t heed::RoTxn,
index: &'t Index,
field_id: FieldId,
documents_ids: RoaringBitmap,
) -> heed::Result<FacetIter<'t, T, KC>>
{
let db = index.facet_field_id_value_docids.remap_key_type::<KC>();
let highest_level = Self::highest_level(rtxn, db, field_id)?.unwrap_or(0);
let highest_iter = FacetRange::new(rtxn, db, field_id, highest_level, Unbounded, Unbounded)?;
let level_iters = vec![(documents_ids, Left(highest_iter))];
Ok(FacetIter { rtxn, db, field_id, level_iters, must_reduce: true })
}
/// Create a `FacetIter` that will iterate on the different facet entries in reverse
/// (facet value + documents ids) and that will reduce the given documents ids
/// while iterating on the different facet levels.
pub fn new_reverse_reducing(
rtxn: &'t heed::RoTxn,
index: &'t Index,
field_id: FieldId,
documents_ids: RoaringBitmap,
) -> heed::Result<FacetIter<'t, T, KC>>
{
let db = index.facet_field_id_value_docids.remap_key_type::<KC>();
let highest_level = Self::highest_level(rtxn, db, field_id)?.unwrap_or(0);
let highest_iter = FacetRevRange::new(rtxn, db, field_id, highest_level, Unbounded, Unbounded)?;
let level_iters = vec![(documents_ids, Right(highest_iter))];
Ok(FacetIter { rtxn, db, field_id, level_iters, must_reduce: true })
}
/// Create a `FacetIter` that will iterate on the different facet entries
/// (facet value + documents ids) and that will not reduce the given documents ids
/// while iterating on the different facet levels, possibly returning multiple times
/// a document id associated with multiple facet values.
pub fn new_non_reducing(
rtxn: &'t heed::RoTxn,
index: &'t Index,
field_id: FieldId,
documents_ids: RoaringBitmap,
) -> heed::Result<FacetIter<'t, T, KC>>
{
let db = index.facet_field_id_value_docids.remap_key_type::<KC>();
let highest_level = Self::highest_level(rtxn, db, field_id)?.unwrap_or(0);
let highest_iter = FacetRange::new(rtxn, db, field_id, highest_level, Unbounded, Unbounded)?;
let level_iters = vec![(documents_ids, Left(highest_iter))];
Ok(FacetIter { rtxn, db, field_id, level_iters, must_reduce: false })
}
fn highest_level<X>(rtxn: &'t heed::RoTxn, db: Database<KC, X>, fid: FieldId) -> heed::Result<Option<u8>> {
let level = db.remap_types::<ByteSlice, DecodeIgnore>()
.prefix_iter(rtxn, &[fid][..])?
.remap_key_type::<KC>()
.last().transpose()?
.map(|((_, level, _, _), _)| level);
Ok(level)
}
}
impl<'t, T: 't, KC> Iterator for FacetIter<'t, T, KC>
where
KC: heed::BytesDecode<'t, DItem = (FieldId, u8, T, T)>,
KC: for<'x> heed::BytesEncode<'x, EItem = (FieldId, u8, T, T)>,
T: PartialOrd + Copy + Bounded + Debug,
{
type Item = heed::Result<(T, RoaringBitmap)>;
fn next(&mut self) -> Option<Self::Item> {
'outer: loop {
let (documents_ids, last) = self.level_iters.last_mut()?;
let is_ascending = last.is_left();
for result in last {
// If the last iterator must find an empty set of documents it means
// that we found all the documents in the sub level iterations already,
// we can pop this level iterator.
if documents_ids.is_empty() {
break;
}
match result {
Ok(((_fid, level, left, right), mut docids)) => {
docids.intersect_with(&documents_ids);
if !docids.is_empty() {
if self.must_reduce {
documents_ids.difference_with(&docids);
}
if level == 0 {
debug!("found {:?} at {:?}", docids, left);
return Some(Ok((left, docids)));
}
let rtxn = self.rtxn;
let db = self.db;
let fid = self.field_id;
let left = Included(left);
let right = Included(right);
debug!("calling with {:?} to {:?} (level {}) to find {:?}",
left, right, level - 1, docids,
);
let result = if is_ascending {
FacetRange::new(rtxn, db, fid, level - 1, left, right).map(Left)
} else {
FacetRevRange::new(rtxn, db, fid, level - 1, left, right).map(Right)
};
match result {
Ok(iter) => {
self.level_iters.push((docids, iter));
continue 'outer;
},
Err(e) => return Some(Err(e)),
}
}
},
Err(e) => return Some(Err(e)),
}
}
self.level_iters.pop();
}
}
}

View File

@ -0,0 +1,12 @@
use once_cell::sync::Lazy;
use pest::prec_climber::{Operator, Assoc, PrecClimber};
pub static PREC_CLIMBER: Lazy<PrecClimber<Rule>> = Lazy::new(|| {
use Assoc::*;
use Rule::*;
pest::prec_climber::PrecClimber::new(vec![Operator::new(or, Left), Operator::new(and, Left)])
});
#[derive(Parser)]
#[grammar = "search/facet/grammar.pest"]
pub struct FilterParser;