Skip to content

Commit

Permalink
fix term date truncation
Browse files Browse the repository at this point in the history
  • Loading branch information
PSeitz committed Jul 30, 2024
1 parent ba33a2a commit f91ced8
Show file tree
Hide file tree
Showing 8 changed files with 177 additions and 52 deletions.
3 changes: 2 additions & 1 deletion src/core/json_utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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);
}
Expand Down
2 changes: 1 addition & 1 deletion src/indexer/merger.rs
Original file line number Diff line number Diff line change
Expand Up @@ -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)
)])?,
Expand Down
2 changes: 1 addition & 1 deletion src/query/more_like_this/more_like_this.rs
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
}
Expand Down
13 changes: 6 additions & 7 deletions src/query/query_parser/query_parser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -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.
///
Expand All @@ -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
Expand Down Expand Up @@ -273,7 +272,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.
/// in the query.

Check warning on line 275 in src/query/query_parser/query_parser.rs

View workflow job for this annotation

GitHub Actions / clippy

doc list item missing indentation

warning: doc list item missing indentation --> src/query/query_parser/query_parser.rs:275:9 | 275 | /// in the query. | ^ | = help: if this is supposed to be its own paragraph, add a blank line = help: for further information visit https://rust-lang.github.io/rust-clippy/master/index.html#doc_lazy_continuation help: indent this line | 275 | /// in the query. | +++
pub fn for_index(index: &Index, default_fields: Vec<Field>) -> QueryParser {
QueryParser::new(index.schema(), default_fields, index.tokenizers().clone())
}
Expand Down Expand Up @@ -569,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) => {
Expand Down Expand Up @@ -701,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*,
Expand Down
5 changes: 1 addition & 4 deletions src/query/range_query/range_query.rs
Original file line number Diff line number Diff line change
Expand Up @@ -116,10 +116,7 @@ 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.bounds.clone(),
)))
Ok(Box::new(FastFieldRangeWeight::new(self.bounds.clone())))
} else {
if field_type.is_json() {
return Err(crate::TantivyError::InvalidArgument(
Expand Down
177 changes: 148 additions & 29 deletions src/query/range_query/range_query_u64_fastfield.rs
Original file line number Diff line number Diff line change
Expand Up @@ -10,24 +10,22 @@ use columnar::{
StrColumn,
};
use common::bounds::{BoundsRange, TransformBound};
use common::BinarySerializable;

use super::fast_field_range_doc_set::RangeDocSet;
use crate::query::{AllScorer, ConstScorer, EmptyScorer, Explanation, Query, Scorer, Weight};
use crate::schema::{Field, Type, ValueBytes};
use crate::schema::{Type, ValueBytes};
use crate::{DocId, DocSet, Score, SegmentReader, TantivyError, Term};

/// `FastFieldRangeWeight` uses the fast field to execute range queries.
#[derive(Clone, Debug)]
pub struct FastFieldRangeWeight {
bounds: BoundsRange<Term>,
field: Field,
}

impl FastFieldRangeWeight {
/// Create a new FastFieldRangeWeight
pub(crate) fn new(field: Field, bounds: BoundsRange<Term>) -> Self {
Self { bounds, field }
pub(crate) fn new(bounds: BoundsRange<Term>) -> Self {
Self { bounds }
}
}

Expand All @@ -46,12 +44,12 @@ impl Weight for FastFieldRangeWeight {
if self.bounds.is_unbounded() {
return Ok(Box::new(AllScorer::new(reader.max_doc())));
}
let field_type = reader.schema().get_field_entry(self.field).field_type();

let term = self
.bounds
.get_inner()
.expect("At least one bound must be set");
let field_type = reader.schema().get_field_entry(term.field()).field_type();
assert_eq!(
term.typ(),
field_type.value_type(),
Expand All @@ -62,10 +60,6 @@ impl Weight for FastFieldRangeWeight {
let field_name = term.get_full_path(reader.schema());

let get_value_bytes = |term: &Term| term.value().value_bytes_payload();
let get_term_u64_internal_representation = |term: &Term| {
let bytes = term.value().value_bytes_payload();
u64::from_be(BinarySerializable::deserialize(&mut &bytes[..]).unwrap())
};

let term_value = term.value();
if field_type.is_json() {
Expand Down Expand Up @@ -175,11 +169,35 @@ impl Weight for FastFieldRangeWeight {
field_type
);

let bounds = self.bounds.map_bound(get_term_u64_internal_representation);
let bounds = self.bounds.map_bound_res(|term| {
let value = term.value();
let val = if let Some(val) = value.as_u64() {
val
} else if let Some(val) = value.as_i64() {
val.to_u64()
} else if let Some(val) = value.as_f64() {
val.to_u64()
} else if let Some(val) = value.as_date() {
val.to_u64()
} else {
return Err(TantivyError::InvalidArgument(format!(
"Expected term with u64, i64, f64 or date, but got {:?}",
term
)));
};
Ok(val)
})?;

let fast_field_reader = reader.fast_fields();
let Some((column, _col_type)) =
fast_field_reader.u64_lenient_for_type(None, &field_name)?
let Some((column, _col_type)) = fast_field_reader.u64_lenient_for_type(
Some(&[
ColumnType::U64,
ColumnType::I64,
ColumnType::F64,
ColumnType::DateTime,
]),
&field_name,
)?
else {
return Ok(Box::new(EmptyScorer));
};
Expand Down Expand Up @@ -212,7 +230,7 @@ fn search_on_json_numerical_field(
boost: Score,
) -> crate::Result<Box<dyn Scorer>> {
// Since we don't know which type was interpolated for the internal column whe
// have to check for all types (only one exists)
// have to check for all numeric types (only one exists)
let allowed_column_types: Option<&[ColumnType]> =
Some(&[ColumnType::F64, ColumnType::I64, ColumnType::U64]);
let fast_field_reader = reader.fast_fields();
Expand Down Expand Up @@ -455,7 +473,8 @@ pub mod tests {
use crate::query::range_query::range_query_u64_fastfield::FastFieldRangeWeight;
use crate::query::{QueryParser, RangeQuery, Weight};
use crate::schema::{
Field, NumericOptions, Schema, SchemaBuilder, FAST, INDEXED, STORED, STRING, TEXT,
DateOptions, Field, NumericOptions, Schema, SchemaBuilder, FAST, INDEXED, STORED, STRING,
TEXT,
};
use crate::{Index, IndexWriter, Term, TERMINATED};

Expand Down Expand Up @@ -518,6 +537,89 @@ pub mod tests {
Ok(())
}

#[test]
fn test_date_range_query() {
let mut schema_builder = Schema::builder();
let options = DateOptions::default()
.set_precision(common::DateTimePrecision::Microseconds)
.set_fast();
let date_field = schema_builder.add_date_field("date", options);
let schema = schema_builder.build();

let index = Index::create_in_ram(schema.clone());
{
let mut index_writer = index.writer_with_num_threads(1, 50_000_000).unwrap();
// This is added a string and creates a string column!
index_writer
.add_document(doc!(date_field => DateTime::from_utc(
OffsetDateTime::parse("2022-12-01T00:00:01Z", &Rfc3339).unwrap(),
)))
.unwrap();
index_writer
.add_document(doc!(date_field => DateTime::from_utc(
OffsetDateTime::parse("2023-12-01T00:00:01Z", &Rfc3339).unwrap(),
)))
.unwrap();
index_writer
.add_document(doc!(date_field => DateTime::from_utc(
OffsetDateTime::parse("2015-02-01T00:00:00.001Z", &Rfc3339).unwrap(),
)))
.unwrap();
index_writer.commit().unwrap();
}

// Date field
let dt1 =
DateTime::from_utc(OffsetDateTime::parse("2022-12-01T00:00:01Z", &Rfc3339).unwrap());
let dt2 =
DateTime::from_utc(OffsetDateTime::parse("2023-12-01T00:00:01Z", &Rfc3339).unwrap());
let dt3 = DateTime::from_utc(
OffsetDateTime::parse("2015-02-01T00:00:00.001Z", &Rfc3339).unwrap(),
);
let dt4 = DateTime::from_utc(
OffsetDateTime::parse("2015-02-01T00:00:00.002Z", &Rfc3339).unwrap(),
);

let reader = index.reader().unwrap();
let searcher = reader.searcher();
let count = |range_query: RangeQuery| searcher.search(&range_query, &Count).unwrap();
assert_eq!(
count(RangeQuery::new(
Bound::Included(Term::from_field_date(date_field, dt3)),
Bound::Excluded(Term::from_field_date(date_field, dt4)),
)),
1
);
assert_eq!(
count(RangeQuery::new(
Bound::Included(Term::from_field_date(date_field, dt3)),
Bound::Included(Term::from_field_date(date_field, dt4)),
)),
1
);
assert_eq!(
count(RangeQuery::new(
Bound::Included(Term::from_field_date(date_field, dt1)),
Bound::Included(Term::from_field_date(date_field, dt2)),
)),
2
);
assert_eq!(
count(RangeQuery::new(
Bound::Included(Term::from_field_date(date_field, dt1)),
Bound::Excluded(Term::from_field_date(date_field, dt2)),
)),
1
);
assert_eq!(
count(RangeQuery::new(
Bound::Excluded(Term::from_field_date(date_field, dt1)),
Bound::Excluded(Term::from_field_date(date_field, dt2)),
)),
0
);
}

fn get_json_term<T: FastValue>(field: Field, path: &str, value: T) -> Term {
let mut term = Term::from_field_json_path(field, path, true);
term.append_type_and_fast_value(value);
Expand Down Expand Up @@ -548,6 +650,10 @@ pub mod tests {
"date": "2023-12-01T00:00:01Z"
});
index_writer.add_document(doc!(json_field => doc)).unwrap();
let doc = json!({
"date": "2015-02-01T00:00:00.001Z"
});
index_writer.add_document(doc!(json_field => doc)).unwrap();

index_writer.commit().unwrap();
}
Expand Down Expand Up @@ -631,6 +737,13 @@ pub mod tests {
)),
2
);
assert_eq!(
count(RangeQuery::new(
Bound::Included(get_json_term(json_field, "id_i64", 1000i64)),
Bound::Excluded(get_json_term(json_field, "id_i64", 1001i64)),
)),
1
);

// u64 on i64
assert_eq!(
Expand Down Expand Up @@ -718,6 +831,18 @@ pub mod tests {
)),
0
);
// Date precision test. We don't want to truncate the precision
let dt3 = DateTime::from_utc(
OffsetDateTime::parse("2015-02-01T00:00:00.001Z", &Rfc3339).unwrap(),
);
let dt4 = DateTime::from_utc(
OffsetDateTime::parse("2015-02-01T00:00:00.002Z", &Rfc3339).unwrap(),
);
let query = RangeQuery::new(
Bound::Included(get_json_term(json_field, "date", dt3)),
Bound::Excluded(get_json_term(json_field, "date", dt4)),
);
assert_eq!(count(query), 1);
}

#[derive(Clone, Debug)]
Expand Down Expand Up @@ -796,13 +921,10 @@ pub mod tests {
writer.add_document(doc!(field=>52_000u64)).unwrap();
writer.commit().unwrap();
let searcher = index.reader().unwrap().searcher();
let range_query = FastFieldRangeWeight::new(
field,
BoundsRange::new(
Bound::Included(Term::from_field_u64(field, 50_000)),
Bound::Included(Term::from_field_u64(field, 50_002)),
),
);
let range_query = FastFieldRangeWeight::new(BoundsRange::new(
Bound::Included(Term::from_field_u64(field, 50_000)),
Bound::Included(Term::from_field_u64(field, 50_002)),
));
let scorer = range_query
.scorer(searcher.segment_reader(0), 1.0f32)
.unwrap();
Expand Down Expand Up @@ -1158,13 +1280,10 @@ pub mod ip_range_tests {
}
writer.commit().unwrap();
let searcher = index.reader().unwrap().searcher();
let range_weight = FastFieldRangeWeight::new(
ips_field,
BoundsRange::new(
Bound::Included(Term::from_field_ip_addr(ips_field, ip_addrs[1])),
Bound::Included(Term::from_field_ip_addr(ips_field, ip_addrs[2])),
),
);
let range_weight = FastFieldRangeWeight::new(BoundsRange::new(
Bound::Included(Term::from_field_ip_addr(ips_field, ip_addrs[1])),
Bound::Included(Term::from_field_ip_addr(ips_field, ip_addrs[2])),
));

let count =
crate::query::weight::Weight::count(&range_weight, searcher.segment_reader(0)).unwrap();
Expand Down
3 changes: 3 additions & 0 deletions src/schema/field_type.rs
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,9 @@ 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),
Expand Down
Loading

0 comments on commit f91ced8

Please sign in to comment.