support ff range queries on json fields (#2456)

* support ff range queries on json fields

* fix term date truncation

* use inverted index range query for phrase prefix queries

* rename to InvertedIndexRangeQuery

* fix column filter, add mixed column test
This commit is contained in:
PSeitz
2024-08-02 01:06:50 +09:00
committed by GitHub
parent 0d4e319965
commit 3d1c4b313a
15 changed files with 1322 additions and 289 deletions

130
common/src/bounds.rs Normal file
View File

@@ -0,0 +1,130 @@
use std::io;
use std::ops::Bound;
#[derive(Clone, Debug)]
pub struct BoundsRange<T> {
pub lower_bound: Bound<T>,
pub upper_bound: Bound<T>,
}
impl<T> BoundsRange<T> {
pub fn new(lower_bound: Bound<T>, upper_bound: Bound<T>) -> Self {
BoundsRange {
lower_bound,
upper_bound,
}
}
pub fn is_unbounded(&self) -> bool {
matches!(self.lower_bound, Bound::Unbounded) && matches!(self.upper_bound, Bound::Unbounded)
}
pub fn map_bound<TTo>(&self, transform: impl Fn(&T) -> TTo) -> BoundsRange<TTo> {
BoundsRange {
lower_bound: map_bound(&self.lower_bound, &transform),
upper_bound: map_bound(&self.upper_bound, &transform),
}
}
pub fn map_bound_res<TTo, Err>(
&self,
transform: impl Fn(&T) -> Result<TTo, Err>,
) -> Result<BoundsRange<TTo>, Err> {
Ok(BoundsRange {
lower_bound: map_bound_res(&self.lower_bound, &transform)?,
upper_bound: map_bound_res(&self.upper_bound, &transform)?,
})
}
pub fn transform_inner<TTo>(
&self,
transform_lower: impl Fn(&T) -> TransformBound<TTo>,
transform_upper: impl Fn(&T) -> TransformBound<TTo>,
) -> BoundsRange<TTo> {
BoundsRange {
lower_bound: transform_bound_inner(&self.lower_bound, &transform_lower),
upper_bound: transform_bound_inner(&self.upper_bound, &transform_upper),
}
}
/// Returns the first set inner value
pub fn get_inner(&self) -> Option<&T> {
inner_bound(&self.lower_bound).or(inner_bound(&self.upper_bound))
}
}
pub enum TransformBound<T> {
/// Overwrite the bounds
NewBound(Bound<T>),
/// Use Existing bounds with new value
Existing(T),
}
/// Takes a bound and transforms the inner value into a new bound via a closure.
/// The bound variant may change by the value returned value from the closure.
pub fn transform_bound_inner_res<TFrom, TTo>(
bound: &Bound<TFrom>,
transform: impl Fn(&TFrom) -> io::Result<TransformBound<TTo>>,
) -> io::Result<Bound<TTo>> {
use self::Bound::*;
Ok(match bound {
Excluded(ref from_val) => match transform(from_val)? {
TransformBound::NewBound(new_val) => new_val,
TransformBound::Existing(new_val) => Excluded(new_val),
},
Included(ref from_val) => match transform(from_val)? {
TransformBound::NewBound(new_val) => new_val,
TransformBound::Existing(new_val) => Included(new_val),
},
Unbounded => Unbounded,
})
}
/// Takes a bound and transforms the inner value into a new bound via a closure.
/// The bound variant may change by the value returned value from the closure.
pub fn transform_bound_inner<TFrom, TTo>(
bound: &Bound<TFrom>,
transform: impl Fn(&TFrom) -> TransformBound<TTo>,
) -> Bound<TTo> {
use self::Bound::*;
match bound {
Excluded(ref from_val) => match transform(from_val) {
TransformBound::NewBound(new_val) => new_val,
TransformBound::Existing(new_val) => Excluded(new_val),
},
Included(ref from_val) => match transform(from_val) {
TransformBound::NewBound(new_val) => new_val,
TransformBound::Existing(new_val) => Included(new_val),
},
Unbounded => Unbounded,
}
}
/// Returns the inner value of a `Bound`
pub fn inner_bound<T>(val: &Bound<T>) -> Option<&T> {
match val {
Bound::Included(term) | Bound::Excluded(term) => Some(term),
Bound::Unbounded => None,
}
}
pub fn map_bound<TFrom, TTo>(
bound: &Bound<TFrom>,
transform: impl Fn(&TFrom) -> TTo,
) -> Bound<TTo> {
use self::Bound::*;
match bound {
Excluded(ref from_val) => Bound::Excluded(transform(from_val)),
Included(ref from_val) => Bound::Included(transform(from_val)),
Unbounded => Unbounded,
}
}
pub fn map_bound_res<TFrom, TTo, Err>(
bound: &Bound<TFrom>,
transform: impl Fn(&TFrom) -> Result<TTo, Err>,
) -> Result<Bound<TTo>, Err> {
use self::Bound::*;
Ok(match bound {
Excluded(ref from_val) => Excluded(transform(from_val)?),
Included(ref from_val) => Included(transform(from_val)?),
Unbounded => Unbounded,
})
}

View File

@@ -5,6 +5,7 @@ use std::ops::Deref;
pub use byteorder::LittleEndian as Endianness;
mod bitset;
pub mod bounds;
mod byte_count;
mod datetime;
pub mod file_slice;

View File

@@ -4,7 +4,7 @@ use rustc_hash::FxHashMap;
use crate::postings::{IndexingContext, IndexingPosition, PostingsWriter};
use crate::schema::document::{ReferenceValue, ReferenceValueLeaf, Value};
use crate::schema::Type;
use crate::schema::{Type, DATE_TIME_PRECISION_INDEXED};
use crate::time::format_description::well_known::Rfc3339;
use crate::time::{OffsetDateTime, UtcOffset};
use crate::tokenizer::TextAnalyzer;
@@ -189,6 +189,7 @@ pub(crate) fn index_json_value<'a, V: Value<'a>>(
ctx.path_to_unordered_id
.get_or_allocate_unordered_id(json_path_writer.as_str()),
);
let val = val.truncate(DATE_TIME_PRECISION_INDEXED);
term_buffer.append_type_and_fast_value(val);
postings_writer.subscribe(doc, 0u32, term_buffer, ctx);
}
@@ -239,7 +240,11 @@ pub(crate) fn index_json_value<'a, V: Value<'a>>(
/// Tries to infer a JSON type from a string and append it to the term.
///
/// The term must be json + JSON path.
pub fn convert_to_fast_value_and_append_to_json_term(mut term: Term, phrase: &str) -> Option<Term> {
pub fn convert_to_fast_value_and_append_to_json_term(
mut term: Term,
phrase: &str,
truncate_date_for_search: bool,
) -> Option<Term> {
assert_eq!(
term.value()
.as_json_value_bytes()
@@ -250,8 +255,11 @@ pub fn convert_to_fast_value_and_append_to_json_term(mut term: Term, phrase: &st
"JSON value bytes should be empty"
);
if let Ok(dt) = OffsetDateTime::parse(phrase, &Rfc3339) {
let dt_utc = dt.to_offset(UtcOffset::UTC);
term.append_type_and_fast_value(DateTime::from_utc(dt_utc));
let mut dt = DateTime::from_utc(dt.to_offset(UtcOffset::UTC));
if truncate_date_for_search {
dt = dt.truncate(DATE_TIME_PRECISION_INDEXED);
}
term.append_type_and_fast_value(dt);
return Some(term);
}
if let Ok(i64_val) = str::parse::<i64>(phrase) {

View File

@@ -673,7 +673,7 @@ mod tests {
]
);
assert_eq!(
get_doc_ids(vec![Term::from_field_date(
get_doc_ids(vec![Term::from_field_date_for_search(
date_field,
DateTime::from_utc(curr_time)
)])?,

View File

@@ -64,9 +64,9 @@ impl SegmentWriter {
///
/// The arguments are defined as follows
///
/// - memory_budget: most of the segment writer data (terms, and postings lists recorders)
/// is stored in a memory arena. This makes it possible for the user to define
/// the flushing behavior as a memory limit.
/// - memory_budget: most of the segment writer data (terms, and postings lists recorders) is
/// stored in a memory arena. This makes it possible for the user to define the flushing
/// behavior as a memory limit.
/// - segment: The segment being written
/// - schema
pub fn for_segment(memory_budget_in_bytes: usize, segment: Segment) -> crate::Result<Self> {
@@ -431,7 +431,7 @@ mod tests {
use crate::query::{PhraseQuery, QueryParser};
use crate::schema::{
Document, IndexRecordOption, OwnedValue, Schema, TextFieldIndexing, TextOptions, Value,
STORED, STRING, TEXT,
DATE_TIME_PRECISION_INDEXED, STORED, STRING, TEXT,
};
use crate::store::{Compressor, StoreReader, StoreWriter};
use crate::time::format_description::well_known::Rfc3339;
@@ -651,7 +651,8 @@ mod tests {
set_fast_val(
DateTime::from_utc(
OffsetDateTime::parse("1985-04-12T23:20:50.52Z", &Rfc3339).unwrap(),
),
)
.truncate(DATE_TIME_PRECISION_INDEXED),
term
)
.serialized_value_bytes()

View File

@@ -54,7 +54,7 @@ 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};
pub use self::range_query::{FastFieldRangeWeight, RangeQuery};
pub use self::range_query::*;
pub use self::regex_query::RegexQuery;
pub use self::reqopt_scorer::RequiredOptionalScorer;
pub use self::score_combiner::{

View File

@@ -241,7 +241,7 @@ impl MoreLikeThis {
let timestamp = value.as_datetime().ok_or_else(|| {
TantivyError::InvalidArgument("invalid value".to_string())
})?;
let term = Term::from_field_date(field, timestamp);
let term = Term::from_field_date_for_search(field, timestamp);
*term_frequencies.entry(term).or_insert(0) += 1;
}
}

View File

@@ -2,7 +2,7 @@ use std::ops::Bound;
use super::{prefix_end, PhrasePrefixWeight};
use crate::query::bm25::Bm25Weight;
use crate::query::{EnableScoring, Query, RangeQuery, Weight};
use crate::query::{EnableScoring, InvertedIndexRangeWeight, Query, Weight};
use crate::schema::{Field, IndexRecordOption, Term};
const DEFAULT_MAX_EXPANSIONS: u32 = 50;
@@ -145,9 +145,15 @@ impl Query for PhrasePrefixQuery {
Bound::Unbounded
};
let mut range_query = RangeQuery::new(Bound::Included(self.prefix.1.clone()), end_term);
range_query.limit(self.max_expansions as u64);
range_query.weight(enable_scoring)
let lower_bound = Bound::Included(self.prefix.1.clone());
let upper_bound = end_term;
Ok(Box::new(InvertedIndexRangeWeight::new(
self.field,
&lower_bound,
&upper_bound,
Some(self.max_expansions as u64),
)))
}
}

View File

@@ -137,7 +137,7 @@ fn trim_ast(logical_ast: LogicalAst) -> Option<LogicalAst> {
/// so-called default fields (as set up in the constructor).
///
/// Assuming that the default fields are `body` and `title`, and the query parser is set with
/// conjunction as a default, our query will be interpreted as.
/// conjunction as a default, our query will be interpreted as.
/// `(body:Barack OR title:Barack) AND (title:Obama OR body:Obama)`.
/// By default, all tokenized and indexed fields are default fields.
///
@@ -148,8 +148,7 @@ fn trim_ast(logical_ast: LogicalAst) -> Option<LogicalAst> {
/// `body:Barack OR (body:Barack OR text:Obama)` .
///
/// * boolean operators `AND`, `OR`. `AND` takes precedence over `OR`, so that `a AND b OR c` is
/// interpreted
/// as `(a AND b) OR c`.
/// interpreted as `(a AND b) OR c`.
///
/// * In addition to the boolean operators, the `-`, `+` can help define. These operators are
/// sufficient to express all queries using boolean operators. For instance `x AND y OR z` can be
@@ -272,8 +271,7 @@ impl QueryParser {
/// Creates a `QueryParser`, given
/// * an index
/// * a set of default fields used to search if no field is specifically defined
/// in the query.
/// * a set of default fields used to search if no field is specifically defined in the query.
pub fn for_index(index: &Index, default_fields: Vec<Field>) -> QueryParser {
QueryParser::new(index.schema(), default_fields, index.tokenizers().clone())
}
@@ -482,16 +480,33 @@ impl QueryParser {
});
if terms.len() != 1 {
return Err(QueryParserError::UnsupportedQuery(format!(
"Range query boundary cannot have multiple tokens: {phrase:?}."
"Range query boundary cannot have multiple tokens: {phrase:?} [{terms:?}]."
)));
}
Ok(terms.into_iter().next().unwrap())
}
FieldType::JsonObject(_) => {
// Json range are not supported.
Err(QueryParserError::UnsupportedQuery(
"Range query are not supported on json field.".to_string(),
))
FieldType::JsonObject(ref json_options) => {
let get_term_with_path = || {
Term::from_field_json_path(
field,
json_path,
json_options.is_expand_dots_enabled(),
)
};
if let Some(term) =
// Try to convert the phrase to a fast value
convert_to_fast_value_and_append_to_json_term(
get_term_with_path(),
phrase,
false,
)
{
Ok(term)
} else {
let mut term = get_term_with_path();
term.append_type_and_str(phrase);
Ok(term)
}
}
FieldType::Facet(_) => match Facet::from_text(phrase) {
Ok(facet) => Ok(Term::from_facet(field, &facet)),
@@ -553,7 +568,7 @@ impl QueryParser {
}
FieldType::Date(_) => {
let dt = OffsetDateTime::parse(phrase, &Rfc3339)?;
let dt_term = Term::from_field_date(field, DateTime::from_utc(dt));
let dt_term = Term::from_field_date_for_search(field, DateTime::from_utc(dt));
Ok(vec![LogicalLiteral::Term(dt_term)])
}
FieldType::Str(ref str_options) => {
@@ -685,8 +700,8 @@ impl QueryParser {
///
/// The terms are identified by a triplet:
/// - tantivy field
/// - field_path: tantivy has JSON fields. It is possible to target a member of a JSON
/// object by naturally extending the json field name with a "." separated field_path
/// - field_path: tantivy has JSON fields. It is possible to target a member of a JSON object by
/// naturally extending the json field name with a "." separated field_path
/// - field_phrase: the phrase that is being searched.
///
/// The literal identifies the targeted field by a so-called *full field path*,
@@ -949,7 +964,8 @@ fn generate_literals_for_json_object(
|| Term::from_field_json_path(field, json_path, json_options.is_expand_dots_enabled());
// Try to convert the phrase to a fast value
if let Some(term) = convert_to_fast_value_and_append_to_json_term(get_term_with_path(), phrase)
if let Some(term) =
convert_to_fast_value_and_append_to_json_term(get_term_with_path(), phrase, true)
{
logical_literals.push(LogicalLiteral::Term(term));
}
@@ -1123,8 +1139,8 @@ mod test {
let query = make_query_parser().parse_query("title:[A TO B]").unwrap();
assert_eq!(
format!("{query:?}"),
"RangeQuery { lower_bound: Included(Term(field=0, type=Str, \"a\")), upper_bound: \
Included(Term(field=0, type=Str, \"b\")), limit: None }"
"RangeQuery { bounds: BoundsRange { lower_bound: Included(Term(field=0, type=Str, \
\"a\")), upper_bound: Included(Term(field=0, type=Str, \"b\")) } }"
);
}

View File

@@ -1,40 +1,19 @@
use std::ops::Bound;
use crate::schema::Type;
mod fast_field_range_doc_set;
mod range_query;
mod range_query_u64_fastfield;
pub use self::range_query::RangeQuery;
pub use self::range_query::*;
pub use self::range_query_u64_fastfield::FastFieldRangeWeight;
// TODO is this correct?
pub(crate) fn is_type_valid_for_fastfield_range_query(typ: Type) -> bool {
match typ {
Type::Str | Type::U64 | Type::I64 | Type::F64 | Type::Bool | Type::Date => true,
Type::Str | Type::U64 | Type::I64 | Type::F64 | Type::Bool | Type::Date | Type::Json => {
true
}
Type::IpAddr => true,
Type::Facet | Type::Bytes | Type::Json => false,
Type::Facet | Type::Bytes => false,
}
}
fn map_bound<TFrom, TTo>(bound: &Bound<TFrom>, transform: impl Fn(&TFrom) -> TTo) -> Bound<TTo> {
use self::Bound::*;
match bound {
Excluded(ref from_val) => Excluded(transform(from_val)),
Included(ref from_val) => Included(transform(from_val)),
Unbounded => Unbounded,
}
}
fn map_bound_res<TFrom, TTo, Err>(
bound: &Bound<TFrom>,
transform: impl Fn(&TFrom) -> Result<TTo, Err>,
) -> Result<Bound<TTo>, Err> {
use self::Bound::*;
Ok(match bound {
Excluded(ref from_val) => Excluded(transform(from_val)?),
Included(ref from_val) => Included(transform(from_val)?),
Unbounded => Unbounded,
})
}

View File

@@ -1,9 +1,9 @@
use std::io;
use std::ops::Bound;
use common::bounds::{map_bound, BoundsRange};
use common::BitSet;
use super::map_bound;
use super::range_query_u64_fastfield::FastFieldRangeWeight;
use crate::index::SegmentReader;
use crate::query::explanation::does_not_match;
@@ -69,17 +69,7 @@ use crate::{DocId, Score};
/// ```
#[derive(Clone, Debug)]
pub struct RangeQuery {
lower_bound: Bound<Term>,
upper_bound: Bound<Term>,
limit: Option<u64>,
}
/// Returns the inner value of a `Bound`
pub(crate) fn inner_bound(val: &Bound<Term>) -> Option<&Term> {
match val {
Bound::Included(term) | Bound::Excluded(term) => Some(term),
Bound::Unbounded => None,
}
bounds: BoundsRange<Term>,
}
impl RangeQuery {
@@ -89,9 +79,7 @@ impl RangeQuery {
/// the `Weight` object is created.
pub fn new(lower_bound: Bound<Term>, upper_bound: Bound<Term>) -> RangeQuery {
RangeQuery {
lower_bound,
upper_bound,
limit: None,
bounds: BoundsRange::new(lower_bound, upper_bound),
}
}
@@ -106,18 +94,10 @@ impl RangeQuery {
}
pub(crate) fn get_term(&self) -> &Term {
inner_bound(&self.lower_bound)
.or(inner_bound(&self.upper_bound))
self.bounds
.get_inner()
.expect("At least one bound must be set")
}
/// Limit the number of term the `RangeQuery` will go through.
///
/// This does not limit the number of matching document, only the number of
/// different terms that get matched.
pub(crate) fn limit(&mut self, limit: u64) {
self.limit = Some(limit);
}
}
impl Query for RangeQuery {
@@ -126,31 +106,90 @@ impl Query for RangeQuery {
let field_type = schema.get_field_entry(self.field()).field_type();
if field_type.is_fast() && is_type_valid_for_fastfield_range_query(self.value_type()) {
Ok(Box::new(FastFieldRangeWeight::new(
self.field(),
self.lower_bound.clone(),
self.upper_bound.clone(),
)))
Ok(Box::new(FastFieldRangeWeight::new(self.bounds.clone())))
} else {
let verify_and_unwrap_term = |val: &Term| val.serialized_value_bytes().to_owned();
Ok(Box::new(RangeWeight {
field: self.field(),
lower_bound: map_bound(&self.lower_bound, verify_and_unwrap_term),
upper_bound: map_bound(&self.upper_bound, verify_and_unwrap_term),
limit: self.limit,
}))
if field_type.is_json() {
return Err(crate::TantivyError::InvalidArgument(
"RangeQuery on JSON is only supported for fast fields currently".to_string(),
));
}
Ok(Box::new(InvertedIndexRangeWeight::new(
self.field(),
&self.bounds.lower_bound,
&self.bounds.upper_bound,
None,
)))
}
}
}
pub struct RangeWeight {
#[derive(Clone, Debug)]
/// `InvertedIndexRangeQuery` is the same as [RangeQuery] but only uses the inverted index
pub struct InvertedIndexRangeQuery {
bounds: BoundsRange<Term>,
limit: Option<u64>,
}
impl InvertedIndexRangeQuery {
/// Create new `InvertedIndexRangeQuery`
pub fn new(lower_bound: Bound<Term>, upper_bound: Bound<Term>) -> InvertedIndexRangeQuery {
InvertedIndexRangeQuery {
bounds: BoundsRange::new(lower_bound, upper_bound),
limit: None,
}
}
/// Limit the number of term the `RangeQuery` will go through.
///
/// This does not limit the number of matching document, only the number of
/// different terms that get matched.
pub fn limit(&mut self, limit: u64) {
self.limit = Some(limit);
}
}
impl Query for InvertedIndexRangeQuery {
fn weight(&self, _enable_scoring: EnableScoring<'_>) -> crate::Result<Box<dyn Weight>> {
let field = self
.bounds
.get_inner()
.expect("At least one bound must be set")
.field();
Ok(Box::new(InvertedIndexRangeWeight::new(
field,
&self.bounds.lower_bound,
&self.bounds.upper_bound,
self.limit,
)))
}
}
/// Range weight on the inverted index
pub struct InvertedIndexRangeWeight {
field: Field,
lower_bound: Bound<Vec<u8>>,
upper_bound: Bound<Vec<u8>>,
limit: Option<u64>,
}
impl RangeWeight {
impl InvertedIndexRangeWeight {
/// Creates a new RangeWeight
///
/// Note: The limit is only enabled with the quickwit feature flag.
pub fn new(
field: Field,
lower_bound: &Bound<Term>,
upper_bound: &Bound<Term>,
limit: Option<u64>,
) -> Self {
let verify_and_unwrap_term = |val: &Term| val.serialized_value_bytes().to_owned();
Self {
field,
lower_bound: map_bound(lower_bound, verify_and_unwrap_term),
upper_bound: map_bound(upper_bound, verify_and_unwrap_term),
limit,
}
}
fn term_range<'a>(&self, term_dict: &'a TermDictionary) -> io::Result<TermStreamer<'a>> {
use std::ops::Bound::*;
let mut term_stream_builder = term_dict.range();
@@ -172,7 +211,7 @@ impl RangeWeight {
}
}
impl Weight for RangeWeight {
impl Weight for InvertedIndexRangeWeight {
fn scorer(&self, reader: &SegmentReader, boost: Score) -> crate::Result<Box<dyn Scorer>> {
let max_doc = reader.max_doc();
let mut doc_bitset = BitSet::with_max_value(max_doc);
@@ -227,6 +266,7 @@ mod tests {
use super::RangeQuery;
use crate::collector::{Count, TopDocs};
use crate::indexer::NoMergePolicy;
use crate::query::range_query::range_query::InvertedIndexRangeQuery;
use crate::query::QueryParser;
use crate::schema::{
Field, IntoIpv6Addr, Schema, TantivyDocument, FAST, INDEXED, STORED, TEXT,
@@ -253,7 +293,7 @@ mod tests {
let reader = index.reader()?;
let searcher = reader.searcher();
let docs_in_the_sixties = RangeQuery::new(
let docs_in_the_sixties = InvertedIndexRangeQuery::new(
Bound::Included(Term::from_field_u64(year_field, 1960)),
Bound::Excluded(Term::from_field_u64(year_field, 1970)),
);
@@ -287,7 +327,7 @@ mod tests {
let reader = index.reader()?;
let searcher = reader.searcher();
let mut docs_in_the_sixties = RangeQuery::new(
let mut docs_in_the_sixties = InvertedIndexRangeQuery::new(
Bound::Included(Term::from_field_u64(year_field, 1960)),
Bound::Excluded(Term::from_field_u64(year_field, 1970)),
);

File diff suppressed because it is too large Load Diff

View File

@@ -3,7 +3,7 @@ use std::str::FromStr;
use base64::engine::general_purpose::STANDARD as BASE64;
use base64::Engine;
use columnar::ColumnType;
use columnar::{ColumnType, NumericalType};
use serde::{Deserialize, Serialize};
use serde_json::Value as JsonValue;
use thiserror::Error;
@@ -102,6 +102,18 @@ const ALL_TYPES: [Type; 10] = [
];
impl Type {
/// Returns the numerical type if applicable
/// It does not do any mapping, e.g. Date is None although it's also stored as I64 in the
/// column store
pub fn numerical_type(&self) -> Option<NumericalType> {
match self {
Type::I64 => Some(NumericalType::I64),
Type::U64 => Some(NumericalType::U64),
Type::F64 => Some(NumericalType::F64),
_ => None,
}
}
/// Returns an iterator over the different values
/// the Type enum can tape.
pub fn iter_values() -> impl Iterator<Item = Type> {
@@ -196,6 +208,11 @@ impl FieldType {
}
}
/// returns true if this is an json field
pub fn is_json(&self) -> bool {
matches!(self, FieldType::JsonObject(_))
}
/// returns true if this is an ip address field
pub fn is_ip_addr(&self) -> bool {
matches!(self, FieldType::IpAddr(_))

View File

@@ -2,12 +2,12 @@ use std::hash::{Hash, Hasher};
use std::net::Ipv6Addr;
use std::{fmt, str};
use columnar::{MonotonicallyMappableToU128, MonotonicallyMappableToU64};
use columnar::MonotonicallyMappableToU128;
use common::json_path_writer::{JSON_END_OF_PATH, JSON_PATH_SEGMENT_SEP_STR};
use common::JsonPathWriter;
use super::date_time_options::DATE_TIME_PRECISION_INDEXED;
use super::Field;
use super::{Field, Schema};
use crate::fastfield::FastValue;
use crate::json_utils::split_json_path;
use crate::schema::{Facet, Type};
@@ -57,6 +57,29 @@ impl Term {
term
}
/// Gets the full path of the field name + optional json path.
pub fn get_full_path(&self, schema: &Schema) -> String {
let field = self.field();
let mut field = schema.get_field_name(field).to_string();
if let Some(json_path) = self.get_json_path() {
field.push('.');
field.push_str(&json_path);
};
field
}
/// Gets the json path if the type is JSON
pub fn get_json_path(&self) -> Option<String> {
let value = self.value();
if let Some((json_path, _)) = value.as_json() {
Some(unsafe {
std::str::from_utf8_unchecked(&json_path[..json_path.len() - 1]).to_string()
})
} else {
None
}
}
pub(crate) fn with_type_and_field(typ: Type, field: Field) -> Term {
let mut term = Self::with_capacity(8);
term.set_field_and_type(field, typ);
@@ -70,7 +93,7 @@ impl Term {
term
}
fn from_fast_value<T: FastValue>(field: Field, val: &T) -> Term {
pub(crate) fn from_fast_value<T: FastValue>(field: Field, val: &T) -> Term {
let mut term = Self::with_type_and_field(T::to_type(), field);
term.set_u64(val.to_u64());
term
@@ -118,8 +141,20 @@ impl Term {
Term::from_fast_value(field, &val)
}
/// Builds a term given a field, and a `DateTime` value
/// Builds a term given a field, and a `DateTime` value.
///
/// The contained value may not match the value, due do the truncation used
/// for indexed data [super::DATE_TIME_PRECISION_INDEXED].
/// To create a term used for search use `from_field_date_for_search`.
pub fn from_field_date(field: Field, val: DateTime) -> Term {
Term::from_fast_value(field, &val)
}
/// Builds a term given a field, and a `DateTime` value to be used in searching the inverted
/// index.
/// It truncates the `DateTime` to the precision used in the index
/// ([super::DATE_TIME_PRECISION_INDEXED]).
pub fn from_field_date_for_search(field: Field, val: DateTime) -> Term {
Term::from_fast_value(field, &val.truncate(DATE_TIME_PRECISION_INDEXED))
}
@@ -191,13 +226,7 @@ impl Term {
/// It will not clear existing bytes.
pub fn append_type_and_fast_value<T: FastValue>(&mut self, val: T) {
self.0.push(T::to_type().to_code());
let value = if T::to_type() == Type::Date {
DateTime::from_u64(val.to_u64())
.truncate(DATE_TIME_PRECISION_INDEXED)
.to_u64()
} else {
val.to_u64()
};
let value = val.to_u64();
self.0.extend(value.to_be_bytes().as_ref());
}
@@ -324,6 +353,11 @@ where B: AsRef<[u8]>
ValueBytes(data)
}
/// Wraps a object holding Vec<u8>
pub fn to_owned(&self) -> ValueBytes<Vec<u8>> {
ValueBytes(self.0.as_ref().to_vec())
}
fn typ_code(&self) -> u8 {
self.0.as_ref()[0]
}
@@ -345,7 +379,7 @@ where B: AsRef<[u8]>
if self.typ() != T::to_type() {
return None;
}
let value_bytes = self.value_bytes();
let value_bytes = self.raw_value_bytes_payload();
let value_u64 = u64::from_be_bytes(value_bytes.try_into().ok()?);
Some(T::from_u64(value_u64))
}
@@ -390,7 +424,7 @@ where B: AsRef<[u8]>
if self.typ() != Type::Str {
return None;
}
str::from_utf8(self.value_bytes()).ok()
str::from_utf8(self.raw_value_bytes_payload()).ok()
}
/// Returns the facet associated with the term.
@@ -401,7 +435,7 @@ where B: AsRef<[u8]>
if self.typ() != Type::Facet {
return None;
}
let facet_encode_str = str::from_utf8(self.value_bytes()).ok()?;
let facet_encode_str = str::from_utf8(self.raw_value_bytes_payload()).ok()?;
Some(Facet::from_encoded_string(facet_encode_str.to_string()))
}
@@ -412,7 +446,7 @@ where B: AsRef<[u8]>
if self.typ() != Type::Bytes {
return None;
}
Some(self.value_bytes())
Some(self.raw_value_bytes_payload())
}
/// Returns a `Ipv6Addr` value from the term.
@@ -420,7 +454,7 @@ where B: AsRef<[u8]>
if self.typ() != Type::IpAddr {
return None;
}
let ip_u128 = u128::from_be_bytes(self.value_bytes().try_into().ok()?);
let ip_u128 = u128::from_be_bytes(self.raw_value_bytes_payload().try_into().ok()?);
Some(Ipv6Addr::from_u128(ip_u128))
}
@@ -441,7 +475,7 @@ where B: AsRef<[u8]>
if self.typ() != Type::Json {
return None;
}
let bytes = self.value_bytes();
let bytes = self.raw_value_bytes_payload();
let pos = bytes.iter().cloned().position(|b| b == JSON_END_OF_PATH)?;
// split at pos + 1, so that json_path_bytes includes the JSON_END_OF_PATH byte.
@@ -456,16 +490,25 @@ where B: AsRef<[u8]>
if self.typ() != Type::Json {
return None;
}
let bytes = self.value_bytes();
let bytes = self.raw_value_bytes_payload();
let pos = bytes.iter().cloned().position(|b| b == JSON_END_OF_PATH)?;
Some(ValueBytes::wrap(&bytes[pos + 1..]))
}
/// Returns the serialized value of ValueBytes without the type.
fn value_bytes(&self) -> &[u8] {
/// Returns the raw value of ValueBytes payload, without the type tag.
pub(crate) fn raw_value_bytes_payload(&self) -> &[u8] {
&self.0.as_ref()[1..]
}
/// Returns the serialized value of ValueBytes payload, without the type tag.
pub(crate) fn value_bytes_payload(&self) -> Vec<u8> {
if let Some(value_bytes) = self.as_json_value_bytes() {
value_bytes.raw_value_bytes_payload().to_vec()
} else {
self.raw_value_bytes_payload().to_vec()
}
}
/// Returns the serialized representation of Term.
///
/// Do NOT rely on this byte representation in the index.

View File

@@ -4,6 +4,7 @@ use std::marker::PhantomData;
use std::ops::{Bound, RangeBounds};
use std::sync::Arc;
use common::bounds::{transform_bound_inner_res, TransformBound};
use common::file_slice::FileSlice;
use common::{BinarySerializable, OwnedBytes};
use tantivy_fst::automaton::AlwaysMatch;
@@ -56,29 +57,6 @@ impl Dictionary<VoidSSTable> {
}
}
fn map_bound<TFrom, TTo>(bound: &Bound<TFrom>, transform: impl Fn(&TFrom) -> TTo) -> Bound<TTo> {
use self::Bound::*;
match bound {
Excluded(ref from_val) => Bound::Excluded(transform(from_val)),
Included(ref from_val) => Bound::Included(transform(from_val)),
Unbounded => Unbounded,
}
}
/// Takes a bound and transforms the inner value into a new bound via a closure.
/// The bound variant may change by the value returned value from the closure.
fn transform_bound_inner<TFrom, TTo>(
bound: &Bound<TFrom>,
transform: impl Fn(&TFrom) -> io::Result<Bound<TTo>>,
) -> io::Result<Bound<TTo>> {
use self::Bound::*;
Ok(match bound {
Excluded(ref from_val) => transform(from_val)?,
Included(ref from_val) => transform(from_val)?,
Unbounded => Unbounded,
})
}
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum TermOrdHit {
/// Exact term ord hit
@@ -409,18 +387,18 @@ impl<TSSTable: SSTable> Dictionary<TSSTable> {
lower_bound: Bound<K>,
upper_bound: Bound<K>,
) -> io::Result<(Bound<TermOrdinal>, Bound<TermOrdinal>)> {
let lower_bound = transform_bound_inner(&lower_bound, |start_bound_bytes| {
let lower_bound = transform_bound_inner_res(&lower_bound, |start_bound_bytes| {
let ord = self.term_ord_or_next(start_bound_bytes)?;
match ord {
TermOrdHit::Exact(ord) => Ok(map_bound(&lower_bound, |_| ord)),
TermOrdHit::Next(ord) => Ok(Bound::Included(ord)), // Change bounds to included
TermOrdHit::Exact(ord) => Ok(TransformBound::Existing(ord)),
TermOrdHit::Next(ord) => Ok(TransformBound::NewBound(Bound::Included(ord))), /* Change bounds to included */
}
})?;
let upper_bound = transform_bound_inner(&upper_bound, |end_bound_bytes| {
let upper_bound = transform_bound_inner_res(&upper_bound, |end_bound_bytes| {
let ord = self.term_ord_or_next(end_bound_bytes)?;
match ord {
TermOrdHit::Exact(ord) => Ok(map_bound(&upper_bound, |_| ord)),
TermOrdHit::Next(ord) => Ok(Bound::Excluded(ord)), // Change bounds to excluded
TermOrdHit::Exact(ord) => Ok(TransformBound::Existing(ord)),
TermOrdHit::Next(ord) => Ok(TransformBound::NewBound(Bound::Excluded(ord))), /* Change bounds to excluded */
}
})?;
Ok((lower_bound, upper_bound))