mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-05-19 22:40:40 +00:00
fix: adjust fill behavior of range query (#3489)
This commit is contained in:
@@ -14,8 +14,8 @@
|
||||
|
||||
use std::any::Any;
|
||||
use std::cmp::Ordering;
|
||||
use std::collections::hash_map::Entry;
|
||||
use std::collections::HashMap;
|
||||
use std::collections::btree_map::Entry;
|
||||
use std::collections::{BTreeMap, HashMap};
|
||||
use std::fmt::Display;
|
||||
use std::pin::Pin;
|
||||
use std::sync::Arc;
|
||||
@@ -218,14 +218,15 @@ impl Display for Fill {
|
||||
}
|
||||
|
||||
impl Fill {
|
||||
pub fn try_from_str(value: &str, datatype: &DataType) -> DfResult<Self> {
|
||||
pub fn try_from_str(value: &str, datatype: &DataType) -> DfResult<Option<Self>> {
|
||||
let s = value.to_uppercase();
|
||||
match s.as_str() {
|
||||
"NULL" | "" => Ok(Self::Null),
|
||||
"PREV" => Ok(Self::Prev),
|
||||
"" => Ok(None),
|
||||
"NULL" => Ok(Some(Self::Null)),
|
||||
"PREV" => Ok(Some(Self::Prev)),
|
||||
"LINEAR" => {
|
||||
if datatype.is_numeric() {
|
||||
Ok(Self::Linear)
|
||||
Ok(Some(Self::Linear))
|
||||
} else {
|
||||
Err(DataFusionError::Plan(format!(
|
||||
"Use FILL LINEAR on Non-numeric DataType {}",
|
||||
@@ -240,13 +241,17 @@ impl Fill {
|
||||
s, err
|
||||
))
|
||||
})
|
||||
.map(Fill::Const),
|
||||
.map(|x| Some(Fill::Const(x))),
|
||||
}
|
||||
}
|
||||
|
||||
/// The input `data` contains data on a complete time series.
|
||||
/// If the filling strategy is `PREV` or `LINEAR`, caller must be ensured that the incoming `ts`&`data` is ascending time order.
|
||||
pub fn apply_fill_strategy(&self, ts: &[i64], data: &mut [ScalarValue]) -> DfResult<()> {
|
||||
// No calculation need in `Fill::Null`
|
||||
if matches!(self, Fill::Null) {
|
||||
return Ok(());
|
||||
}
|
||||
let len = data.len();
|
||||
if *self == Fill::Linear {
|
||||
return Self::fill_linear(ts, data);
|
||||
@@ -254,7 +259,6 @@ impl Fill {
|
||||
for i in 0..len {
|
||||
if data[i].is_null() {
|
||||
match self {
|
||||
Fill::Null => continue,
|
||||
Fill::Prev => {
|
||||
if i != 0 {
|
||||
data[i] = data[i - 1].clone()
|
||||
@@ -262,7 +266,8 @@ impl Fill {
|
||||
}
|
||||
// The calculation of linear interpolation is relatively complicated.
|
||||
// `Self::fill_linear` is used to dispose `Fill::Linear`.
|
||||
Fill::Linear => unreachable!(),
|
||||
// No calculation need in `Fill::Null`
|
||||
Fill::Linear | Fill::Null => unreachable!(),
|
||||
Fill::Const(v) => data[i] = v.clone(),
|
||||
}
|
||||
}
|
||||
@@ -359,12 +364,12 @@ fn linear_interpolation(
|
||||
|
||||
#[derive(Eq, Clone, Debug)]
|
||||
pub struct RangeFn {
|
||||
/// with format like `max(a) RANGE 300s FILL NULL`
|
||||
/// with format like `max(a) RANGE 300s [FILL NULL]`
|
||||
pub name: String,
|
||||
pub data_type: DataType,
|
||||
pub expr: Expr,
|
||||
pub range: Duration,
|
||||
pub fill: Fill,
|
||||
pub fill: Option<Fill>,
|
||||
/// If the `FIll` strategy is `Linear` and the output is an integer,
|
||||
/// it is possible to calculate a floating point number.
|
||||
/// So for `FILL==LINEAR`, the entire data will be implicitly converted to Float type
|
||||
@@ -465,7 +470,7 @@ impl RangeSelect {
|
||||
name,
|
||||
data_type.clone(),
|
||||
// Only when data fill with Const option, the data can't be null
|
||||
!matches!(fill, Fill::Const(..)),
|
||||
!matches!(fill, Some(Fill::Const(..))),
|
||||
))
|
||||
},
|
||||
)
|
||||
@@ -810,10 +815,26 @@ struct RangeFnExec {
|
||||
pub expr: Arc<dyn AggregateExpr>,
|
||||
pub args: Vec<Arc<dyn PhysicalExpr>>,
|
||||
pub range: Millisecond,
|
||||
pub fill: Fill,
|
||||
pub fill: Option<Fill>,
|
||||
pub need_cast: Option<DataType>,
|
||||
}
|
||||
|
||||
impl Display for RangeFnExec {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
if let Some(fill) = &self.fill {
|
||||
write!(
|
||||
f,
|
||||
"{} RANGE {}s FILL {}",
|
||||
self.expr.name(),
|
||||
self.range / 1000,
|
||||
fill
|
||||
)
|
||||
} else {
|
||||
write!(f, "{} RANGE {}s", self.expr.name(), self.range / 1000)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct RangeSelectExec {
|
||||
input: Arc<dyn ExecutionPlan>,
|
||||
@@ -834,18 +855,8 @@ impl DisplayAs for RangeSelectExec {
|
||||
match t {
|
||||
DisplayFormatType::Default | DisplayFormatType::Verbose => {
|
||||
write!(f, "RangeSelectExec: ")?;
|
||||
let range_expr_strs: Vec<String> = self
|
||||
.range_exec
|
||||
.iter()
|
||||
.map(|e| {
|
||||
format!(
|
||||
"{} RANGE {}s FILL {}",
|
||||
e.expr.name(),
|
||||
e.range / 1000,
|
||||
e.fill
|
||||
)
|
||||
})
|
||||
.collect();
|
||||
let range_expr_strs: Vec<String> =
|
||||
self.range_exec.iter().map(RangeFnExec::to_string).collect();
|
||||
let by: Vec<String> = self.by.iter().map(|e| e.to_string()).collect();
|
||||
write!(
|
||||
f,
|
||||
@@ -939,7 +950,7 @@ impl ExecutionPlan for RangeSelectExec {
|
||||
by: self.by.clone(),
|
||||
series_map: HashMap::new(),
|
||||
exec_state: ExecutionState::ReadingInput,
|
||||
output_num_rows: 0,
|
||||
num_not_null_rows: 0,
|
||||
row_converter,
|
||||
modify_map: HashMap::new(),
|
||||
metric: baseline_metric,
|
||||
@@ -979,8 +990,8 @@ struct RangeSelectStream {
|
||||
/// value: `[row_ids]`
|
||||
/// It is used to record the data that needs to be aggregated in each time slot during the data update process
|
||||
modify_map: HashMap<(u64, Millisecond), Vec<u32>>,
|
||||
/// The number of rows of the final output
|
||||
output_num_rows: usize,
|
||||
/// The number of rows of not null rows in the final output
|
||||
num_not_null_rows: usize,
|
||||
metric: BaselineMetrics,
|
||||
schema_project: Option<Vec<usize>>,
|
||||
schema_before_project: SchemaRef,
|
||||
@@ -992,7 +1003,7 @@ struct SeriesState {
|
||||
row: OwnedRow,
|
||||
/// key: align_ts
|
||||
/// value: a vector, each element is a range_fn follow the order of `range_exec`
|
||||
align_ts_accumulator: HashMap<Millisecond, Vec<Box<dyn Accumulator>>>,
|
||||
align_ts_accumulator: BTreeMap<Millisecond, Vec<Box<dyn Accumulator>>>,
|
||||
}
|
||||
|
||||
/// Use `align_to` as time origin.
|
||||
@@ -1108,7 +1119,7 @@ impl RangeSelectStream {
|
||||
let accumulators_map =
|
||||
self.series_map.entry(*hash).or_insert_with(|| SeriesState {
|
||||
row: by_rows.row(*row as usize).owned(),
|
||||
align_ts_accumulator: HashMap::new(),
|
||||
align_ts_accumulator: BTreeMap::new(),
|
||||
});
|
||||
match accumulators_map.align_ts_accumulator.entry(*ts) {
|
||||
Entry::Occupied(mut e) => {
|
||||
@@ -1116,7 +1127,7 @@ impl RangeSelectStream {
|
||||
accumulators[i].update_batch(&sliced_arrays)
|
||||
}
|
||||
Entry::Vacant(e) => {
|
||||
self.output_num_rows += 1;
|
||||
self.num_not_null_rows += 1;
|
||||
let mut accumulators = self
|
||||
.range_exec
|
||||
.iter()
|
||||
@@ -1141,29 +1152,47 @@ impl RangeSelectStream {
|
||||
// 1 for time index column
|
||||
let mut columns: Vec<Arc<dyn Array>> =
|
||||
Vec::with_capacity(1 + self.range_exec.len() + self.by.len());
|
||||
let mut ts_builder = TimestampMillisecondBuilder::with_capacity(self.output_num_rows);
|
||||
let mut all_scalar = vec![Vec::with_capacity(self.output_num_rows); self.range_exec.len()];
|
||||
let mut by_rows = Vec::with_capacity(self.output_num_rows);
|
||||
let mut ts_builder = TimestampMillisecondBuilder::with_capacity(self.num_not_null_rows);
|
||||
let mut all_scalar =
|
||||
vec![Vec::with_capacity(self.num_not_null_rows); self.range_exec.len()];
|
||||
let mut by_rows = Vec::with_capacity(self.num_not_null_rows);
|
||||
let mut start_index = 0;
|
||||
// RangePlan is calculated on a row basis. If a column uses the PREV or LINEAR filling strategy,
|
||||
// we must arrange the data in the entire data row to determine the NULL filling value.
|
||||
let need_sort_output = self
|
||||
// If any range expr need fill, we need fill both the missing align_ts and null value.
|
||||
let need_fill_output = self.range_exec.iter().any(|range| range.fill.is_some());
|
||||
// The padding value for each accumulator
|
||||
let padding_values = self
|
||||
.range_exec
|
||||
.iter()
|
||||
.any(|range| range.fill == Fill::Linear || range.fill == Fill::Prev);
|
||||
.map(|e| e.expr.create_accumulator()?.evaluate())
|
||||
.collect::<DfResult<Vec<_>>>()?;
|
||||
for SeriesState {
|
||||
row,
|
||||
align_ts_accumulator,
|
||||
} in self.series_map.values()
|
||||
{
|
||||
// collect data on time series
|
||||
let mut align_ts = align_ts_accumulator.keys().copied().collect::<Vec<_>>();
|
||||
if need_sort_output {
|
||||
align_ts.sort();
|
||||
// skip empty time series
|
||||
if align_ts_accumulator.is_empty() {
|
||||
continue;
|
||||
}
|
||||
// find the first and last align_ts
|
||||
let begin_ts = *align_ts_accumulator.first_key_value().unwrap().0;
|
||||
let end_ts = *align_ts_accumulator.last_key_value().unwrap().0;
|
||||
let align_ts = if need_fill_output {
|
||||
// we need to fill empty align_ts which not data in that solt
|
||||
(begin_ts..=end_ts).step_by(self.align as usize).collect()
|
||||
} else {
|
||||
align_ts_accumulator.keys().copied().collect::<Vec<_>>()
|
||||
};
|
||||
for ts in &align_ts {
|
||||
for (i, accumulator) in align_ts_accumulator.get(ts).unwrap().iter().enumerate() {
|
||||
all_scalar[i].push(accumulator.evaluate()?);
|
||||
if let Some(slot) = align_ts_accumulator.get(ts) {
|
||||
for (column, acc) in all_scalar.iter_mut().zip(slot.iter()) {
|
||||
column.push(acc.evaluate()?);
|
||||
}
|
||||
} else {
|
||||
// fill null in empty time solt
|
||||
for (column, padding) in all_scalar.iter_mut().zip(padding_values.iter()) {
|
||||
column.push(padding.clone())
|
||||
}
|
||||
}
|
||||
}
|
||||
ts_builder.append_slice(&align_ts);
|
||||
@@ -1176,14 +1205,16 @@ impl RangeSelectStream {
|
||||
) in self.range_exec.iter().enumerate()
|
||||
{
|
||||
let time_series_data =
|
||||
&mut all_scalar[i][start_index..start_index + align_ts_accumulator.len()];
|
||||
&mut all_scalar[i][start_index..start_index + align_ts.len()];
|
||||
if let Some(data_type) = need_cast {
|
||||
cast_scalar_values(time_series_data, data_type)?;
|
||||
}
|
||||
fill.apply_fill_strategy(&align_ts, time_series_data)?;
|
||||
if let Some(fill) = fill {
|
||||
fill.apply_fill_strategy(&align_ts, time_series_data)?;
|
||||
}
|
||||
}
|
||||
by_rows.resize(by_rows.len() + align_ts_accumulator.len(), row.row());
|
||||
start_index += align_ts_accumulator.len();
|
||||
by_rows.resize(by_rows.len() + align_ts.len(), row.row());
|
||||
start_index += align_ts.len();
|
||||
}
|
||||
for column_scalar in all_scalar {
|
||||
columns.push(ScalarValue::iter_to_array(column_scalar)?);
|
||||
@@ -1304,7 +1335,7 @@ mod test {
|
||||
|
||||
const TIME_INDEX_COLUMN: &str = "timestamp";
|
||||
|
||||
fn prepare_test_data(is_float: bool) -> MemoryExec {
|
||||
fn prepare_test_data(is_float: bool, is_gap: bool) -> MemoryExec {
|
||||
let schema = Arc::new(Schema::new(vec![
|
||||
Field::new(TIME_INDEX_COLUMN, TimestampMillisecondType::DATA_TYPE, true),
|
||||
Field::new(
|
||||
@@ -1318,16 +1349,23 @@ mod test {
|
||||
),
|
||||
Field::new("host", DataType::Utf8, true),
|
||||
]));
|
||||
let timestamp_column: Arc<dyn Array> = Arc::new(TimestampMillisecondArray::from(vec![
|
||||
0, 5_000, 10_000, 15_000, 20_000, // host 1 every 5s
|
||||
0, 5_000, 10_000, 15_000, 20_000, // host 2 every 5s
|
||||
])) as _;
|
||||
let mut host = vec!["host1"; 5];
|
||||
host.extend(vec!["host2"; 5]);
|
||||
let value_column: Arc<dyn Array> = if is_float {
|
||||
Arc::new(nullable_array!(Float64;
|
||||
0.0, null, 1.0, null, 2.0, // data for host 1
|
||||
3.0, null, 4.0, null, 5.0 // data for host 2
|
||||
let timestamp_column: Arc<dyn Array> = if !is_gap {
|
||||
Arc::new(TimestampMillisecondArray::from(vec![
|
||||
0, 5_000, 10_000, 15_000, 20_000, // host 1 every 5s
|
||||
0, 5_000, 10_000, 15_000, 20_000, // host 2 every 5s
|
||||
])) as _
|
||||
} else {
|
||||
Arc::new(TimestampMillisecondArray::from(vec![
|
||||
0, 15_000, // host 1 every 5s, missing data on 5_000, 10_000
|
||||
0, 15_000, // host 2 every 5s, missing data on 5_000, 10_000
|
||||
])) as _
|
||||
};
|
||||
let mut host = vec!["host1"; timestamp_column.len() / 2];
|
||||
host.extend(vec!["host2"; timestamp_column.len() / 2]);
|
||||
let mut value_column: Arc<dyn Array> = if is_gap {
|
||||
Arc::new(nullable_array!(Int64;
|
||||
0, 6, // data for host 1
|
||||
6, 12 // data for host 2
|
||||
)) as _
|
||||
} else {
|
||||
Arc::new(nullable_array!(Int64;
|
||||
@@ -1335,6 +1373,11 @@ mod test {
|
||||
3, null, 4, null, 5 // data for host 2
|
||||
)) as _
|
||||
};
|
||||
if is_float {
|
||||
value_column =
|
||||
cast_with_options(&value_column, &DataType::Float64, &CastOptions::default())
|
||||
.unwrap();
|
||||
}
|
||||
let host_column: Arc<dyn Array> = Arc::new(StringArray::from(host)) as _;
|
||||
let data = RecordBatch::try_new(
|
||||
schema.clone(),
|
||||
@@ -1349,8 +1392,9 @@ mod test {
|
||||
range1: Millisecond,
|
||||
range2: Millisecond,
|
||||
align: Millisecond,
|
||||
fill: Fill,
|
||||
fill: Option<Fill>,
|
||||
is_float: bool,
|
||||
is_gap: bool,
|
||||
expected: String,
|
||||
) {
|
||||
let data_type = if is_float {
|
||||
@@ -1358,13 +1402,13 @@ mod test {
|
||||
} else {
|
||||
DataType::Int64
|
||||
};
|
||||
let (need_cast, schema_data_type) = if !is_float && fill == Fill::Linear {
|
||||
let (need_cast, schema_data_type) = if !is_float && matches!(fill, Some(Fill::Linear)) {
|
||||
// data_type = DataType::Float64;
|
||||
(Some(DataType::Float64), DataType::Float64)
|
||||
} else {
|
||||
(None, data_type.clone())
|
||||
};
|
||||
let memory_exec = Arc::new(prepare_test_data(is_float));
|
||||
let memory_exec = Arc::new(prepare_test_data(is_float, is_gap));
|
||||
let schema = Arc::new(Schema::new(vec![
|
||||
Field::new("MIN(value)", schema_data_type.clone(), true),
|
||||
Field::new("MAX(value)", schema_data_type, true),
|
||||
@@ -1449,7 +1493,16 @@ mod test {
|
||||
\n| 3.0 | 3.0 | 1970-01-01T00:00:00 | host2 |\
|
||||
\n+------------+------------+---------------------+-------+",
|
||||
);
|
||||
do_range_select_test(10_000, 10_000, 1_000_000, Fill::Null, true, expected).await;
|
||||
do_range_select_test(
|
||||
10_000,
|
||||
10_000,
|
||||
1_000_000,
|
||||
Some(Fill::Null),
|
||||
true,
|
||||
false,
|
||||
expected,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -1472,7 +1525,16 @@ mod test {
|
||||
\n| 5.0 | 5.0 | 1970-01-01T00:00:20 | host2 |\
|
||||
\n+------------+------------+---------------------+-------+",
|
||||
);
|
||||
do_range_select_test(10_000, 5_000, 5_000, Fill::Null, true, expected).await;
|
||||
do_range_select_test(
|
||||
10_000,
|
||||
5_000,
|
||||
5_000,
|
||||
Some(Fill::Null),
|
||||
true,
|
||||
false,
|
||||
expected,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -1495,7 +1557,16 @@ mod test {
|
||||
\n| 5.0 | 5.0 | 1970-01-01T00:00:20 | host2 |\
|
||||
\n+------------+------------+---------------------+-------+",
|
||||
);
|
||||
do_range_select_test(10_000, 5_000, 5_000, Fill::Prev, true, expected).await;
|
||||
do_range_select_test(
|
||||
10_000,
|
||||
5_000,
|
||||
5_000,
|
||||
Some(Fill::Prev),
|
||||
true,
|
||||
false,
|
||||
expected,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -1518,7 +1589,16 @@ mod test {
|
||||
\n| 5.0 | 5.0 | 1970-01-01T00:00:20 | host2 |\
|
||||
\n+------------+------------+---------------------+-------+",
|
||||
);
|
||||
do_range_select_test(10_000, 5_000, 5_000, Fill::Linear, true, expected).await;
|
||||
do_range_select_test(
|
||||
10_000,
|
||||
5_000,
|
||||
5_000,
|
||||
Some(Fill::Linear),
|
||||
true,
|
||||
false,
|
||||
expected,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -1541,7 +1621,16 @@ mod test {
|
||||
\n| 5.0 | 5.0 | 1970-01-01T00:00:20 | host2 |\
|
||||
\n+------------+------------+---------------------+-------+",
|
||||
);
|
||||
do_range_select_test(10_000, 5_000, 5_000, Fill::Linear, false, expected).await;
|
||||
do_range_select_test(
|
||||
10_000,
|
||||
5_000,
|
||||
5_000,
|
||||
Some(Fill::Linear),
|
||||
false,
|
||||
false,
|
||||
expected,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -1568,7 +1657,101 @@ mod test {
|
||||
10_000,
|
||||
5_000,
|
||||
5_000,
|
||||
Fill::Const(ScalarValue::Float64(Some(6.6))),
|
||||
Some(Fill::Const(ScalarValue::Float64(Some(6.6)))),
|
||||
true,
|
||||
false,
|
||||
expected,
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn range_fill_gap() {
|
||||
let expected = String::from(
|
||||
"+------------+------------+---------------------+-------+\
|
||||
\n| MIN(value) | MAX(value) | timestamp | host |\
|
||||
\n+------------+------------+---------------------+-------+\
|
||||
\n| 0.0 | 0.0 | 1970-01-01T00:00:00 | host1 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:15 | host1 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:00 | host2 |\
|
||||
\n| 12.0 | 12.0 | 1970-01-01T00:00:15 | host2 |\
|
||||
\n+------------+------------+---------------------+-------+",
|
||||
);
|
||||
do_range_select_test(5_000, 5_000, 5_000, None, true, true, expected).await;
|
||||
let expected = String::from(
|
||||
"+------------+------------+---------------------+-------+\
|
||||
\n| MIN(value) | MAX(value) | timestamp | host |\
|
||||
\n+------------+------------+---------------------+-------+\
|
||||
\n| 0.0 | 0.0 | 1970-01-01T00:00:00 | host1 |\
|
||||
\n| | | 1970-01-01T00:00:05 | host1 |\
|
||||
\n| | | 1970-01-01T00:00:10 | host1 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:15 | host1 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:00 | host2 |\
|
||||
\n| | | 1970-01-01T00:00:05 | host2 |\
|
||||
\n| | | 1970-01-01T00:00:10 | host2 |\
|
||||
\n| 12.0 | 12.0 | 1970-01-01T00:00:15 | host2 |\
|
||||
\n+------------+------------+---------------------+-------+",
|
||||
);
|
||||
do_range_select_test(5_000, 5_000, 5_000, Some(Fill::Null), true, true, expected).await;
|
||||
let expected = String::from(
|
||||
"+------------+------------+---------------------+-------+\
|
||||
\n| MIN(value) | MAX(value) | timestamp | host |\
|
||||
\n+------------+------------+---------------------+-------+\
|
||||
\n| 0.0 | 0.0 | 1970-01-01T00:00:00 | host1 |\
|
||||
\n| 0.0 | 0.0 | 1970-01-01T00:00:05 | host1 |\
|
||||
\n| 0.0 | 0.0 | 1970-01-01T00:00:10 | host1 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:15 | host1 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:00 | host2 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:05 | host2 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:10 | host2 |\
|
||||
\n| 12.0 | 12.0 | 1970-01-01T00:00:15 | host2 |\
|
||||
\n+------------+------------+---------------------+-------+",
|
||||
);
|
||||
do_range_select_test(5_000, 5_000, 5_000, Some(Fill::Prev), true, true, expected).await;
|
||||
let expected = String::from(
|
||||
"+------------+------------+---------------------+-------+\
|
||||
\n| MIN(value) | MAX(value) | timestamp | host |\
|
||||
\n+------------+------------+---------------------+-------+\
|
||||
\n| 0.0 | 0.0 | 1970-01-01T00:00:00 | host1 |\
|
||||
\n| 2.0 | 2.0 | 1970-01-01T00:00:05 | host1 |\
|
||||
\n| 4.0 | 4.0 | 1970-01-01T00:00:10 | host1 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:15 | host1 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:00 | host2 |\
|
||||
\n| 8.0 | 8.0 | 1970-01-01T00:00:05 | host2 |\
|
||||
\n| 10.0 | 10.0 | 1970-01-01T00:00:10 | host2 |\
|
||||
\n| 12.0 | 12.0 | 1970-01-01T00:00:15 | host2 |\
|
||||
\n+------------+------------+---------------------+-------+",
|
||||
);
|
||||
do_range_select_test(
|
||||
5_000,
|
||||
5_000,
|
||||
5_000,
|
||||
Some(Fill::Linear),
|
||||
true,
|
||||
true,
|
||||
expected,
|
||||
)
|
||||
.await;
|
||||
let expected = String::from(
|
||||
"+------------+------------+---------------------+-------+\
|
||||
\n| MIN(value) | MAX(value) | timestamp | host |\
|
||||
\n+------------+------------+---------------------+-------+\
|
||||
\n| 0.0 | 0.0 | 1970-01-01T00:00:00 | host1 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:05 | host1 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:10 | host1 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:15 | host1 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:00 | host2 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:05 | host2 |\
|
||||
\n| 6.0 | 6.0 | 1970-01-01T00:00:10 | host2 |\
|
||||
\n| 12.0 | 12.0 | 1970-01-01T00:00:15 | host2 |\
|
||||
\n+------------+------------+---------------------+-------+",
|
||||
);
|
||||
do_range_select_test(
|
||||
5_000,
|
||||
5_000,
|
||||
5_000,
|
||||
Some(Fill::Const(ScalarValue::Float64(Some(6.0)))),
|
||||
true,
|
||||
true,
|
||||
expected,
|
||||
)
|
||||
@@ -1577,7 +1760,8 @@ mod test {
|
||||
|
||||
#[test]
|
||||
fn fill_test() {
|
||||
assert!(Fill::try_from_str("Linear", &DataType::UInt8).unwrap() == Fill::Linear);
|
||||
assert!(Fill::try_from_str("", &DataType::UInt8).unwrap().is_none());
|
||||
assert!(Fill::try_from_str("Linear", &DataType::UInt8).unwrap() == Some(Fill::Linear));
|
||||
assert_eq!(
|
||||
Fill::try_from_str("Linear", &DataType::Boolean)
|
||||
.unwrap_err()
|
||||
@@ -1598,7 +1782,7 @@ mod test {
|
||||
);
|
||||
assert!(
|
||||
Fill::try_from_str("8", &DataType::UInt8).unwrap()
|
||||
== Fill::Const(ScalarValue::UInt8(Some(8)))
|
||||
== Some(Fill::Const(ScalarValue::UInt8(Some(8))))
|
||||
);
|
||||
let mut test1 = vec![
|
||||
ScalarValue::UInt8(Some(8)),
|
||||
|
||||
@@ -216,7 +216,7 @@ impl<'a> TreeNodeRewriter for RangeExprRewriter<'a> {
|
||||
let mut data_type = range_expr.get_type(self.input_plan.schema())?;
|
||||
let mut need_cast = false;
|
||||
let fill = Fill::try_from_str(parse_str_expr(&func.args, 2)?, &data_type)?;
|
||||
if matches!(fill, Fill::Linear) && data_type.is_integer() {
|
||||
if matches!(fill, Some(Fill::Linear)) && data_type.is_integer() {
|
||||
data_type = DataType::Float64;
|
||||
need_cast = true;
|
||||
}
|
||||
@@ -224,12 +224,20 @@ impl<'a> TreeNodeRewriter for RangeExprRewriter<'a> {
|
||||
inconsistent_check!(self.align, self.align != Duration::default());
|
||||
inconsistent_check!(self.align_to, self.align_to != 0);
|
||||
let range_fn = RangeFn {
|
||||
name: format!(
|
||||
"{} RANGE {} FILL {}",
|
||||
range_expr.display_name()?,
|
||||
parse_expr_to_string(&func.args, 1)?,
|
||||
fill
|
||||
),
|
||||
name: if let Some(fill) = &fill {
|
||||
format!(
|
||||
"{} RANGE {} FILL {}",
|
||||
range_expr.display_name()?,
|
||||
parse_expr_to_string(&func.args, 1)?,
|
||||
fill
|
||||
)
|
||||
} else {
|
||||
format!(
|
||||
"{} RANGE {}",
|
||||
range_expr.display_name()?,
|
||||
parse_expr_to_string(&func.args, 1)?,
|
||||
)
|
||||
},
|
||||
data_type,
|
||||
expr: range_expr,
|
||||
range,
|
||||
@@ -551,7 +559,7 @@ mod test {
|
||||
async fn range_no_project() {
|
||||
let query = r#"SELECT timestamp, tag_0, tag_1, avg(field_0 + field_1) RANGE '5m' FROM test ALIGN '1h' by (tag_0,tag_1);"#;
|
||||
let expected = String::from(
|
||||
"RangeSelect: range_exprs=[AVG(test.field_0 + test.field_1) RANGE 5m FILL NULL], align=3600000ms, align_to=0ms, align_by=[test.tag_0, test.tag_1], time_index=timestamp [timestamp:Timestamp(Millisecond, None), tag_0:Utf8, tag_1:Utf8, AVG(test.field_0 + test.field_1) RANGE 5m FILL NULL:Float64;N]\
|
||||
"RangeSelect: range_exprs=[AVG(test.field_0 + test.field_1) RANGE 5m], align=3600000ms, align_to=0ms, align_by=[test.tag_0, test.tag_1], time_index=timestamp [timestamp:Timestamp(Millisecond, None), tag_0:Utf8, tag_1:Utf8, AVG(test.field_0 + test.field_1) RANGE 5m:Float64;N]\
|
||||
\n TableScan: test [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, tag_4:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N, field_1:Float64;N, field_2:Float64;N, field_3:Float64;N, field_4:Float64;N]"
|
||||
);
|
||||
query_plan_compare(query, expected).await;
|
||||
@@ -561,8 +569,8 @@ mod test {
|
||||
async fn range_expr_calculation() {
|
||||
let query = r#"SELECT (avg(field_0 + field_1)/4) RANGE '5m' FROM test ALIGN '1h' by (tag_0,tag_1);"#;
|
||||
let expected = String::from(
|
||||
"Projection: AVG(test.field_0 + test.field_1) RANGE 5m FILL NULL / Int64(4) [AVG(test.field_0 + test.field_1) RANGE 5m FILL NULL / Int64(4):Float64;N]\
|
||||
\n RangeSelect: range_exprs=[AVG(test.field_0 + test.field_1) RANGE 5m FILL NULL], align=3600000ms, align_to=0ms, align_by=[test.tag_0, test.tag_1], time_index=timestamp [AVG(test.field_0 + test.field_1) RANGE 5m FILL NULL:Float64;N, timestamp:Timestamp(Millisecond, None), tag_0:Utf8, tag_1:Utf8]\
|
||||
"Projection: AVG(test.field_0 + test.field_1) RANGE 5m / Int64(4) [AVG(test.field_0 + test.field_1) RANGE 5m / Int64(4):Float64;N]\
|
||||
\n RangeSelect: range_exprs=[AVG(test.field_0 + test.field_1) RANGE 5m], align=3600000ms, align_to=0ms, align_by=[test.tag_0, test.tag_1], time_index=timestamp [AVG(test.field_0 + test.field_1) RANGE 5m:Float64;N, timestamp:Timestamp(Millisecond, None), tag_0:Utf8, tag_1:Utf8]\
|
||||
\n TableScan: test [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, tag_4:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N, field_1:Float64;N, field_2:Float64;N, field_3:Float64;N, field_4:Float64;N]"
|
||||
);
|
||||
query_plan_compare(query, expected).await;
|
||||
@@ -573,8 +581,8 @@ mod test {
|
||||
let query =
|
||||
r#"SELECT (covar(field_0 + field_1, field_1)/4) RANGE '5m' FROM test ALIGN '1h';"#;
|
||||
let expected = String::from(
|
||||
"Projection: COVARIANCE(test.field_0 + test.field_1,test.field_1) RANGE 5m FILL NULL / Int64(4) [COVARIANCE(test.field_0 + test.field_1,test.field_1) RANGE 5m FILL NULL / Int64(4):Float64;N]\
|
||||
\n RangeSelect: range_exprs=[COVARIANCE(test.field_0 + test.field_1,test.field_1) RANGE 5m FILL NULL], align=3600000ms, align_to=0ms, align_by=[test.tag_0, test.tag_1, test.tag_2, test.tag_3, test.tag_4], time_index=timestamp [COVARIANCE(test.field_0 + test.field_1,test.field_1) RANGE 5m FILL NULL:Float64;N, timestamp:Timestamp(Millisecond, None), tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, tag_4:Utf8]\
|
||||
"Projection: COVARIANCE(test.field_0 + test.field_1,test.field_1) RANGE 5m / Int64(4) [COVARIANCE(test.field_0 + test.field_1,test.field_1) RANGE 5m / Int64(4):Float64;N]\
|
||||
\n RangeSelect: range_exprs=[COVARIANCE(test.field_0 + test.field_1,test.field_1) RANGE 5m], align=3600000ms, align_to=0ms, align_by=[test.tag_0, test.tag_1, test.tag_2, test.tag_3, test.tag_4], time_index=timestamp [COVARIANCE(test.field_0 + test.field_1,test.field_1) RANGE 5m:Float64;N, timestamp:Timestamp(Millisecond, None), tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, tag_4:Utf8]\
|
||||
\n TableScan: test [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, tag_4:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N, field_1:Float64;N, field_2:Float64;N, field_3:Float64;N, field_4:Float64;N]"
|
||||
);
|
||||
query_plan_compare(query, expected).await;
|
||||
@@ -621,8 +629,8 @@ mod test {
|
||||
async fn range_in_expr() {
|
||||
let query = r#"SELECT sin(avg(field_0 + field_1) RANGE '5m' + 1) FROM test ALIGN '1h' by (tag_0,tag_1);"#;
|
||||
let expected = String::from(
|
||||
"Projection: sin(AVG(test.field_0 + test.field_1) RANGE 5m FILL NULL + Int64(1)) [sin(AVG(test.field_0 + test.field_1) RANGE 5m FILL NULL + Int64(1)):Float64;N]\
|
||||
\n RangeSelect: range_exprs=[AVG(test.field_0 + test.field_1) RANGE 5m FILL NULL], align=3600000ms, align_to=0ms, align_by=[test.tag_0, test.tag_1], time_index=timestamp [AVG(test.field_0 + test.field_1) RANGE 5m FILL NULL:Float64;N, timestamp:Timestamp(Millisecond, None), tag_0:Utf8, tag_1:Utf8]\
|
||||
"Projection: sin(AVG(test.field_0 + test.field_1) RANGE 5m + Int64(1)) [sin(AVG(test.field_0 + test.field_1) RANGE 5m + Int64(1)):Float64;N]\
|
||||
\n RangeSelect: range_exprs=[AVG(test.field_0 + test.field_1) RANGE 5m], align=3600000ms, align_to=0ms, align_by=[test.tag_0, test.tag_1], time_index=timestamp [AVG(test.field_0 + test.field_1) RANGE 5m:Float64;N, timestamp:Timestamp(Millisecond, None), tag_0:Utf8, tag_1:Utf8]\
|
||||
\n TableScan: test [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, tag_4:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N, field_1:Float64;N, field_2:Float64;N, field_3:Float64;N, field_4:Float64;N]"
|
||||
);
|
||||
query_plan_compare(query, expected).await;
|
||||
@@ -643,8 +651,8 @@ mod test {
|
||||
async fn deep_nest_range_expr() {
|
||||
let query = r#"SELECT round(sin(avg(field_0 + field_1) RANGE '5m' + 1)) FROM test ALIGN '1h' by (tag_0,tag_1);"#;
|
||||
let expected = String::from(
|
||||
"Projection: round(sin(AVG(test.field_0 + test.field_1) RANGE 5m FILL NULL + Int64(1))) [round(sin(AVG(test.field_0 + test.field_1) RANGE 5m FILL NULL + Int64(1))):Float64;N]\
|
||||
\n RangeSelect: range_exprs=[AVG(test.field_0 + test.field_1) RANGE 5m FILL NULL], align=3600000ms, align_to=0ms, align_by=[test.tag_0, test.tag_1], time_index=timestamp [AVG(test.field_0 + test.field_1) RANGE 5m FILL NULL:Float64;N, timestamp:Timestamp(Millisecond, None), tag_0:Utf8, tag_1:Utf8]\
|
||||
"Projection: round(sin(AVG(test.field_0 + test.field_1) RANGE 5m + Int64(1))) [round(sin(AVG(test.field_0 + test.field_1) RANGE 5m + Int64(1))):Float64;N]\
|
||||
\n RangeSelect: range_exprs=[AVG(test.field_0 + test.field_1) RANGE 5m], align=3600000ms, align_to=0ms, align_by=[test.tag_0, test.tag_1], time_index=timestamp [AVG(test.field_0 + test.field_1) RANGE 5m:Float64;N, timestamp:Timestamp(Millisecond, None), tag_0:Utf8, tag_1:Utf8]\
|
||||
\n TableScan: test [tag_0:Utf8, tag_1:Utf8, tag_2:Utf8, tag_3:Utf8, tag_4:Utf8, timestamp:Timestamp(Millisecond, None), field_0:Float64;N, field_1:Float64;N, field_2:Float64;N, field_3:Float64;N, field_4:Float64;N]"
|
||||
);
|
||||
query_plan_compare(query, expected).await;
|
||||
|
||||
@@ -23,47 +23,47 @@ Affected Rows: 10
|
||||
-- Test by calculate
|
||||
SELECT ts, length(host), max(val) RANGE '5s' FROM host ALIGN '20s' BY (length(host)) ORDER BY ts;
|
||||
|
||||
+---------------------+-----------------------------+----------------------------------+
|
||||
| ts | character_length(host.host) | MAX(host.val) RANGE 5s FILL NULL |
|
||||
+---------------------+-----------------------------+----------------------------------+
|
||||
| 1970-01-01T00:00:00 | 5 | 3 |
|
||||
| 1970-01-01T00:00:20 | 5 | 5 |
|
||||
+---------------------+-----------------------------+----------------------------------+
|
||||
+---------------------+-----------------------------+------------------------+
|
||||
| ts | character_length(host.host) | MAX(host.val) RANGE 5s |
|
||||
+---------------------+-----------------------------+------------------------+
|
||||
| 1970-01-01T00:00:00 | 5 | 3 |
|
||||
| 1970-01-01T00:00:20 | 5 | 5 |
|
||||
+---------------------+-----------------------------+------------------------+
|
||||
|
||||
SELECT ts, max(val) RANGE '5s' FROM host ALIGN '20s' BY (2) ORDER BY ts;
|
||||
|
||||
+---------------------+----------------------------------+
|
||||
| ts | MAX(host.val) RANGE 5s FILL NULL |
|
||||
+---------------------+----------------------------------+
|
||||
| 1970-01-01T00:00:00 | 3 |
|
||||
| 1970-01-01T00:00:20 | 5 |
|
||||
+---------------------+----------------------------------+
|
||||
+---------------------+------------------------+
|
||||
| ts | MAX(host.val) RANGE 5s |
|
||||
+---------------------+------------------------+
|
||||
| 1970-01-01T00:00:00 | 3 |
|
||||
| 1970-01-01T00:00:20 | 5 |
|
||||
+---------------------+------------------------+
|
||||
|
||||
-- The user explicitly specifies that the aggregation key is empty. In this case, there is no aggregation key. All data will be aggregated into a group.
|
||||
-- Implement by rewrite `BY()` to `BY(1)` automatically through sqlparser. They are semantically equivalent.
|
||||
SELECT ts, max(val) RANGE '5s' FROM host ALIGN '20s' BY () ORDER BY ts;
|
||||
|
||||
+---------------------+----------------------------------+
|
||||
| ts | MAX(host.val) RANGE 5s FILL NULL |
|
||||
+---------------------+----------------------------------+
|
||||
| 1970-01-01T00:00:00 | 3 |
|
||||
| 1970-01-01T00:00:20 | 5 |
|
||||
+---------------------+----------------------------------+
|
||||
+---------------------+------------------------+
|
||||
| ts | MAX(host.val) RANGE 5s |
|
||||
+---------------------+------------------------+
|
||||
| 1970-01-01T00:00:00 | 3 |
|
||||
| 1970-01-01T00:00:20 | 5 |
|
||||
+---------------------+------------------------+
|
||||
|
||||
SELECT ts, length(host)::INT64 + 2, max(val) RANGE '5s' FROM host ALIGN '20s' BY (length(host)::INT64 + 2) ORDER BY ts;
|
||||
|
||||
+---------------------+----------------------------------------+----------------------------------+
|
||||
| ts | character_length(host.host) + Int64(2) | MAX(host.val) RANGE 5s FILL NULL |
|
||||
+---------------------+----------------------------------------+----------------------------------+
|
||||
| 1970-01-01T00:00:00 | 7 | 3 |
|
||||
| 1970-01-01T00:00:20 | 7 | 5 |
|
||||
+---------------------+----------------------------------------+----------------------------------+
|
||||
+---------------------+----------------------------------------+------------------------+
|
||||
| ts | character_length(host.host) + Int64(2) | MAX(host.val) RANGE 5s |
|
||||
+---------------------+----------------------------------------+------------------------+
|
||||
| 1970-01-01T00:00:00 | 7 | 3 |
|
||||
| 1970-01-01T00:00:20 | 7 | 5 |
|
||||
+---------------------+----------------------------------------+------------------------+
|
||||
|
||||
-- Test error
|
||||
-- project non-aggregation key
|
||||
SELECT ts, host, max(val) RANGE '5s' FROM host ALIGN '20s' BY () ORDER BY ts;
|
||||
|
||||
Error: 3001(EngineExecuteQuery), DataFusion error: No field named host.host. Valid fields are "MAX(host.val) RANGE 5s FILL NULL", host.ts, "Int64(1)".
|
||||
Error: 3001(EngineExecuteQuery), DataFusion error: No field named host.host. Valid fields are "MAX(host.val) RANGE 5s", host.ts, "Int64(1)".
|
||||
|
||||
DROP TABLE host;
|
||||
|
||||
@@ -94,12 +94,12 @@ Affected Rows: 10
|
||||
|
||||
SELECT ts, max(val) RANGE '5s' FROM host ALIGN '20s' ORDER BY ts;
|
||||
|
||||
+---------------------+----------------------------------+
|
||||
| ts | MAX(host.val) RANGE 5s FILL NULL |
|
||||
+---------------------+----------------------------------+
|
||||
| 1970-01-01T00:00:00 | 3 |
|
||||
| 1970-01-01T00:00:20 | 5 |
|
||||
+---------------------+----------------------------------+
|
||||
+---------------------+------------------------+
|
||||
| ts | MAX(host.val) RANGE 5s |
|
||||
+---------------------+------------------------+
|
||||
| 1970-01-01T00:00:00 | 3 |
|
||||
| 1970-01-01T00:00:20 | 5 |
|
||||
+---------------------+------------------------+
|
||||
|
||||
DROP TABLE host;
|
||||
|
||||
|
||||
@@ -23,151 +23,151 @@ Affected Rows: 10
|
||||
-- Test range expr calculate
|
||||
SELECT ts, host, covar(val::DOUBLE, val::DOUBLE) RANGE '20s' FROM host ALIGN '10s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+---------------------------------------------------+
|
||||
| ts | host | COVARIANCE(host.val,host.val) RANGE 20s FILL NULL |
|
||||
+---------------------+-------+---------------------------------------------------+
|
||||
| 1969-12-31T23:59:50 | host1 | |
|
||||
| 1970-01-01T00:00:00 | host1 | 0.5 |
|
||||
| 1970-01-01T00:00:10 | host1 | 0.5 |
|
||||
| 1970-01-01T00:00:20 | host1 | |
|
||||
| 1969-12-31T23:59:50 | host2 | |
|
||||
| 1970-01-01T00:00:00 | host2 | 0.5 |
|
||||
| 1970-01-01T00:00:10 | host2 | 0.5 |
|
||||
| 1970-01-01T00:00:20 | host2 | |
|
||||
+---------------------+-------+---------------------------------------------------+
|
||||
+---------------------+-------+-----------------------------------------+
|
||||
| ts | host | COVARIANCE(host.val,host.val) RANGE 20s |
|
||||
+---------------------+-------+-----------------------------------------+
|
||||
| 1969-12-31T23:59:50 | host1 | |
|
||||
| 1970-01-01T00:00:00 | host1 | 0.5 |
|
||||
| 1970-01-01T00:00:10 | host1 | 0.5 |
|
||||
| 1970-01-01T00:00:20 | host1 | |
|
||||
| 1969-12-31T23:59:50 | host2 | |
|
||||
| 1970-01-01T00:00:00 | host2 | 0.5 |
|
||||
| 1970-01-01T00:00:10 | host2 | 0.5 |
|
||||
| 1970-01-01T00:00:20 | host2 | |
|
||||
+---------------------+-------+-----------------------------------------+
|
||||
|
||||
SELECT ts, host, 2 * min(val) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+---------------------------------------------+
|
||||
| ts | host | Int64(2) * MIN(host.val) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+---------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 2 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 4 |
|
||||
| 1970-01-01T00:00:00 | host2 | 6 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 8 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 10 |
|
||||
+---------------------+-------+---------------------------------------------+
|
||||
+---------------------+-------+-----------------------------------+
|
||||
| ts | host | Int64(2) * MIN(host.val) RANGE 5s |
|
||||
+---------------------+-------+-----------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 2 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 4 |
|
||||
| 1970-01-01T00:00:00 | host2 | 6 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 8 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 10 |
|
||||
+---------------------+-------+-----------------------------------+
|
||||
|
||||
SELECT ts, host, min(val * 2) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+---------------------------------------------+
|
||||
| ts | host | MIN(host.val * Int64(2)) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+---------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 2 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 4 |
|
||||
| 1970-01-01T00:00:00 | host2 | 6 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 8 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 10 |
|
||||
+---------------------+-------+---------------------------------------------+
|
||||
+---------------------+-------+-----------------------------------+
|
||||
| ts | host | MIN(host.val * Int64(2)) RANGE 5s |
|
||||
+---------------------+-------+-----------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 2 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 4 |
|
||||
| 1970-01-01T00:00:00 | host2 | 6 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 8 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 10 |
|
||||
+---------------------+-------+-----------------------------------+
|
||||
|
||||
SELECT ts, host, min(val::DOUBLE) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+----------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0.0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 1.0 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 2.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3.0 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 4.0 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 5.0 |
|
||||
+---------------------+-------+----------------------------------+
|
||||
+---------------------+-------+------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s |
|
||||
+---------------------+-------+------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0.0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 1.0 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 2.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3.0 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 4.0 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 5.0 |
|
||||
+---------------------+-------+------------------------+
|
||||
|
||||
SELECT ts, host, min(floor(val::DOUBLE)) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+-----------------------------------------+
|
||||
| ts | host | MIN(floor(host.val)) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+-----------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0.0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 1.0 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 2.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3.0 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 4.0 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 5.0 |
|
||||
+---------------------+-------+-----------------------------------------+
|
||||
+---------------------+-------+-------------------------------+
|
||||
| ts | host | MIN(floor(host.val)) RANGE 5s |
|
||||
+---------------------+-------+-------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0.0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 1.0 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 2.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3.0 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 4.0 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 5.0 |
|
||||
+---------------------+-------+-------------------------------+
|
||||
|
||||
SELECT ts, host, floor(min(val) RANGE '5s') FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+-----------------------------------------+
|
||||
| ts | host | floor(MIN(host.val) RANGE 5s FILL NULL) |
|
||||
+---------------------+-------+-----------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0.0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 1.0 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 2.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3.0 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 4.0 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 5.0 |
|
||||
+---------------------+-------+-----------------------------------------+
|
||||
+---------------------+-------+-------------------------------+
|
||||
| ts | host | floor(MIN(host.val) RANGE 5s) |
|
||||
+---------------------+-------+-------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0.0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 1.0 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 2.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3.0 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 4.0 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 5.0 |
|
||||
+---------------------+-------+-------------------------------+
|
||||
|
||||
-- Test complex range expr calculate
|
||||
SELECT ts, host, (min(val) + max(val)) RANGE '20s' + 1.0 FROM host ALIGN '10s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+------------------------------------------------------------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 20s FILL NULL + MAX(host.val) RANGE 20s FILL NULL + Float64(1) |
|
||||
+---------------------+-------+------------------------------------------------------------------------------------+
|
||||
| 1969-12-31T23:59:50 | host1 | 1.0 |
|
||||
| 1970-01-01T00:00:00 | host1 | 2.0 |
|
||||
| 1970-01-01T00:00:10 | host1 | 4.0 |
|
||||
| 1970-01-01T00:00:20 | host1 | 5.0 |
|
||||
| 1969-12-31T23:59:50 | host2 | 7.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 8.0 |
|
||||
| 1970-01-01T00:00:10 | host2 | 10.0 |
|
||||
| 1970-01-01T00:00:20 | host2 | 11.0 |
|
||||
+---------------------+-------+------------------------------------------------------------------------------------+
|
||||
+---------------------+-------+----------------------------------------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 20s + MAX(host.val) RANGE 20s + Float64(1) |
|
||||
+---------------------+-------+----------------------------------------------------------------+
|
||||
| 1969-12-31T23:59:50 | host1 | 1.0 |
|
||||
| 1970-01-01T00:00:00 | host1 | 2.0 |
|
||||
| 1970-01-01T00:00:10 | host1 | 4.0 |
|
||||
| 1970-01-01T00:00:20 | host1 | 5.0 |
|
||||
| 1969-12-31T23:59:50 | host2 | 7.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 8.0 |
|
||||
| 1970-01-01T00:00:10 | host2 | 10.0 |
|
||||
| 1970-01-01T00:00:20 | host2 | 11.0 |
|
||||
+---------------------+-------+----------------------------------------------------------------+
|
||||
|
||||
SELECT ts, host, covar(ceil(val::DOUBLE), floor(val::DOUBLE)) RANGE '20s' FROM host ALIGN '10s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------------------------------------+
|
||||
| ts | host | COVARIANCE(ceil(host.val),floor(host.val)) RANGE 20s FILL NULL |
|
||||
+---------------------+-------+----------------------------------------------------------------+
|
||||
| 1969-12-31T23:59:50 | host1 | |
|
||||
| 1970-01-01T00:00:00 | host1 | 0.5 |
|
||||
| 1970-01-01T00:00:10 | host1 | 0.5 |
|
||||
| 1970-01-01T00:00:20 | host1 | |
|
||||
| 1969-12-31T23:59:50 | host2 | |
|
||||
| 1970-01-01T00:00:00 | host2 | 0.5 |
|
||||
| 1970-01-01T00:00:10 | host2 | 0.5 |
|
||||
| 1970-01-01T00:00:20 | host2 | |
|
||||
+---------------------+-------+----------------------------------------------------------------+
|
||||
+---------------------+-------+------------------------------------------------------+
|
||||
| ts | host | COVARIANCE(ceil(host.val),floor(host.val)) RANGE 20s |
|
||||
+---------------------+-------+------------------------------------------------------+
|
||||
| 1969-12-31T23:59:50 | host1 | |
|
||||
| 1970-01-01T00:00:00 | host1 | 0.5 |
|
||||
| 1970-01-01T00:00:10 | host1 | 0.5 |
|
||||
| 1970-01-01T00:00:20 | host1 | |
|
||||
| 1969-12-31T23:59:50 | host2 | |
|
||||
| 1970-01-01T00:00:00 | host2 | 0.5 |
|
||||
| 1970-01-01T00:00:10 | host2 | 0.5 |
|
||||
| 1970-01-01T00:00:20 | host2 | |
|
||||
+---------------------+-------+------------------------------------------------------+
|
||||
|
||||
SELECT ts, host, floor(cos(ceil(sin(min(val) RANGE '5s')))) FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+---------------------------------------------------------+
|
||||
| ts | host | floor(cos(ceil(sin(MIN(host.val) RANGE 5s FILL NULL)))) |
|
||||
+---------------------+-------+---------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 1.0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 0.0 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 0.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 0.0 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 1.0 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 1.0 |
|
||||
+---------------------+-------+---------------------------------------------------------+
|
||||
+---------------------+-------+-----------------------------------------------+
|
||||
| ts | host | floor(cos(ceil(sin(MIN(host.val) RANGE 5s)))) |
|
||||
+---------------------+-------+-----------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 1.0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 0.0 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 0.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 0.0 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 1.0 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 1.0 |
|
||||
+---------------------+-------+-----------------------------------------------+
|
||||
|
||||
SELECT ts, host, gcd(CAST(max(floor(val::DOUBLE)) RANGE '10s' FILL PREV as INT64) * 4, max(val * 4) RANGE '10s' FILL PREV) * length(host) + 1 FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
|
||||
@@ -41,7 +41,7 @@ Error: 2000(InvalidSyntax), sql parser error: Illegal Range select, no RANGE key
|
||||
|
||||
SELECT min(val) RANGE '10s', max(val) FROM host ALIGN '5s';
|
||||
|
||||
Error: 3001(EngineExecuteQuery), DataFusion error: No field named "MAX(host.val)". Valid fields are "MIN(host.val) RANGE 10s FILL NULL", host.ts, host.host.
|
||||
Error: 3001(EngineExecuteQuery), DataFusion error: No field named "MAX(host.val)". Valid fields are "MIN(host.val) RANGE 10s", host.ts, host.host.
|
||||
|
||||
SELECT min(val) * 2 RANGE '10s' FROM host ALIGN '5s';
|
||||
|
||||
@@ -73,10 +73,6 @@ SELECT rank() OVER (PARTITION BY host ORDER BY ts DESC) RANGE '10s' FROM host AL
|
||||
Error: 2000(InvalidSyntax), Range Query: Window functions is not allowed in Range Query
|
||||
|
||||
-- 2.6 invalid fill
|
||||
SELECT min(val) RANGE '5s', min(val) RANGE '5s' FILL NULL FROM host ALIGN '5s';
|
||||
|
||||
Error: 3001(EngineExecuteQuery), DataFusion error: Schema contains duplicate unqualified field name "MIN(host.val) RANGE 5s FILL NULL"
|
||||
|
||||
SELECT min(val) RANGE '5s' FROM host ALIGN '5s' FILL 3.0;
|
||||
|
||||
Error: 3000(PlanQuery), DataFusion error: Error during planning: 3.0 is not a valid fill option, fail to convert to a const value. { Arrow error: Cast error: Cannot cast string '3.0' to value of Int64 type }
|
||||
@@ -96,7 +92,7 @@ Error: 2000(InvalidSyntax), Range Query: Can't use 0 as align in Range Query
|
||||
|
||||
SELECT min(val) RANGE (INTERVAL '0' day) FROM host ALIGN '5s';
|
||||
|
||||
Error: 2000(InvalidSyntax), Range Query: Invalid Range expr `MIN(host.val) RANGE IntervalMonthDayNano("0") FILL NULL`, Can't use 0 as range in Range Query
|
||||
Error: 2000(InvalidSyntax), Range Query: Invalid Range expr `MIN(host.val) RANGE IntervalMonthDayNano("0")`, Can't use 0 as range in Range Query
|
||||
|
||||
DROP TABLE host;
|
||||
|
||||
|
||||
@@ -54,8 +54,6 @@ SELECT rank() OVER (PARTITION BY host ORDER BY ts DESC) RANGE '10s' FROM host AL
|
||||
|
||||
-- 2.6 invalid fill
|
||||
|
||||
SELECT min(val) RANGE '5s', min(val) RANGE '5s' FILL NULL FROM host ALIGN '5s';
|
||||
|
||||
SELECT min(val) RANGE '5s' FROM host ALIGN '5s' FILL 3.0;
|
||||
|
||||
-- 2.7 zero align/range
|
||||
|
||||
@@ -20,23 +20,23 @@ INSERT INTO TABLE host VALUES
|
||||
|
||||
Affected Rows: 10
|
||||
|
||||
-- Test Fill
|
||||
-- Test Fill when aggregate result is null
|
||||
SELECT ts, host, min(val) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+----------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 1 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 4 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 5 |
|
||||
+---------------------+-------+----------------------------------+
|
||||
+---------------------+-------+------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s |
|
||||
+---------------------+-------+------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 1 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 4 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 5 |
|
||||
+---------------------+-------+------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s' FROM host ALIGN '5s' FILL NULL ORDER BY host, ts;
|
||||
|
||||
@@ -57,54 +57,231 @@ SELECT ts, host, min(val) RANGE '5s' FROM host ALIGN '5s' FILL NULL ORDER BY hos
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s', min(val) RANGE '5s' FILL 6 FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------+-------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s FILL NULL | MIN(host.val) RANGE 5s FILL 6 |
|
||||
+---------------------+-------+----------------------------------+-------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | | 6 |
|
||||
| 1970-01-01T00:00:10 | host1 | 1 | 1 |
|
||||
| 1970-01-01T00:00:15 | host1 | | 6 |
|
||||
| 1970-01-01T00:00:20 | host1 | 2 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 | 3 |
|
||||
| 1970-01-01T00:00:05 | host2 | | 6 |
|
||||
| 1970-01-01T00:00:10 | host2 | 4 | 4 |
|
||||
| 1970-01-01T00:00:15 | host2 | | 6 |
|
||||
| 1970-01-01T00:00:20 | host2 | 5 | 5 |
|
||||
+---------------------+-------+----------------------------------+-------------------------------+
|
||||
+---------------------+-------+------------------------+-------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s | MIN(host.val) RANGE 5s FILL 6 |
|
||||
+---------------------+-------+------------------------+-------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | | 6 |
|
||||
| 1970-01-01T00:00:10 | host1 | 1 | 1 |
|
||||
| 1970-01-01T00:00:15 | host1 | | 6 |
|
||||
| 1970-01-01T00:00:20 | host1 | 2 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 | 3 |
|
||||
| 1970-01-01T00:00:05 | host2 | | 6 |
|
||||
| 1970-01-01T00:00:10 | host2 | 4 | 4 |
|
||||
| 1970-01-01T00:00:15 | host2 | | 6 |
|
||||
| 1970-01-01T00:00:20 | host2 | 5 | 5 |
|
||||
+---------------------+-------+------------------------+-------------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s', min(val) RANGE '5s' FILL PREV FROM host ALIGN '5s'ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s FILL NULL | MIN(host.val) RANGE 5s FILL PREV |
|
||||
+---------------------+-------+----------------------------------+----------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | | 0 |
|
||||
| 1970-01-01T00:00:10 | host1 | 1 | 1 |
|
||||
| 1970-01-01T00:00:15 | host1 | | 1 |
|
||||
| 1970-01-01T00:00:20 | host1 | 2 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 | 3 |
|
||||
| 1970-01-01T00:00:05 | host2 | | 3 |
|
||||
| 1970-01-01T00:00:10 | host2 | 4 | 4 |
|
||||
| 1970-01-01T00:00:15 | host2 | | 4 |
|
||||
| 1970-01-01T00:00:20 | host2 | 5 | 5 |
|
||||
+---------------------+-------+----------------------------------+----------------------------------+
|
||||
+---------------------+-------+------------------------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s | MIN(host.val) RANGE 5s FILL PREV |
|
||||
+---------------------+-------+------------------------+----------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | | 0 |
|
||||
| 1970-01-01T00:00:10 | host1 | 1 | 1 |
|
||||
| 1970-01-01T00:00:15 | host1 | | 1 |
|
||||
| 1970-01-01T00:00:20 | host1 | 2 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 | 3 |
|
||||
| 1970-01-01T00:00:05 | host2 | | 3 |
|
||||
| 1970-01-01T00:00:10 | host2 | 4 | 4 |
|
||||
| 1970-01-01T00:00:15 | host2 | | 4 |
|
||||
| 1970-01-01T00:00:20 | host2 | 5 | 5 |
|
||||
+---------------------+-------+------------------------+----------------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s', min(val) RANGE '5s' FILL LINEAR FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------+------------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s FILL NULL | MIN(host.val) RANGE 5s FILL LINEAR |
|
||||
+---------------------+-------+----------------------------------+------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 | 0.0 |
|
||||
| 1970-01-01T00:00:05 | host1 | | 0.5 |
|
||||
| 1970-01-01T00:00:10 | host1 | 1 | 1.0 |
|
||||
| 1970-01-01T00:00:15 | host1 | | 1.5 |
|
||||
| 1970-01-01T00:00:20 | host1 | 2 | 2.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 | 3.0 |
|
||||
| 1970-01-01T00:00:05 | host2 | | 3.5 |
|
||||
| 1970-01-01T00:00:10 | host2 | 4 | 4.0 |
|
||||
| 1970-01-01T00:00:15 | host2 | | 4.5 |
|
||||
| 1970-01-01T00:00:20 | host2 | 5 | 5.0 |
|
||||
+---------------------+-------+----------------------------------+------------------------------------+
|
||||
+---------------------+-------+------------------------+------------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s | MIN(host.val) RANGE 5s FILL LINEAR |
|
||||
+---------------------+-------+------------------------+------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 | 0.0 |
|
||||
| 1970-01-01T00:00:05 | host1 | | 0.5 |
|
||||
| 1970-01-01T00:00:10 | host1 | 1 | 1.0 |
|
||||
| 1970-01-01T00:00:15 | host1 | | 1.5 |
|
||||
| 1970-01-01T00:00:20 | host1 | 2 | 2.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 | 3.0 |
|
||||
| 1970-01-01T00:00:05 | host2 | | 3.5 |
|
||||
| 1970-01-01T00:00:10 | host2 | 4 | 4.0 |
|
||||
| 1970-01-01T00:00:15 | host2 | | 4.5 |
|
||||
| 1970-01-01T00:00:20 | host2 | 5 | 5.0 |
|
||||
+---------------------+-------+------------------------+------------------------------------+
|
||||
|
||||
DROP TABLE host;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
-- Test Fill when time slot data is missing
|
||||
CREATE TABLE host (
|
||||
ts timestamp(3) time index,
|
||||
host STRING PRIMARY KEY,
|
||||
val BIGINT,
|
||||
);
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
INSERT INTO TABLE host VALUES
|
||||
(0, 'host1', 0),
|
||||
(1000, 'host1', 1),
|
||||
(2000, 'host1', 2),
|
||||
-- missing data for 5000, 10000
|
||||
(15000, 'host1', 6),
|
||||
(16000, 'host1', 7),
|
||||
(17000, 'host1', 8),
|
||||
(0, 'host2', 6),
|
||||
(1000, 'host2', 7),
|
||||
(2000, 'host2', 8),
|
||||
-- missing data for 5000, 10000
|
||||
(15000, 'host2', 12),
|
||||
(16000, 'host2', 13),
|
||||
(17000, 'host2', 14);
|
||||
|
||||
Affected Rows: 12
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s |
|
||||
+---------------------+-------+------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-01-01T00:00:15 | host1 | 6 |
|
||||
| 1970-01-01T00:00:00 | host2 | 6 |
|
||||
| 1970-01-01T00:00:15 | host2 | 12 |
|
||||
+---------------------+-------+------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s' FILL NULL FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+----------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | |
|
||||
| 1970-01-01T00:00:15 | host1 | 6 |
|
||||
| 1970-01-01T00:00:00 | host2 | 6 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | |
|
||||
| 1970-01-01T00:00:15 | host2 | 12 |
|
||||
+---------------------+-------+----------------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s' FILL PREV FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s FILL PREV |
|
||||
+---------------------+-------+----------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | 0 |
|
||||
| 1970-01-01T00:00:10 | host1 | 0 |
|
||||
| 1970-01-01T00:00:15 | host1 | 6 |
|
||||
| 1970-01-01T00:00:00 | host2 | 6 |
|
||||
| 1970-01-01T00:00:05 | host2 | 6 |
|
||||
| 1970-01-01T00:00:10 | host2 | 6 |
|
||||
| 1970-01-01T00:00:15 | host2 | 12 |
|
||||
+---------------------+-------+----------------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s' FILL LINEAR FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+------------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s FILL LINEAR |
|
||||
+---------------------+-------+------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0.0 |
|
||||
| 1970-01-01T00:00:05 | host1 | 2.0 |
|
||||
| 1970-01-01T00:00:10 | host1 | 4.0 |
|
||||
| 1970-01-01T00:00:15 | host1 | 6.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 6.0 |
|
||||
| 1970-01-01T00:00:05 | host2 | 8.0 |
|
||||
| 1970-01-01T00:00:10 | host2 | 10.0 |
|
||||
| 1970-01-01T00:00:15 | host2 | 12.0 |
|
||||
+---------------------+-------+------------------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s' FILL 6 FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+-------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s FILL 6 |
|
||||
+---------------------+-------+-------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | 6 |
|
||||
| 1970-01-01T00:00:10 | host1 | 6 |
|
||||
| 1970-01-01T00:00:15 | host1 | 6 |
|
||||
| 1970-01-01T00:00:00 | host2 | 6 |
|
||||
| 1970-01-01T00:00:05 | host2 | 6 |
|
||||
| 1970-01-01T00:00:10 | host2 | 6 |
|
||||
| 1970-01-01T00:00:15 | host2 | 12 |
|
||||
+---------------------+-------+-------------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s' FROM host ALIGN '5s' FILL NULL ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+----------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | |
|
||||
| 1970-01-01T00:00:15 | host1 | 6 |
|
||||
| 1970-01-01T00:00:00 | host2 | 6 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | |
|
||||
| 1970-01-01T00:00:15 | host2 | 12 |
|
||||
+---------------------+-------+----------------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s', min(val) RANGE '5s' FILL NULL FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+------------------------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s | MIN(host.val) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+------------------------+----------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | | |
|
||||
| 1970-01-01T00:00:10 | host1 | | |
|
||||
| 1970-01-01T00:00:15 | host1 | 6 | 6 |
|
||||
| 1970-01-01T00:00:00 | host2 | 6 | 6 |
|
||||
| 1970-01-01T00:00:05 | host2 | | |
|
||||
| 1970-01-01T00:00:10 | host2 | | |
|
||||
| 1970-01-01T00:00:15 | host2 | 12 | 12 |
|
||||
+---------------------+-------+------------------------+----------------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s', min(val) RANGE '5s' FILL 6 FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+------------------------+-------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s | MIN(host.val) RANGE 5s FILL 6 |
|
||||
+---------------------+-------+------------------------+-------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | | 6 |
|
||||
| 1970-01-01T00:00:10 | host1 | | 6 |
|
||||
| 1970-01-01T00:00:15 | host1 | 6 | 6 |
|
||||
| 1970-01-01T00:00:00 | host2 | 6 | 6 |
|
||||
| 1970-01-01T00:00:05 | host2 | | 6 |
|
||||
| 1970-01-01T00:00:10 | host2 | | 6 |
|
||||
| 1970-01-01T00:00:15 | host2 | 12 | 12 |
|
||||
+---------------------+-------+------------------------+-------------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s', min(val) RANGE '5s' FILL PREV FROM host ALIGN '5s'ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+------------------------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s | MIN(host.val) RANGE 5s FILL PREV |
|
||||
+---------------------+-------+------------------------+----------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | | 0 |
|
||||
| 1970-01-01T00:00:10 | host1 | | 0 |
|
||||
| 1970-01-01T00:00:15 | host1 | 6 | 6 |
|
||||
| 1970-01-01T00:00:00 | host2 | 6 | 6 |
|
||||
| 1970-01-01T00:00:05 | host2 | | 6 |
|
||||
| 1970-01-01T00:00:10 | host2 | | 6 |
|
||||
| 1970-01-01T00:00:15 | host2 | 12 | 12 |
|
||||
+---------------------+-------+------------------------+----------------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s', min(val) RANGE '5s' FILL LINEAR FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+------------------------+------------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 5s | MIN(host.val) RANGE 5s FILL LINEAR |
|
||||
+---------------------+-------+------------------------+------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 | 0.0 |
|
||||
| 1970-01-01T00:00:05 | host1 | | 2.0 |
|
||||
| 1970-01-01T00:00:10 | host1 | | 4.0 |
|
||||
| 1970-01-01T00:00:15 | host1 | 6 | 6.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 6 | 6.0 |
|
||||
| 1970-01-01T00:00:05 | host2 | | 8.0 |
|
||||
| 1970-01-01T00:00:10 | host2 | | 10.0 |
|
||||
| 1970-01-01T00:00:15 | host2 | 12 | 12.0 |
|
||||
+---------------------+-------+------------------------+------------------------------------+
|
||||
|
||||
DROP TABLE host;
|
||||
|
||||
|
||||
@@ -16,7 +16,7 @@ INSERT INTO TABLE host VALUES
|
||||
(15000, 'host2', null),
|
||||
(20000, 'host2', 5);
|
||||
|
||||
-- Test Fill
|
||||
-- Test Fill when aggregate result is null
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
@@ -29,3 +29,50 @@ SELECT ts, host, min(val) RANGE '5s', min(val) RANGE '5s' FILL PREV FROM host AL
|
||||
SELECT ts, host, min(val) RANGE '5s', min(val) RANGE '5s' FILL LINEAR FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
DROP TABLE host;
|
||||
|
||||
-- Test Fill when time slot data is missing
|
||||
|
||||
CREATE TABLE host (
|
||||
ts timestamp(3) time index,
|
||||
host STRING PRIMARY KEY,
|
||||
val BIGINT,
|
||||
);
|
||||
|
||||
INSERT INTO TABLE host VALUES
|
||||
(0, 'host1', 0),
|
||||
(1000, 'host1', 1),
|
||||
(2000, 'host1', 2),
|
||||
-- missing data for 5000, 10000
|
||||
(15000, 'host1', 6),
|
||||
(16000, 'host1', 7),
|
||||
(17000, 'host1', 8),
|
||||
|
||||
(0, 'host2', 6),
|
||||
(1000, 'host2', 7),
|
||||
(2000, 'host2', 8),
|
||||
-- missing data for 5000, 10000
|
||||
(15000, 'host2', 12),
|
||||
(16000, 'host2', 13),
|
||||
(17000, 'host2', 14);
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s' FILL NULL FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s' FILL PREV FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s' FILL LINEAR FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s' FILL 6 FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s' FROM host ALIGN '5s' FILL NULL ORDER BY host, ts;
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s', min(val) RANGE '5s' FILL NULL FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s', min(val) RANGE '5s' FILL 6 FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s', min(val) RANGE '5s' FILL PREV FROM host ALIGN '5s'ORDER BY host, ts;
|
||||
|
||||
SELECT ts, host, min(val) RANGE '5s', min(val) RANGE '5s' FILL LINEAR FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
DROP TABLE host;
|
||||
|
||||
@@ -20,25 +20,25 @@ Affected Rows: 8
|
||||
|
||||
SELECT ts, host, min(val) RANGE (INTERVAL '1 year') FROM host ALIGN (INTERVAL '1 year') ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+--------------------------------------------------------------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE IntervalMonthDayNano("950737950171172051122527404032") FILL NULL |
|
||||
+---------------------+-------+--------------------------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-12-27T00:00:00 | host1 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 4 |
|
||||
| 1970-12-27T00:00:00 | host2 | 6 |
|
||||
+---------------------+-------+--------------------------------------------------------------------------------------+
|
||||
+---------------------+-------+----------------------------------------------------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE IntervalMonthDayNano("950737950171172051122527404032") |
|
||||
+---------------------+-------+----------------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-12-27T00:00:00 | host1 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 4 |
|
||||
| 1970-12-27T00:00:00 | host2 | 6 |
|
||||
+---------------------+-------+----------------------------------------------------------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE (INTERVAL '1' year) FROM host ALIGN (INTERVAL '1' year) ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+--------------------------------------------------------------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE IntervalMonthDayNano("950737950171172051122527404032") FILL NULL |
|
||||
+---------------------+-------+--------------------------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-12-27T00:00:00 | host1 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 4 |
|
||||
| 1970-12-27T00:00:00 | host2 | 6 |
|
||||
+---------------------+-------+--------------------------------------------------------------------------------------+
|
||||
+---------------------+-------+----------------------------------------------------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE IntervalMonthDayNano("950737950171172051122527404032") |
|
||||
+---------------------+-------+----------------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-12-27T00:00:00 | host1 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 4 |
|
||||
| 1970-12-27T00:00:00 | host2 | 6 |
|
||||
+---------------------+-------+----------------------------------------------------------------------------+
|
||||
|
||||
DROP TABLE host;
|
||||
|
||||
|
||||
@@ -35,15 +35,15 @@ SELECT ts, host, foo FROM (SELECT ts, host, min(val) RANGE '5s' AS foo FROM host
|
||||
|
||||
SELECT ts, b, min(c) RANGE '5s' FROM (SELECT ts, host AS b, val AS c FROM host WHERE host = 'host1') ALIGN '5s' BY (b) ORDER BY b, ts;
|
||||
|
||||
+---------------------+-------+---------------------------+
|
||||
| ts | b | MIN(c) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+---------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 1 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 2 |
|
||||
+---------------------+-------+---------------------------+
|
||||
+---------------------+-------+-----------------+
|
||||
| ts | b | MIN(c) RANGE 5s |
|
||||
+---------------------+-------+-----------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 1 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 2 |
|
||||
+---------------------+-------+-----------------+
|
||||
|
||||
-- Test EXPLAIN and ANALYZE
|
||||
-- SQLNESS REPLACE (-+) -
|
||||
@@ -55,9 +55,9 @@ EXPLAIN SELECT ts, host, min(val) RANGE '5s' FROM host ALIGN '5s';
|
||||
+-+-+
|
||||
| plan_type_| plan_|
|
||||
+-+-+
|
||||
| logical_plan_| RangeSelect: range_exprs=[MIN(host.val) RANGE 5s FILL NULL], align=5000ms, align_to=0ms, align_by=[host.host], time_index=ts |
|
||||
| logical_plan_| RangeSelect: range_exprs=[MIN(host.val) RANGE 5s], align=5000ms, align_to=0ms, align_by=[host.host], time_index=ts |
|
||||
|_|_MergeScan [is_placeholder=false]_|
|
||||
| physical_plan | RangeSelectExec: range_expr=[MIN(host.val) RANGE 5s FILL NULL], align=5000ms, align_to=0ms, align_by=[host@1], time_index=ts |
|
||||
| physical_plan | RangeSelectExec: range_expr=[MIN(host.val) RANGE 5s], align=5000ms, align_to=0ms, align_by=[host@1], time_index=ts |
|
||||
|_|_MergeScanExec: REDACTED
|
||||
|_|_|
|
||||
+-+-+
|
||||
@@ -71,7 +71,7 @@ EXPLAIN ANALYZE SELECT ts, host, min(val) RANGE '5s' FROM host ALIGN '5s';
|
||||
+-+-+
|
||||
| plan_type_| plan_|
|
||||
+-+-+
|
||||
| Plan with Metrics | RangeSelectExec: range_expr=[MIN(host.val) RANGE 5s FILL NULL], align=5000ms, align_to=0ms, align_by=[host@1], time_index=ts, REDACTED
|
||||
| Plan with Metrics | RangeSelectExec: range_expr=[MIN(host.val) RANGE 5s], align=5000ms, align_to=0ms, align_by=[host@1], time_index=ts, REDACTED
|
||||
|_|_MergeScanExec: REDACTED
|
||||
|_|_|
|
||||
+-+-+
|
||||
|
||||
@@ -23,20 +23,20 @@ Affected Rows: 10
|
||||
-- Test on Timestamps of different precisions
|
||||
SELECT ts, host, min(val) RANGE '5s' FROM host_sec ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+--------------------------------------+
|
||||
| ts | host | MIN(host_sec.val) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+--------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0.0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 1.0 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 2.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3.0 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 4.0 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 5.0 |
|
||||
+---------------------+-------+--------------------------------------+
|
||||
+---------------------+-------+----------------------------+
|
||||
| ts | host | MIN(host_sec.val) RANGE 5s |
|
||||
+---------------------+-------+----------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0.0 |
|
||||
| 1970-01-01T00:00:05 | host1 | |
|
||||
| 1970-01-01T00:00:10 | host1 | 1.0 |
|
||||
| 1970-01-01T00:00:15 | host1 | |
|
||||
| 1970-01-01T00:00:20 | host1 | 2.0 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3.0 |
|
||||
| 1970-01-01T00:00:05 | host2 | |
|
||||
| 1970-01-01T00:00:10 | host2 | 4.0 |
|
||||
| 1970-01-01T00:00:15 | host2 | |
|
||||
| 1970-01-01T00:00:20 | host2 | 5.0 |
|
||||
+---------------------+-------+----------------------------+
|
||||
|
||||
DROP TABLE host_sec;
|
||||
|
||||
|
||||
@@ -43,173 +43,173 @@ Affected Rows: 30
|
||||
|
||||
SELECT ts, host, first_value(val) RANGE '5s', last_value(val) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+------------------------------------------+-----------------------------------------+
|
||||
| ts | host | FIRST_VALUE(host.val) RANGE 5s FILL NULL | LAST_VALUE(host.val) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+------------------------------------------+-----------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 | 2 |
|
||||
| 1970-01-01T00:00:05 | host1 | | |
|
||||
| 1970-01-01T00:00:10 | host1 | | 5 |
|
||||
| 1970-01-01T00:00:15 | host1 | 6 | 7 |
|
||||
| 1970-01-01T00:00:20 | host1 | 8 | |
|
||||
| 1970-01-01T00:00:00 | host2 | 0 | 2 |
|
||||
| 1970-01-01T00:00:05 | host2 | | |
|
||||
| 1970-01-01T00:00:10 | host2 | | 5 |
|
||||
| 1970-01-01T00:00:15 | host2 | 6 | 7 |
|
||||
| 1970-01-01T00:00:20 | host2 | 8 | |
|
||||
+---------------------+-------+------------------------------------------+-----------------------------------------+
|
||||
+---------------------+-------+--------------------------------+-------------------------------+
|
||||
| ts | host | FIRST_VALUE(host.val) RANGE 5s | LAST_VALUE(host.val) RANGE 5s |
|
||||
+---------------------+-------+--------------------------------+-------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 | 2 |
|
||||
| 1970-01-01T00:00:05 | host1 | | |
|
||||
| 1970-01-01T00:00:10 | host1 | | 5 |
|
||||
| 1970-01-01T00:00:15 | host1 | 6 | 7 |
|
||||
| 1970-01-01T00:00:20 | host1 | 8 | |
|
||||
| 1970-01-01T00:00:00 | host2 | 0 | 2 |
|
||||
| 1970-01-01T00:00:05 | host2 | | |
|
||||
| 1970-01-01T00:00:10 | host2 | | 5 |
|
||||
| 1970-01-01T00:00:15 | host2 | 6 | 7 |
|
||||
| 1970-01-01T00:00:20 | host2 | 8 | |
|
||||
+---------------------+-------+--------------------------------+-------------------------------+
|
||||
|
||||
SELECT ts, host, first_value(addon ORDER BY val DESC) RANGE '5s', last_value(addon ORDER BY val DESC) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+---------------------------------------------------------------------------------+--------------------------------------------------------------------------------+
|
||||
| ts | host | FIRST_VALUE(host.addon) ORDER BY [host.val DESC NULLS FIRST] RANGE 5s FILL NULL | LAST_VALUE(host.addon) ORDER BY [host.val DESC NULLS FIRST] RANGE 5s FILL NULL |
|
||||
+---------------------+-------+---------------------------------------------------------------------------------+--------------------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 3 | 1 |
|
||||
| 1970-01-01T00:00:05 | host1 | 4 | 4 |
|
||||
| 1970-01-01T00:00:10 | host1 | 7 | 8 |
|
||||
| 1970-01-01T00:00:15 | host1 | 11 | 10 |
|
||||
| 1970-01-01T00:00:20 | host1 | 15 | 13 |
|
||||
| 1970-01-01T00:00:00 | host2 | 18 | 16 |
|
||||
| 1970-01-01T00:00:05 | host2 | 19 | 19 |
|
||||
| 1970-01-01T00:00:10 | host2 | 22 | 23 |
|
||||
| 1970-01-01T00:00:15 | host2 | 26 | 25 |
|
||||
| 1970-01-01T00:00:20 | host2 | 30 | 28 |
|
||||
+---------------------+-------+---------------------------------------------------------------------------------+--------------------------------------------------------------------------------+
|
||||
+---------------------+-------+-----------------------------------------------------------------------+----------------------------------------------------------------------+
|
||||
| ts | host | FIRST_VALUE(host.addon) ORDER BY [host.val DESC NULLS FIRST] RANGE 5s | LAST_VALUE(host.addon) ORDER BY [host.val DESC NULLS FIRST] RANGE 5s |
|
||||
+---------------------+-------+-----------------------------------------------------------------------+----------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 3 | 1 |
|
||||
| 1970-01-01T00:00:05 | host1 | 4 | 4 |
|
||||
| 1970-01-01T00:00:10 | host1 | 7 | 8 |
|
||||
| 1970-01-01T00:00:15 | host1 | 11 | 10 |
|
||||
| 1970-01-01T00:00:20 | host1 | 15 | 13 |
|
||||
| 1970-01-01T00:00:00 | host2 | 18 | 16 |
|
||||
| 1970-01-01T00:00:05 | host2 | 19 | 19 |
|
||||
| 1970-01-01T00:00:10 | host2 | 22 | 23 |
|
||||
| 1970-01-01T00:00:15 | host2 | 26 | 25 |
|
||||
| 1970-01-01T00:00:20 | host2 | 30 | 28 |
|
||||
+---------------------+-------+-----------------------------------------------------------------------+----------------------------------------------------------------------+
|
||||
|
||||
SELECT ts, host, first_value(addon ORDER BY val DESC NULLS LAST) RANGE '5s', last_value(addon ORDER BY val DESC NULLS LAST) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+--------------------------------------------------------------------------------+-------------------------------------------------------------------------------+
|
||||
| ts | host | FIRST_VALUE(host.addon) ORDER BY [host.val DESC NULLS LAST] RANGE 5s FILL NULL | LAST_VALUE(host.addon) ORDER BY [host.val DESC NULLS LAST] RANGE 5s FILL NULL |
|
||||
+---------------------+-------+--------------------------------------------------------------------------------+-------------------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 3 | 1 |
|
||||
| 1970-01-01T00:00:05 | host1 | 4 | 4 |
|
||||
| 1970-01-01T00:00:10 | host1 | 9 | 7 |
|
||||
| 1970-01-01T00:00:15 | host1 | 12 | 11 |
|
||||
| 1970-01-01T00:00:20 | host1 | 14 | 15 |
|
||||
| 1970-01-01T00:00:00 | host2 | 18 | 16 |
|
||||
| 1970-01-01T00:00:05 | host2 | 19 | 19 |
|
||||
| 1970-01-01T00:00:10 | host2 | 24 | 22 |
|
||||
| 1970-01-01T00:00:15 | host2 | 27 | 26 |
|
||||
| 1970-01-01T00:00:20 | host2 | 29 | 30 |
|
||||
+---------------------+-------+--------------------------------------------------------------------------------+-------------------------------------------------------------------------------+
|
||||
+---------------------+-------+----------------------------------------------------------------------+---------------------------------------------------------------------+
|
||||
| ts | host | FIRST_VALUE(host.addon) ORDER BY [host.val DESC NULLS LAST] RANGE 5s | LAST_VALUE(host.addon) ORDER BY [host.val DESC NULLS LAST] RANGE 5s |
|
||||
+---------------------+-------+----------------------------------------------------------------------+---------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 3 | 1 |
|
||||
| 1970-01-01T00:00:05 | host1 | 4 | 4 |
|
||||
| 1970-01-01T00:00:10 | host1 | 9 | 7 |
|
||||
| 1970-01-01T00:00:15 | host1 | 12 | 11 |
|
||||
| 1970-01-01T00:00:20 | host1 | 14 | 15 |
|
||||
| 1970-01-01T00:00:00 | host2 | 18 | 16 |
|
||||
| 1970-01-01T00:00:05 | host2 | 19 | 19 |
|
||||
| 1970-01-01T00:00:10 | host2 | 24 | 22 |
|
||||
| 1970-01-01T00:00:15 | host2 | 27 | 26 |
|
||||
| 1970-01-01T00:00:20 | host2 | 29 | 30 |
|
||||
+---------------------+-------+----------------------------------------------------------------------+---------------------------------------------------------------------+
|
||||
|
||||
SELECT ts, host, first_value(addon ORDER BY val ASC) RANGE '5s', last_value(addon ORDER BY val ASC) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+-------------------------------------------------------------------------------+------------------------------------------------------------------------------+
|
||||
| ts | host | FIRST_VALUE(host.addon) ORDER BY [host.val ASC NULLS LAST] RANGE 5s FILL NULL | LAST_VALUE(host.addon) ORDER BY [host.val ASC NULLS LAST] RANGE 5s FILL NULL |
|
||||
+---------------------+-------+-------------------------------------------------------------------------------+------------------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 1 | 3 |
|
||||
| 1970-01-01T00:00:05 | host1 | 4 | 4 |
|
||||
| 1970-01-01T00:00:10 | host1 | 8 | 7 |
|
||||
| 1970-01-01T00:00:15 | host1 | 10 | 11 |
|
||||
| 1970-01-01T00:00:20 | host1 | 13 | 15 |
|
||||
| 1970-01-01T00:00:00 | host2 | 16 | 18 |
|
||||
| 1970-01-01T00:00:05 | host2 | 19 | 19 |
|
||||
| 1970-01-01T00:00:10 | host2 | 23 | 22 |
|
||||
| 1970-01-01T00:00:15 | host2 | 25 | 26 |
|
||||
| 1970-01-01T00:00:20 | host2 | 28 | 30 |
|
||||
+---------------------+-------+-------------------------------------------------------------------------------+------------------------------------------------------------------------------+
|
||||
+---------------------+-------+---------------------------------------------------------------------+--------------------------------------------------------------------+
|
||||
| ts | host | FIRST_VALUE(host.addon) ORDER BY [host.val ASC NULLS LAST] RANGE 5s | LAST_VALUE(host.addon) ORDER BY [host.val ASC NULLS LAST] RANGE 5s |
|
||||
+---------------------+-------+---------------------------------------------------------------------+--------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 1 | 3 |
|
||||
| 1970-01-01T00:00:05 | host1 | 4 | 4 |
|
||||
| 1970-01-01T00:00:10 | host1 | 8 | 7 |
|
||||
| 1970-01-01T00:00:15 | host1 | 10 | 11 |
|
||||
| 1970-01-01T00:00:20 | host1 | 13 | 15 |
|
||||
| 1970-01-01T00:00:00 | host2 | 16 | 18 |
|
||||
| 1970-01-01T00:00:05 | host2 | 19 | 19 |
|
||||
| 1970-01-01T00:00:10 | host2 | 23 | 22 |
|
||||
| 1970-01-01T00:00:15 | host2 | 25 | 26 |
|
||||
| 1970-01-01T00:00:20 | host2 | 28 | 30 |
|
||||
+---------------------+-------+---------------------------------------------------------------------+--------------------------------------------------------------------+
|
||||
|
||||
SELECT ts, host, first_value(addon ORDER BY val ASC NULLS FIRST) RANGE '5s', last_value(addon ORDER BY val ASC NULLS FIRST) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+--------------------------------------------------------------------------------+-------------------------------------------------------------------------------+
|
||||
| ts | host | FIRST_VALUE(host.addon) ORDER BY [host.val ASC NULLS FIRST] RANGE 5s FILL NULL | LAST_VALUE(host.addon) ORDER BY [host.val ASC NULLS FIRST] RANGE 5s FILL NULL |
|
||||
+---------------------+-------+--------------------------------------------------------------------------------+-------------------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 1 | 3 |
|
||||
| 1970-01-01T00:00:05 | host1 | 4 | 4 |
|
||||
| 1970-01-01T00:00:10 | host1 | 7 | 9 |
|
||||
| 1970-01-01T00:00:15 | host1 | 11 | 12 |
|
||||
| 1970-01-01T00:00:20 | host1 | 15 | 14 |
|
||||
| 1970-01-01T00:00:00 | host2 | 16 | 18 |
|
||||
| 1970-01-01T00:00:05 | host2 | 19 | 19 |
|
||||
| 1970-01-01T00:00:10 | host2 | 22 | 24 |
|
||||
| 1970-01-01T00:00:15 | host2 | 26 | 27 |
|
||||
| 1970-01-01T00:00:20 | host2 | 30 | 29 |
|
||||
+---------------------+-------+--------------------------------------------------------------------------------+-------------------------------------------------------------------------------+
|
||||
+---------------------+-------+----------------------------------------------------------------------+---------------------------------------------------------------------+
|
||||
| ts | host | FIRST_VALUE(host.addon) ORDER BY [host.val ASC NULLS FIRST] RANGE 5s | LAST_VALUE(host.addon) ORDER BY [host.val ASC NULLS FIRST] RANGE 5s |
|
||||
+---------------------+-------+----------------------------------------------------------------------+---------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 1 | 3 |
|
||||
| 1970-01-01T00:00:05 | host1 | 4 | 4 |
|
||||
| 1970-01-01T00:00:10 | host1 | 7 | 9 |
|
||||
| 1970-01-01T00:00:15 | host1 | 11 | 12 |
|
||||
| 1970-01-01T00:00:20 | host1 | 15 | 14 |
|
||||
| 1970-01-01T00:00:00 | host2 | 16 | 18 |
|
||||
| 1970-01-01T00:00:05 | host2 | 19 | 19 |
|
||||
| 1970-01-01T00:00:10 | host2 | 22 | 24 |
|
||||
| 1970-01-01T00:00:15 | host2 | 26 | 27 |
|
||||
| 1970-01-01T00:00:20 | host2 | 30 | 29 |
|
||||
+---------------------+-------+----------------------------------------------------------------------+---------------------------------------------------------------------+
|
||||
|
||||
SELECT ts, host, first_value(addon ORDER BY val ASC, ts ASC) RANGE '5s', last_value(addon ORDER BY val ASC, ts ASC) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+-------------------------------------------------------------------------------------------------------+------------------------------------------------------------------------------------------------------+
|
||||
| ts | host | FIRST_VALUE(host.addon) ORDER BY [host.val ASC NULLS LAST, host.ts ASC NULLS LAST] RANGE 5s FILL NULL | LAST_VALUE(host.addon) ORDER BY [host.val ASC NULLS LAST, host.ts ASC NULLS LAST] RANGE 5s FILL NULL |
|
||||
+---------------------+-------+-------------------------------------------------------------------------------------------------------+------------------------------------------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 1 | 3 |
|
||||
| 1970-01-01T00:00:05 | host1 | 4 | 6 |
|
||||
| 1970-01-01T00:00:10 | host1 | 8 | 7 |
|
||||
| 1970-01-01T00:00:15 | host1 | 10 | 11 |
|
||||
| 1970-01-01T00:00:20 | host1 | 13 | 15 |
|
||||
| 1970-01-01T00:00:00 | host2 | 16 | 18 |
|
||||
| 1970-01-01T00:00:05 | host2 | 19 | 21 |
|
||||
| 1970-01-01T00:00:10 | host2 | 23 | 22 |
|
||||
| 1970-01-01T00:00:15 | host2 | 25 | 26 |
|
||||
| 1970-01-01T00:00:20 | host2 | 28 | 30 |
|
||||
+---------------------+-------+-------------------------------------------------------------------------------------------------------+------------------------------------------------------------------------------------------------------+
|
||||
+---------------------+-------+---------------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------------+
|
||||
| ts | host | FIRST_VALUE(host.addon) ORDER BY [host.val ASC NULLS LAST, host.ts ASC NULLS LAST] RANGE 5s | LAST_VALUE(host.addon) ORDER BY [host.val ASC NULLS LAST, host.ts ASC NULLS LAST] RANGE 5s |
|
||||
+---------------------+-------+---------------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 1 | 3 |
|
||||
| 1970-01-01T00:00:05 | host1 | 4 | 6 |
|
||||
| 1970-01-01T00:00:10 | host1 | 8 | 7 |
|
||||
| 1970-01-01T00:00:15 | host1 | 10 | 11 |
|
||||
| 1970-01-01T00:00:20 | host1 | 13 | 15 |
|
||||
| 1970-01-01T00:00:00 | host2 | 16 | 18 |
|
||||
| 1970-01-01T00:00:05 | host2 | 19 | 21 |
|
||||
| 1970-01-01T00:00:10 | host2 | 23 | 22 |
|
||||
| 1970-01-01T00:00:15 | host2 | 25 | 26 |
|
||||
| 1970-01-01T00:00:20 | host2 | 28 | 30 |
|
||||
+---------------------+-------+---------------------------------------------------------------------------------------------+--------------------------------------------------------------------------------------------+
|
||||
|
||||
SELECT ts, host, count(val) RANGE '5s'FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+------------------------------------+
|
||||
| ts | host | COUNT(host.val) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 3 |
|
||||
| 1970-01-01T00:00:05 | host1 | 0 |
|
||||
| 1970-01-01T00:00:10 | host1 | 2 |
|
||||
| 1970-01-01T00:00:15 | host1 | 2 |
|
||||
| 1970-01-01T00:00:20 | host1 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 |
|
||||
| 1970-01-01T00:00:05 | host2 | 0 |
|
||||
| 1970-01-01T00:00:10 | host2 | 2 |
|
||||
| 1970-01-01T00:00:15 | host2 | 2 |
|
||||
| 1970-01-01T00:00:20 | host2 | 2 |
|
||||
+---------------------+-------+------------------------------------+
|
||||
+---------------------+-------+--------------------------+
|
||||
| ts | host | COUNT(host.val) RANGE 5s |
|
||||
+---------------------+-------+--------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 3 |
|
||||
| 1970-01-01T00:00:05 | host1 | 0 |
|
||||
| 1970-01-01T00:00:10 | host1 | 2 |
|
||||
| 1970-01-01T00:00:15 | host1 | 2 |
|
||||
| 1970-01-01T00:00:20 | host1 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 |
|
||||
| 1970-01-01T00:00:05 | host2 | 0 |
|
||||
| 1970-01-01T00:00:10 | host2 | 2 |
|
||||
| 1970-01-01T00:00:15 | host2 | 2 |
|
||||
| 1970-01-01T00:00:20 | host2 | 2 |
|
||||
+---------------------+-------+--------------------------+
|
||||
|
||||
SELECT ts, host, count(distinct val) RANGE '5s'FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+---------------------------------------------+
|
||||
| ts | host | COUNT(DISTINCT host.val) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+---------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 3 |
|
||||
| 1970-01-01T00:00:05 | host1 | 0 |
|
||||
| 1970-01-01T00:00:10 | host1 | 2 |
|
||||
| 1970-01-01T00:00:15 | host1 | 2 |
|
||||
| 1970-01-01T00:00:20 | host1 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 |
|
||||
| 1970-01-01T00:00:05 | host2 | 0 |
|
||||
| 1970-01-01T00:00:10 | host2 | 2 |
|
||||
| 1970-01-01T00:00:15 | host2 | 2 |
|
||||
| 1970-01-01T00:00:20 | host2 | 2 |
|
||||
+---------------------+-------+---------------------------------------------+
|
||||
+---------------------+-------+-----------------------------------+
|
||||
| ts | host | COUNT(DISTINCT host.val) RANGE 5s |
|
||||
+---------------------+-------+-----------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 3 |
|
||||
| 1970-01-01T00:00:05 | host1 | 0 |
|
||||
| 1970-01-01T00:00:10 | host1 | 2 |
|
||||
| 1970-01-01T00:00:15 | host1 | 2 |
|
||||
| 1970-01-01T00:00:20 | host1 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 |
|
||||
| 1970-01-01T00:00:05 | host2 | 0 |
|
||||
| 1970-01-01T00:00:10 | host2 | 2 |
|
||||
| 1970-01-01T00:00:15 | host2 | 2 |
|
||||
| 1970-01-01T00:00:20 | host2 | 2 |
|
||||
+---------------------+-------+-----------------------------------+
|
||||
|
||||
SELECT ts, host, count(*) RANGE '5s'FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+-----------------------------+
|
||||
| ts | host | COUNT(*) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+-----------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 3 |
|
||||
| 1970-01-01T00:00:05 | host1 | 3 |
|
||||
| 1970-01-01T00:00:10 | host1 | 3 |
|
||||
| 1970-01-01T00:00:15 | host1 | 3 |
|
||||
| 1970-01-01T00:00:20 | host1 | 3 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 |
|
||||
| 1970-01-01T00:00:05 | host2 | 3 |
|
||||
| 1970-01-01T00:00:10 | host2 | 3 |
|
||||
| 1970-01-01T00:00:15 | host2 | 3 |
|
||||
| 1970-01-01T00:00:20 | host2 | 3 |
|
||||
+---------------------+-------+-----------------------------+
|
||||
+---------------------+-------+-------------------+
|
||||
| ts | host | COUNT(*) RANGE 5s |
|
||||
+---------------------+-------+-------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 3 |
|
||||
| 1970-01-01T00:00:05 | host1 | 3 |
|
||||
| 1970-01-01T00:00:10 | host1 | 3 |
|
||||
| 1970-01-01T00:00:15 | host1 | 3 |
|
||||
| 1970-01-01T00:00:20 | host1 | 3 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 |
|
||||
| 1970-01-01T00:00:05 | host2 | 3 |
|
||||
| 1970-01-01T00:00:10 | host2 | 3 |
|
||||
| 1970-01-01T00:00:15 | host2 | 3 |
|
||||
| 1970-01-01T00:00:20 | host2 | 3 |
|
||||
+---------------------+-------+-------------------+
|
||||
|
||||
SELECT ts, host, count(distinct *) RANGE '5s'FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+--------------------------------------+
|
||||
| ts | host | COUNT(DISTINCT *) RANGE 5s FILL NULL |
|
||||
+---------------------+-------+--------------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 3 |
|
||||
| 1970-01-01T00:00:05 | host1 | 3 |
|
||||
| 1970-01-01T00:00:10 | host1 | 3 |
|
||||
| 1970-01-01T00:00:15 | host1 | 3 |
|
||||
| 1970-01-01T00:00:20 | host1 | 3 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 |
|
||||
| 1970-01-01T00:00:05 | host2 | 3 |
|
||||
| 1970-01-01T00:00:10 | host2 | 3 |
|
||||
| 1970-01-01T00:00:15 | host2 | 3 |
|
||||
| 1970-01-01T00:00:20 | host2 | 3 |
|
||||
+---------------------+-------+--------------------------------------+
|
||||
+---------------------+-------+----------------------------+
|
||||
| ts | host | COUNT(DISTINCT *) RANGE 5s |
|
||||
+---------------------+-------+----------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 3 |
|
||||
| 1970-01-01T00:00:05 | host1 | 3 |
|
||||
| 1970-01-01T00:00:10 | host1 | 3 |
|
||||
| 1970-01-01T00:00:15 | host1 | 3 |
|
||||
| 1970-01-01T00:00:20 | host1 | 3 |
|
||||
| 1970-01-01T00:00:00 | host2 | 3 |
|
||||
| 1970-01-01T00:00:05 | host2 | 3 |
|
||||
| 1970-01-01T00:00:10 | host2 | 3 |
|
||||
| 1970-01-01T00:00:15 | host2 | 3 |
|
||||
| 1970-01-01T00:00:20 | host2 | 3 |
|
||||
+---------------------+-------+----------------------------+
|
||||
|
||||
-- Test error first_value/last_value
|
||||
SELECT ts, host, first_value(val, val) RANGE '5s' FROM host ALIGN '5s' ORDER BY host, ts;
|
||||
@@ -241,11 +241,11 @@ Affected Rows: 3
|
||||
|
||||
SELECT ts, first_value(val ORDER BY addon ASC) RANGE '5s', last_value(val ORDER BY addon ASC) RANGE '5s' FROM host ALIGN '5s';
|
||||
|
||||
+---------------------+-------------------------------------------------------------------------------+------------------------------------------------------------------------------+
|
||||
| ts | FIRST_VALUE(host.val) ORDER BY [host.addon ASC NULLS LAST] RANGE 5s FILL NULL | LAST_VALUE(host.val) ORDER BY [host.addon ASC NULLS LAST] RANGE 5s FILL NULL |
|
||||
+---------------------+-------------------------------------------------------------------------------+------------------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | 2 | 0 |
|
||||
+---------------------+-------------------------------------------------------------------------------+------------------------------------------------------------------------------+
|
||||
+---------------------+---------------------------------------------------------------------+--------------------------------------------------------------------+
|
||||
| ts | FIRST_VALUE(host.val) ORDER BY [host.addon ASC NULLS LAST] RANGE 5s | LAST_VALUE(host.val) ORDER BY [host.addon ASC NULLS LAST] RANGE 5s |
|
||||
+---------------------+---------------------------------------------------------------------+--------------------------------------------------------------------+
|
||||
| 1970-01-01T00:00:00 | 2 | 0 |
|
||||
+---------------------+---------------------------------------------------------------------+--------------------------------------------------------------------+
|
||||
|
||||
DROP TABLE host;
|
||||
|
||||
|
||||
@@ -20,14 +20,14 @@ Affected Rows: 8
|
||||
|
||||
SELECT ts, host, min(val) RANGE '1d' FROM host ALIGN '1d' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 1d FILL NULL |
|
||||
+---------------------+-------+----------------------------------+
|
||||
| 2024-01-23T00:00:00 | host1 | 0 |
|
||||
| 2024-01-24T00:00:00 | host1 | 2 |
|
||||
| 2024-01-23T00:00:00 | host2 | 4 |
|
||||
| 2024-01-24T00:00:00 | host2 | 6 |
|
||||
+---------------------+-------+----------------------------------+
|
||||
+---------------------+-------+------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 1d |
|
||||
+---------------------+-------+------------------------+
|
||||
| 2024-01-23T00:00:00 | host1 | 0 |
|
||||
| 2024-01-24T00:00:00 | host1 | 2 |
|
||||
| 2024-01-23T00:00:00 | host2 | 4 |
|
||||
| 2024-01-24T00:00:00 | host2 | 6 |
|
||||
+---------------------+-------+------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '1d' FROM host ALIGN '1d' TO UNKNOWN ORDER BY host, ts;
|
||||
|
||||
@@ -35,52 +35,52 @@ Error: 3000(PlanQuery), DataFusion error: Error during planning: Illegal `align
|
||||
|
||||
SELECT ts, host, min(val) RANGE '1d' FROM host ALIGN '1d' TO '1900-01-01T00:00:00+01:00' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 1d FILL NULL |
|
||||
+---------------------+-------+----------------------------------+
|
||||
| 2024-01-22T23:00:00 | host1 | 0 |
|
||||
| 2024-01-23T23:00:00 | host1 | 1 |
|
||||
| 2024-01-24T23:00:00 | host1 | 3 |
|
||||
| 2024-01-22T23:00:00 | host2 | 4 |
|
||||
| 2024-01-23T23:00:00 | host2 | 5 |
|
||||
| 2024-01-24T23:00:00 | host2 | 7 |
|
||||
+---------------------+-------+----------------------------------+
|
||||
+---------------------+-------+------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 1d |
|
||||
+---------------------+-------+------------------------+
|
||||
| 2024-01-22T23:00:00 | host1 | 0 |
|
||||
| 2024-01-23T23:00:00 | host1 | 1 |
|
||||
| 2024-01-24T23:00:00 | host1 | 3 |
|
||||
| 2024-01-22T23:00:00 | host2 | 4 |
|
||||
| 2024-01-23T23:00:00 | host2 | 5 |
|
||||
| 2024-01-24T23:00:00 | host2 | 7 |
|
||||
+---------------------+-------+------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '1d' FROM host ALIGN '1d' TO '2024-01-23T00:00:00+01:00' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 1d FILL NULL |
|
||||
+---------------------+-------+----------------------------------+
|
||||
| 2024-01-22T23:00:00 | host1 | 0 |
|
||||
| 2024-01-23T23:00:00 | host1 | 1 |
|
||||
| 2024-01-24T23:00:00 | host1 | 3 |
|
||||
| 2024-01-22T23:00:00 | host2 | 4 |
|
||||
| 2024-01-23T23:00:00 | host2 | 5 |
|
||||
| 2024-01-24T23:00:00 | host2 | 7 |
|
||||
+---------------------+-------+----------------------------------+
|
||||
+---------------------+-------+------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 1d |
|
||||
+---------------------+-------+------------------------+
|
||||
| 2024-01-22T23:00:00 | host1 | 0 |
|
||||
| 2024-01-23T23:00:00 | host1 | 1 |
|
||||
| 2024-01-24T23:00:00 | host1 | 3 |
|
||||
| 2024-01-22T23:00:00 | host2 | 4 |
|
||||
| 2024-01-23T23:00:00 | host2 | 5 |
|
||||
| 2024-01-24T23:00:00 | host2 | 7 |
|
||||
+---------------------+-------+------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE '1d' FROM host ALIGN '1d' TO '2023-01-01T00:00:00+01:00' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 1d FILL NULL |
|
||||
+---------------------+-------+----------------------------------+
|
||||
| 2024-01-22T23:00:00 | host1 | 0 |
|
||||
| 2024-01-23T23:00:00 | host1 | 1 |
|
||||
| 2024-01-24T23:00:00 | host1 | 3 |
|
||||
| 2024-01-22T23:00:00 | host2 | 4 |
|
||||
| 2024-01-23T23:00:00 | host2 | 5 |
|
||||
| 2024-01-24T23:00:00 | host2 | 7 |
|
||||
+---------------------+-------+----------------------------------+
|
||||
+---------------------+-------+------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 1d |
|
||||
+---------------------+-------+------------------------+
|
||||
| 2024-01-22T23:00:00 | host1 | 0 |
|
||||
| 2024-01-23T23:00:00 | host1 | 1 |
|
||||
| 2024-01-24T23:00:00 | host1 | 3 |
|
||||
| 2024-01-22T23:00:00 | host2 | 4 |
|
||||
| 2024-01-23T23:00:00 | host2 | 5 |
|
||||
| 2024-01-24T23:00:00 | host2 | 7 |
|
||||
+---------------------+-------+------------------------+
|
||||
|
||||
SELECT ts, min(val) RANGE (INTERVAL '1' day) FROM host ALIGN (INTERVAL '1' day) TO '1900-01-01T00:00:00+01:00' by (1) ORDER BY ts;
|
||||
|
||||
+---------------------+----------------------------------------------------------------------------+
|
||||
| ts | MIN(host.val) RANGE IntervalMonthDayNano("18446744073709551616") FILL NULL |
|
||||
+---------------------+----------------------------------------------------------------------------+
|
||||
| 2024-01-22T23:00:00 | 0 |
|
||||
| 2024-01-23T23:00:00 | 1 |
|
||||
| 2024-01-24T23:00:00 | 3 |
|
||||
+---------------------+----------------------------------------------------------------------------+
|
||||
+---------------------+------------------------------------------------------------------+
|
||||
| ts | MIN(host.val) RANGE IntervalMonthDayNano("18446744073709551616") |
|
||||
+---------------------+------------------------------------------------------------------+
|
||||
| 2024-01-22T23:00:00 | 0 |
|
||||
| 2024-01-23T23:00:00 | 1 |
|
||||
| 2024-01-24T23:00:00 | 3 |
|
||||
+---------------------+------------------------------------------------------------------+
|
||||
|
||||
--- ALIGN TO with time zone ---
|
||||
set time_zone='Asia/Shanghai';
|
||||
@@ -90,14 +90,14 @@ Affected Rows: 0
|
||||
---- align to 'Asia/Shanghai' unix epoch 0 ----
|
||||
SELECT ts, host, min(val) RANGE '1d' FROM host ALIGN '1d' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 1d FILL NULL |
|
||||
+---------------------+-------+----------------------------------+
|
||||
| 2024-01-23T08:00:00 | host1 | 0 |
|
||||
| 2024-01-24T08:00:00 | host1 | 2 |
|
||||
| 2024-01-23T08:00:00 | host2 | 4 |
|
||||
| 2024-01-24T08:00:00 | host2 | 6 |
|
||||
+---------------------+-------+----------------------------------+
|
||||
+---------------------+-------+------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 1d |
|
||||
+---------------------+-------+------------------------+
|
||||
| 2024-01-23T08:00:00 | host1 | 0 |
|
||||
| 2024-01-24T08:00:00 | host1 | 2 |
|
||||
| 2024-01-23T08:00:00 | host2 | 4 |
|
||||
| 2024-01-24T08:00:00 | host2 | 6 |
|
||||
+---------------------+-------+------------------------+
|
||||
|
||||
set time_zone='+23:00';
|
||||
|
||||
@@ -106,16 +106,16 @@ Affected Rows: 0
|
||||
---- align to '+23:00' unix epoch 0 ----
|
||||
SELECT ts, host, min(val) RANGE '1d' FROM host ALIGN '1d' ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+----------------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 1d FILL NULL |
|
||||
+---------------------+-------+----------------------------------+
|
||||
| 2024-01-22T23:00:00 | host1 | 0 |
|
||||
| 2024-01-23T23:00:00 | host1 | 1 |
|
||||
| 2024-01-24T23:00:00 | host1 | 3 |
|
||||
| 2024-01-22T23:00:00 | host2 | 4 |
|
||||
| 2024-01-23T23:00:00 | host2 | 5 |
|
||||
| 2024-01-24T23:00:00 | host2 | 7 |
|
||||
+---------------------+-------+----------------------------------+
|
||||
+---------------------+-------+------------------------+
|
||||
| ts | host | MIN(host.val) RANGE 1d |
|
||||
+---------------------+-------+------------------------+
|
||||
| 2024-01-22T23:00:00 | host1 | 0 |
|
||||
| 2024-01-23T23:00:00 | host1 | 1 |
|
||||
| 2024-01-24T23:00:00 | host1 | 3 |
|
||||
| 2024-01-22T23:00:00 | host2 | 4 |
|
||||
| 2024-01-23T23:00:00 | host2 | 5 |
|
||||
| 2024-01-24T23:00:00 | host2 | 7 |
|
||||
+---------------------+-------+------------------------+
|
||||
|
||||
set time_zone='UTC';
|
||||
|
||||
|
||||
Reference in New Issue
Block a user