Skip to content

Commit

Permalink
enable date for range aggregation
Browse files Browse the repository at this point in the history
  • Loading branch information
PSeitz committed Nov 25, 2022
1 parent 788ac05 commit 514c0f0
Show file tree
Hide file tree
Showing 6 changed files with 168 additions and 37 deletions.
6 changes: 6 additions & 0 deletions src/aggregation/agg_result.rs
Expand Up @@ -245,4 +245,10 @@ pub struct RangeBucketEntry {
/// The to range of the bucket. Equals `f64::MAX` when `None`.
#[serde(skip_serializing_if = "Option::is_none")]
pub to: Option<f64>,
/// The optional string representation for the `from` range.
#[serde(skip_serializing_if = "Option::is_none")]
pub from_as_string: Option<String>,
/// The optional string representation for the `to` range.
#[serde(skip_serializing_if = "Option::is_none")]
pub to_as_string: Option<String>,
}
23 changes: 4 additions & 19 deletions src/aggregation/bucket/histogram/histogram.rs
Expand Up @@ -4,19 +4,17 @@ use std::fmt::Display;
use fastfield_codecs::Column;
use itertools::Itertools;
use serde::{Deserialize, Serialize};
use time::format_description::well_known::Rfc3339;
use time::OffsetDateTime;

use crate::aggregation::agg_req::AggregationsInternal;
use crate::aggregation::agg_req_with_accessor::{
AggregationsWithAccessor, BucketAggregationWithAccessor,
};
use crate::aggregation::agg_result::BucketEntry;
use crate::aggregation::f64_from_fastfield_u64;
use crate::aggregation::intermediate_agg_result::{
IntermediateAggregationResults, IntermediateBucketResult, IntermediateHistogramBucketEntry,
};
use crate::aggregation::segment_agg_result::SegmentAggregationResultsCollector;
use crate::aggregation::{f64_from_fastfield_u64, format_date};
use crate::schema::{Schema, Type};
use crate::{DocId, TantivyError};

Expand Down Expand Up @@ -534,22 +532,9 @@ pub(crate) fn intermediate_histogram_buckets_to_final_buckets(
.ok_or_else(|| TantivyError::FieldNotFound(histogram_req.field.to_string()))?;
if schema.get_field_entry(field).field_type().is_date() {
for bucket in buckets.iter_mut() {
match bucket.key {
crate::aggregation::Key::F64(val) => {
let datetime = OffsetDateTime::from_unix_timestamp_nanos(1_000 * (val as i128))
.map_err(|err| {
TantivyError::InvalidArgument(format!(
"Could not convert {:?} to OffsetDateTime, err {:?}",
val, err
))
})?;
let key_as_string = datetime.format(&Rfc3339).map_err(|_err| {
TantivyError::InvalidArgument("Could not serialize date".to_string())
})?;

bucket.key_as_string = Some(key_as_string);
}
_ => {}
if let crate::aggregation::Key::F64(val) = bucket.key {
let key_as_string = format_date(val as i64)?;
bucket.key_as_string = Some(key_as_string);
}
}
}
Expand Down
132 changes: 121 additions & 11 deletions src/aggregation/bucket/range.rs
@@ -1,6 +1,7 @@
use std::fmt::Debug;
use std::ops::Range;

use fastfield_codecs::MonotonicallyMappableToU64;
use rustc_hash::FxHashMap;
use serde::{Deserialize, Serialize};

Expand All @@ -11,7 +12,9 @@ use crate::aggregation::intermediate_agg_result::{
IntermediateBucketResult, IntermediateRangeBucketEntry, IntermediateRangeBucketResult,
};
use crate::aggregation::segment_agg_result::{BucketCount, SegmentAggregationResultsCollector};
use crate::aggregation::{f64_from_fastfield_u64, f64_to_fastfield_u64, Key, SerializedKey};
use crate::aggregation::{
f64_from_fastfield_u64, f64_to_fastfield_u64, format_date, Key, SerializedKey,
};
use crate::schema::Type;
use crate::{DocId, TantivyError};

Expand Down Expand Up @@ -136,6 +139,10 @@ pub(crate) struct SegmentRangeBucketEntry {
/// The to range of the bucket. Equals `f64::MAX` when `None`. Open interval, `to` is not
/// inclusive.
pub to: Option<f64>,
/// The optional string representation for the `from` range.
pub from_as_string: Option<String>,
/// The optional string representation for the `to` range.
pub to_as_string: Option<String>,
}

impl Debug for SegmentRangeBucketEntry {
Expand Down Expand Up @@ -165,6 +172,8 @@ impl SegmentRangeBucketEntry {
sub_aggregation,
from: self.from,
to: self.to,
from_as_string: self.from_as_string,
to_as_string: self.to_as_string,
})
}
}
Expand All @@ -181,7 +190,7 @@ impl SegmentRangeCollector {
.into_iter()
.map(move |range_bucket| {
Ok((
range_to_string(&range_bucket.range, &field_type),
range_to_string(&range_bucket.range, &field_type)?,
range_bucket
.bucket
.into_intermediate_bucket_entry(&agg_with_accessor.sub_aggregation)?,
Expand Down Expand Up @@ -209,8 +218,8 @@ impl SegmentRangeCollector {
let key = range
.key
.clone()
.map(Key::Str)
.unwrap_or_else(|| range_to_key(&range.range, &field_type));
.map(|key| Ok(Key::Str(key)))
.unwrap_or_else(|| range_to_key(&range.range, &field_type))?;
let to = if range.range.end == u64::MAX {
None
} else {
Expand All @@ -228,9 +237,28 @@ impl SegmentRangeCollector {
sub_aggregation,
)?)
};

let get_as_string = |val| -> crate::Result<Option<String>> {
if field_type == Type::Date {
if let Some(val) = val {
let val: String = format_date(val as i64)?;
Ok(Some(val))
} else {
Ok(None)
}
} else {
Ok(None)
}
};

let from_as_string = get_as_string(from)?;
let to_as_string = get_as_string(to)?;

Ok(SegmentRangeAndBucketEntry {
range: range.range.clone(),
bucket: SegmentRangeBucketEntry {
from_as_string,
to_as_string,
doc_count: 0,
sub_aggregation,
key,
Expand Down Expand Up @@ -402,34 +430,45 @@ fn extend_validate_ranges(
Ok(converted_buckets)
}

pub(crate) fn range_to_string(range: &Range<u64>, field_type: &Type) -> String {
pub(crate) fn range_to_string(range: &Range<u64>, field_type: &Type) -> crate::Result<String> {
// is_start is there for malformed requests, e.g. ig the user passes the range u64::MIN..0.0,
// it should be rendered as "*-0" and not "*-*"
let to_str = |val: u64, is_start: bool| {
if (is_start && val == u64::MIN) || (!is_start && val == u64::MAX) {
"*".to_string()
Ok("*".to_string())
} else if *field_type == Type::Date {
let val = i64::from_u64(val);
format_date(val)
} else {
f64_from_fastfield_u64(val, field_type).to_string()
Ok(f64_from_fastfield_u64(val, field_type).to_string())
}
};

format!("{}-{}", to_str(range.start, true), to_str(range.end, false))
Ok(format!(
"{}-{}",
to_str(range.start, true)?,
to_str(range.end, false)?
))
}

pub(crate) fn range_to_key(range: &Range<u64>, field_type: &Type) -> Key {
Key::Str(range_to_string(range, field_type))
pub(crate) fn range_to_key(range: &Range<u64>, field_type: &Type) -> crate::Result<Key> {
Ok(Key::Str(range_to_string(range, field_type)?))
}

#[cfg(test)]
mod tests {

use fastfield_codecs::MonotonicallyMappableToU64;
use serde_json::Value;

use super::*;
use crate::aggregation::agg_req::{
Aggregation, Aggregations, BucketAggregation, BucketAggregationType,
};
use crate::aggregation::tests::{exec_request_with_query, get_test_index_with_num_docs};
use crate::aggregation::tests::{
exec_request, exec_request_with_query, get_test_index_2_segments,
get_test_index_with_num_docs,
};

pub fn get_collector_from_ranges(
ranges: Vec<RangeAggregationRange>,
Expand Down Expand Up @@ -567,6 +606,77 @@ mod tests {
Ok(())
}

#[test]
fn range_date_test_single_segment() -> crate::Result<()> {
range_date_test_with_opt(true)
}

#[test]
fn range_date_test_multi_segment() -> crate::Result<()> {
range_date_test_with_opt(false)
}

fn range_date_test_with_opt(merge_segments: bool) -> crate::Result<()> {
let index = get_test_index_2_segments(merge_segments)?;

let agg_req: Aggregations = vec![(
"date_ranges".to_string(),
Aggregation::Bucket(BucketAggregation {
bucket_agg: BucketAggregationType::Range(RangeAggregation {
field: "date".to_string(),
ranges: vec![
RangeAggregationRange {
key: None,
from: None,
to: Some(1546300800000000.0f64),
},
RangeAggregationRange {
key: None,
from: Some(1546300800000000.0f64),
to: Some(1546387200000000.0f64),
},
],
keyed: false,
}),
sub_aggregation: Default::default(),
}),
)]
.into_iter()
.collect();

let agg_res = exec_request(agg_req, &index)?;

let res: Value = serde_json::from_str(&serde_json::to_string(&agg_res)?)?;

assert_eq!(
res["date_ranges"]["buckets"][0]["from_as_string"],
Value::Null
);
assert_eq!(
res["date_ranges"]["buckets"][0]["key"],
"*-2019-01-01T00:00:00Z"
);
assert_eq!(
res["date_ranges"]["buckets"][1]["from_as_string"],
"2019-01-01T00:00:00Z"
);
assert_eq!(
res["date_ranges"]["buckets"][1]["to_as_string"],
"2019-01-02T00:00:00Z"
);

assert_eq!(
res["date_ranges"]["buckets"][2]["from_as_string"],
"2019-01-02T00:00:00Z"
);
assert_eq!(
res["date_ranges"]["buckets"][2]["to_as_string"],
Value::Null
);

Ok(())
}

#[test]
fn range_custom_key_keyed_buckets_test() -> crate::Result<()> {
let index = get_test_index_with_num_docs(false, 100)?;
Expand Down
18 changes: 18 additions & 0 deletions src/aggregation/date.rs
@@ -0,0 +1,18 @@
use time::format_description::well_known::Rfc3339;
use time::OffsetDateTime;

use crate::TantivyError;

pub(crate) fn format_date(val: i64) -> crate::Result<String> {
let datetime =
OffsetDateTime::from_unix_timestamp_nanos(1_000 * (val as i128)).map_err(|err| {
TantivyError::InvalidArgument(format!(
"Could not convert {:?} to OffsetDateTime, err {:?}",
val, err
))
})?;
let key_as_string = datetime
.format(&Rfc3339)
.map_err(|_err| TantivyError::InvalidArgument("Could not serialize date".to_string()))?;
Ok(key_as_string)
}
12 changes: 12 additions & 0 deletions src/aggregation/intermediate_agg_result.rs
Expand Up @@ -581,6 +581,12 @@ pub struct IntermediateRangeBucketEntry {
/// The to range of the bucket. Equals `f64::MAX` when `None`.
#[serde(skip_serializing_if = "Option::is_none")]
pub to: Option<f64>,
/// The optional string representation for the `from` range.
#[serde(skip_serializing_if = "Option::is_none")]
pub from_as_string: Option<String>,
/// The optional string representation for the `to` range.
#[serde(skip_serializing_if = "Option::is_none")]
pub to_as_string: Option<String>,
}

impl IntermediateRangeBucketEntry {
Expand All @@ -597,6 +603,8 @@ impl IntermediateRangeBucketEntry {
.into_final_bucket_result_internal(req, schema)?,
to: self.to,
from: self.from,
to_as_string: self.to_as_string,
from_as_string: self.from_as_string,
})
}
}
Expand Down Expand Up @@ -652,6 +660,8 @@ mod tests {
sub_aggregation: Default::default(),
from: None,
to: None,
from_as_string: None,
to_as_string: None,
},
);
}
Expand All @@ -677,6 +687,8 @@ mod tests {
key: Key::Str(key.to_string()),
doc_count: *doc_count,
from: None,
to_as_string: None,
from_as_string: None,
to: None,
sub_aggregation: get_sub_test_tree(&[(
sub_aggregation_key.to_string(),
Expand Down
14 changes: 7 additions & 7 deletions src/aggregation/mod.rs
Expand Up @@ -158,6 +158,7 @@ mod agg_req_with_accessor;
pub mod agg_result;
pub mod bucket;
mod collector;
mod date;
pub mod intermediate_agg_result;
pub mod metric;
mod segment_agg_result;
Expand All @@ -168,6 +169,7 @@ pub use collector::{
AggregationCollector, AggregationSegmentCollector, DistributedAggregationCollector,
MAX_BUCKET_COUNT,
};
pub(crate) use date::format_date;
use fastfield_codecs::MonotonicallyMappableToU64;
use itertools::Itertools;
use serde::{Deserialize, Serialize};
Expand Down Expand Up @@ -284,23 +286,21 @@ impl Display for Key {
/// Inverse of `to_fastfield_u64`. Used to convert to `f64` for metrics.
///
/// # Panics
/// Only `u64`, `f64`, and `i64` are supported.
/// Only `u64`, `f64`, `date`, and `i64` are supported.
pub(crate) fn f64_from_fastfield_u64(val: u64, field_type: &Type) -> f64 {
match field_type {
Type::U64 => val as f64,
Type::I64 => i64::from_u64(val) as f64,
Type::Date => i64::from_u64(val) as f64,
Type::I64 | Type::Date => i64::from_u64(val) as f64,
Type::F64 => f64::from_u64(val),
_ => {
panic!("unexpected type {:?}. This should not happen", field_type)
}
}
}

/// Converts the `f64` value to fast field value space.
/// Converts the `f64` value to fast field value space, which is always u64.
///
/// If the fast field has `u64`, values are stored as `u64` in the fast field.
/// A `f64` value of e.g. `2.0` therefore needs to be converted to `1u64`.
/// If the fast field has `u64`, values are stored unchanged as `u64` in the fast field.
///
/// If the fast field has `f64` values are converted and stored to `u64` using a
/// monotonic mapping.
Expand All @@ -310,7 +310,7 @@ pub(crate) fn f64_from_fastfield_u64(val: u64, field_type: &Type) -> f64 {
pub(crate) fn f64_to_fastfield_u64(val: f64, field_type: &Type) -> Option<u64> {
match field_type {
Type::U64 => Some(val as u64),
Type::I64 => Some((val as i64).to_u64()),
Type::I64 | Type::Date => Some((val as i64).to_u64()),
Type::F64 => Some(val.to_u64()),
_ => None,
}
Expand Down

0 comments on commit 514c0f0

Please sign in to comment.