From 3e5d8a3f3f4585e612daf7f95f38a72629849c54 Mon Sep 17 00:00:00 2001 From: Jordan Frazier Date: Tue, 26 Sep 2023 14:35:39 -0700 Subject: [PATCH 1/7] use order preserving cast --- crates/sparrow-runtime/src/prepare/preparer.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/crates/sparrow-runtime/src/prepare/preparer.rs b/crates/sparrow-runtime/src/prepare/preparer.rs index 565fbb7d7..3d81d8bcb 100644 --- a/crates/sparrow-runtime/src/prepare/preparer.rs +++ b/crates/sparrow-runtime/src/prepare/preparer.rs @@ -11,6 +11,7 @@ use error_stack::{IntoReport, IntoReportCompat, ResultExt}; use futures::stream::FuturesUnordered; use futures::{StreamExt, TryStreamExt}; use sparrow_api::kaskada::v1alpha::{PreparedFile, SourceData, TableConfig}; +use sparrow_kernels::order_preserving_cast_to_u64; use uuid::Uuid; use crate::stores::{ObjectStoreRegistry, ObjectStoreUrl}; @@ -174,8 +175,8 @@ impl Preparer { let num_rows = batch.num_rows(); let subsort = if let Some(subsort_column_name) = subsort_column_name.as_ref() { - let subsort = get_required_column(&batch, subsort_column_name)?; - arrow::compute::cast(subsort.as_ref(), &DataType::UInt64) + let subsort = get_required_column(&batch, &subsort_column_name)?; + order_preserving_cast_to_u64(subsort) .into_report() .change_context_lazy(|| Error::ConvertSubsort(subsort.data_type().clone()))? } else { From 9cd4a9eeadacb3c9b298bd839b77f438761aad21 Mon Sep 17 00:00:00 2001 From: Jordan Frazier Date: Tue, 26 Sep 2023 15:28:15 -0700 Subject: [PATCH 2/7] Remove column behavior and use new prepare batch --- crates/sparrow-runtime/src/prepare.rs | 1 - .../src/prepare/column_behavior.rs | 421 ------------------ .../src/prepare/execute_input_stream.rs | 58 +-- .../src/prepare/prepare_input_stream.rs | 101 +---- .../sparrow-runtime/src/prepare/preparer.rs | 146 +++--- 5 files changed, 101 insertions(+), 626 deletions(-) delete mode 100644 crates/sparrow-runtime/src/prepare/column_behavior.rs diff --git a/crates/sparrow-runtime/src/prepare.rs b/crates/sparrow-runtime/src/prepare.rs index 45fcc87f4..7c5159ef2 100644 --- a/crates/sparrow-runtime/src/prepare.rs +++ b/crates/sparrow-runtime/src/prepare.rs @@ -11,7 +11,6 @@ use sparrow_api::kaskada::v1alpha::{ slice_plan, source_data, PreparedFile, SourceData, TableConfig, }; -mod column_behavior; mod error; pub(crate) mod execute_input_stream; mod prepare_input_stream; diff --git a/crates/sparrow-runtime/src/prepare/column_behavior.rs b/crates/sparrow-runtime/src/prepare/column_behavior.rs deleted file mode 100644 index 6c5018d05..000000000 --- a/crates/sparrow-runtime/src/prepare/column_behavior.rs +++ /dev/null @@ -1,421 +0,0 @@ -use anyhow::Context; -use arrow::{ - array::ArrayRef, - datatypes::{DataType, Field, SchemaRef}, - record_batch::RecordBatch, -}; -use sparrow_core::context_code; - -use std::sync::Arc; - -use anyhow::anyhow; -use arrow::array::{Array, UInt64Array}; - -use error_stack::{IntoReport, IntoReportCompat, ResultExt}; -use sparrow_arrow::utils::make_null_array; -use sparrow_kernels::order_preserving_cast_to_u64; - -use crate::prepare::Error; - -/// Defines how each column in the resulting prepared batch -/// is computed. -#[derive(Debug)] -pub enum ColumnBehavior { - /// Cast the given column to the given data type. - Cast { - index: usize, - data_type: DataType, - nullable: bool, - }, - /// Perform an "order preserving" cast from a primitive number to u64. - OrderPreservingCastToU64 { index: usize, nullable: bool }, - /// Reference the given column. - Reference { index: usize, nullable: bool }, - /// Hash the given column. - EntityKey { index: usize, nullable: bool }, - /// Generates a row of monotically increasing u64s, starting - /// at the defined offset. - SequentialU64 { next_offset: u64 }, - /// Create a column of nulls. - /// - /// The `DataType` indicates the type of column to produce. - Null(DataType), -} - -impl ColumnBehavior { - /// Create a column behavior that references the given field as the - /// specified type. - /// - /// This may be a reference (if the source already has that type) or a cast. - /// - /// # Errors - /// Internal error if the source field doesn't exist or is not convertible - /// to the given type. - pub fn try_new_cast( - source_schema: &SchemaRef, - source_name: &str, - to_type: &DataType, - nullable: bool, - ) -> anyhow::Result { - let (source_index, source_field) = source_schema - .column_with_name(source_name) - .with_context(|| { - context_code!( - tonic::Code::Internal, - "column to cast '{}' not present in schema {:?}", - source_name, - source_schema - ) - })?; - - match (source_field.data_type(), to_type) { - (from, to) if from == to => Ok(Self::Reference { - index: source_index, - nullable, - }), - (DataType::Timestamp(_from_unit_, Some(_)), DataType::Timestamp(to_unit, None)) => { - Ok(Self::Cast { - index: source_index, - data_type: DataType::Timestamp(to_unit.clone(), None), - nullable, - }) - } - (from, to) if arrow::compute::can_cast_types(from, to) => Ok(Self::Cast { - index: source_index, - data_type: to_type.clone(), - nullable, - }), - (_, _) => Err(anyhow!( - "Expected column '{}' to be castable to {:?}, but {:?} was not", - source_field.name(), - to_type, - source_field.data_type(), - ) - .context(tonic::Code::Internal)), - } - } - - /// Create a column behavior specifically for the subsort. - /// - /// This is a bit different from other columns since we want to attempt - /// to preserve the order. - /// - /// # Errors - /// Internal error if the source field doesn't exist or is not valid - /// as the subsort. - pub fn try_new_subsort(source_schema: &SchemaRef, source_name: &str) -> anyhow::Result { - let (source_index, source_field) = source_schema - .column_with_name(source_name) - .with_context(|| { - context_code!( - tonic::Code::Internal, - "subsort column '{}' not present in schema {:?}", - source_name, - source_schema - ) - })?; - - match source_field.data_type() { - DataType::UInt64 => Ok(Self::Reference { - index: source_index, - nullable: false, - }), - DataType::UInt8 | DataType::UInt16 | DataType::UInt32 => Ok(Self::Cast { - index: source_index, - data_type: DataType::UInt64, - nullable: false, - }), - - DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64 => { - Ok(Self::OrderPreservingCastToU64 { - index: source_index, - nullable: false, - }) - } - - _ => Err(anyhow!( - "Expected subsort column '{}' to be numeric but was {:?}", - source_field.name(), - source_field.data_type(), - ) - .context(tonic::Code::Internal)), - } - } - - /// Create a column behavior that generates a random u64. - pub fn try_default_subsort(prepare_hash: u64) -> anyhow::Result { - Ok(Self::SequentialU64 { - next_offset: prepare_hash, - }) - } - - /// Create a column behavior that hashes the given field (and index) to - /// `u64`. This is only used for the entity key. - /// - /// # Errors - /// Internal error if the source field doesn't exist. - pub fn try_new_entity_key( - source_schema: &SchemaRef, - source_name: &str, - nullable: bool, - ) -> anyhow::Result { - let (source_index, _) = source_schema - .column_with_name(source_name) - .with_context(|| { - context_code!( - tonic::Code::Internal, - "entity key column '{}' not present in schema {:?}", - source_name, - source_schema - ) - })?; - - Ok(Self::EntityKey { - index: source_index, - nullable, - }) - } - - /// Create a behavior that projects a field from the source schema to the - /// result field. - /// - /// If the `result_field` doesn't exist in the `source_schema` the result - /// is a column of nulls. - /// - /// In the special case of a `Timestamp` with a time zone, this will cast to - /// a `Timestamp` with no time zone. - /// - /// # Errors - /// Internal error if the type of the column in the source schema is - /// different than in the result schema. - pub fn try_cast_or_reference_or_null( - source_schema: &SchemaRef, - result_field: &Field, - ) -> anyhow::Result { - if let Some((column, source_field)) = source_schema.column_with_name(result_field.name()) { - match (source_field.data_type(), result_field.data_type()) { - (DataType::Timestamp(_, Some(_)), DataType::Timestamp(to_unit, None)) => { - Ok(Self::Cast { - index: column, - data_type: DataType::Timestamp(to_unit.clone(), None), - nullable: true, - }) - } - (source_type, expected_type) if source_type == expected_type => { - Ok(Self::Reference { - index: column, - nullable: true, - }) - } - (source_type, expected_type) => Err(anyhow!( - "Unable to get field '{}' as type {:?} from file containing {:?}", - result_field.name(), - expected_type, - source_type, - ) - .context(tonic::Code::Internal)), - } - } else { - anyhow::ensure!( - result_field.is_nullable(), - "Result field must be nullable if absent in source, but was {:?}", - result_field - ); - Ok(Self::Null(result_field.data_type().clone())) - } - } - - pub async fn get_result( - &mut self, - batch: &RecordBatch, - ) -> error_stack::Result { - let result = match self { - ColumnBehavior::Cast { - index, - data_type, - nullable, - } => { - let column = batch.column(*index); - error_stack::ensure!( - *nullable || column.null_count() == 0, - Error::NullInNonNullableColumn { - field: batch.schema().field(*index).name().to_owned(), - null_count: column.null_count() - } - ); - arrow::compute::cast(column, data_type) - .into_report() - .change_context(Error::PreparingColumn)? - } - ColumnBehavior::OrderPreservingCastToU64 { index, nullable } => { - let column = batch.column(*index); - error_stack::ensure!( - *nullable || column.null_count() == 0, - Error::NullInNonNullableColumn { - field: batch.schema().field(*index).name().to_owned(), - null_count: column.null_count() - } - ); - - order_preserving_cast_to_u64(column) - .into_report() - .change_context(Error::PreparingColumn)? - } - ColumnBehavior::Reference { index, nullable } => { - let column = batch.column(*index); - error_stack::ensure!( - *nullable || column.null_count() == 0, - Error::NullInNonNullableColumn { - field: batch.schema().field(*index).name().to_owned(), - null_count: column.null_count() - } - ); - column.clone() - } - ColumnBehavior::EntityKey { index, nullable } => { - let column = batch.column(*index); - error_stack::ensure!( - *nullable || column.null_count() == 0, - Error::NullInNonNullableColumn { - field: batch.schema().field(*index).name().to_owned(), - null_count: column.null_count() - } - ); - - let entity_column = - sparrow_arrow::hash::hash(column).change_context(Error::PreparingColumn)?; - - Arc::new(entity_column) - } - ColumnBehavior::Null(result_type) => make_null_array(result_type, batch.num_rows()), - ColumnBehavior::SequentialU64 { next_offset } => { - // 1. The result is going to be [next_offset, next_offset + length). - let length = batch.num_rows() as u64; - // TODO: There is a potential u64 overflow. If an overflow will happen - // the subsort will start at 0 to length. - let (start, end) = if let Some(end) = next_offset.checked_add(length) { - (*next_offset, end) - } else { - (0, length) - }; - - let result = UInt64Array::from_iter_values(start..end); - - // 2. Update next_offset so the *next batch* gets new values. - *next_offset = end; - - Arc::new(result) - } - }; - Ok(result) - } -} - -#[cfg(test)] -mod tests { - use std::sync::Arc; - - use super::ColumnBehavior; - use arrow::array::{Int64Array, TimestampNanosecondArray, UInt64Array}; - use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit}; - use arrow::record_batch::RecordBatch; - use static_init::dynamic; - - #[dynamic] - static COMPLETE_SCHEMA: SchemaRef = { - Arc::new(Schema::new(vec![ - Field::new( - "_time", - DataType::Timestamp(TimeUnit::Nanosecond, None), - false, - ), - Field::new("_subsort", DataType::UInt64, false), - Field::new("_key_hash", DataType::UInt64, false), - Field::new("a", DataType::Int64, true), - ])) - }; - - fn make_test_batch(num_rows: usize) -> RecordBatch { - let time = TimestampNanosecondArray::from_iter_values(0..num_rows as i64); - let subsort = UInt64Array::from_iter_values(0..num_rows as u64); - let key = UInt64Array::from_iter_values(0..num_rows as u64); - let a = Int64Array::from_iter_values(0..num_rows as i64); - - RecordBatch::try_new( - COMPLETE_SCHEMA.clone(), - vec![ - Arc::new(time), - Arc::new(subsort), - Arc::new(key), - Arc::new(a), - ], - ) - .unwrap() - } - - #[tokio::test] - async fn test_sequential_u64_zero() { - let mut behavior = ColumnBehavior::SequentialU64 { next_offset: 0 }; - assert_eq!( - behavior - .get_result(&make_test_batch(5)) - .await - .unwrap() - .as_ref(), - &UInt64Array::from(vec![0, 1, 2, 3, 4]) - ); - assert_eq!( - behavior - .get_result(&make_test_batch(3)) - .await - .unwrap() - .as_ref(), - &UInt64Array::from(vec![5, 6, 7]) - ); - } - - #[tokio::test] - async fn test_sequential_u64_overflow() { - let mut behavior = ColumnBehavior::SequentialU64 { - next_offset: u64::MAX - 3, - }; - // Current behavior is to immediately wrap. - assert_eq!( - behavior - .get_result(&make_test_batch(5)) - .await - .unwrap() - .as_ref(), - &UInt64Array::from(vec![0, 1, 2, 3, 4]) - ); - assert_eq!( - behavior - .get_result(&make_test_batch(3)) - .await - .unwrap() - .as_ref(), - &UInt64Array::from(vec![5, 6, 7]) - ); - } - - #[tokio::test] - async fn test_sequential_u64_nonzero() { - let mut behavior = ColumnBehavior::SequentialU64 { next_offset: 100 }; - assert_eq!( - behavior - .get_result(&make_test_batch(5)) - .await - .unwrap() - .as_ref(), - &UInt64Array::from(vec![100, 101, 102, 103, 104]) - ); - assert_eq!( - behavior - .get_result(&make_test_batch(3)) - .await - .unwrap() - .as_ref(), - &UInt64Array::from(vec![105, 106, 107]) - ); - } -} diff --git a/crates/sparrow-runtime/src/prepare/execute_input_stream.rs b/crates/sparrow-runtime/src/prepare/execute_input_stream.rs index 8932980fa..bdb194fba 100644 --- a/crates/sparrow-runtime/src/prepare/execute_input_stream.rs +++ b/crates/sparrow-runtime/src/prepare/execute_input_stream.rs @@ -1,3 +1,4 @@ +use std::sync::atomic::AtomicU64; use std::sync::Arc; use anyhow::Context; @@ -17,8 +18,7 @@ use sparrow_core::TableSchema; use crate::key_hash_inverse::ThreadSafeKeyHashInverse; use crate::prepare::slice_preparer::SlicePreparer; use crate::prepare::Error; - -use super::column_behavior::ColumnBehavior; +use crate::preparer::prepare_batch; /// Struct to store logic for handling late data with a watermark and /// bounded lateness. @@ -70,39 +70,6 @@ pub async fn prepare_input<'a>( let prepared_schema = TableSchema::try_from_data_schema(projected_schema.clone())?; let prepared_schema = prepared_schema.schema_ref().clone(); - // Add column behaviors for each of the 3 key columns. - let mut columns = Vec::with_capacity(prepared_schema.fields().len()); - columns.push(ColumnBehavior::try_new_cast( - &raw_schema, - &config.time_column_name, - &TimestampNanosecondType::DATA_TYPE, - false, - )?); - if let Some(subsort_column_name) = &config.subsort_column_name { - columns.push(ColumnBehavior::try_new_subsort( - &raw_schema, - subsort_column_name, - )?); - } else { - columns.push(ColumnBehavior::try_default_subsort(prepare_hash)?); - } - - columns.push(ColumnBehavior::try_new_entity_key( - &raw_schema, - &config.group_column_name, - false, - )?); - - // Add column behaviors for each column. This means we include the key columns - // redundantly, but cleaning that up is a big refactor. - // See https://github.com/riptano/kaskada/issues/90 - for field in projected_schema.fields() { - columns.push(ColumnBehavior::try_cast_or_reference_or_null( - &raw_schema, - field, - )?); - } - // we've already checked that the group column exists so we can just unwrap it here let (entity_column_index, entity_key_column) = raw_schema .column_with_name(&config.group_column_name) @@ -185,21 +152,14 @@ pub async fn prepare_input<'a>( // 2. Slicing may reduce the number of entities to operate and sort on. let record_batch = slice_preparer.slice_batch(record_batch)?; - // 3. Prepare each of the columns by getting the column behavior result - let mut prepared_columns: Vec = Vec::new(); - for c in columns.iter_mut() { - let result = c.get_result(&record_batch).await?; - prepared_columns.push(result); - } - - // 4. Update the key hash mappings - let key_column = record_batch.column(entity_column_index); - let key_hashes = prepared_columns.get(2).expect("key column"); - update_key_inverse(key_column, key_hashes, key_hash_inverse.clone()).await?; + // 3. Prepare the batch + let next_subsort = AtomicU64::new(prepare_hash); + let prepared_batch = prepare_batch(&record_batch, &config, prepared_schema.clone(), &next_subsort, None).unwrap(); - let record_batch = RecordBatch::try_new(prepared_schema.clone(), prepared_columns) - .into_report() - .change_context(Error::PreparingColumn)?; + // 4. Update the key inverse + let key_hash_column = prepared_batch.column(2); + let key_column = prepared_batch.column(entity_column_index + 3); + update_key_inverse(key_column, key_hash_column, key_hash_inverse.clone()).await?; // 5. After preparing the batch, concatenate the leftovers from the previous batch // Note this is done after slicing, since the leftovers were already sliced. diff --git a/crates/sparrow-runtime/src/prepare/prepare_input_stream.rs b/crates/sparrow-runtime/src/prepare/prepare_input_stream.rs index 3e9d75be9..be91fce01 100644 --- a/crates/sparrow-runtime/src/prepare/prepare_input_stream.rs +++ b/crates/sparrow-runtime/src/prepare/prepare_input_stream.rs @@ -1,23 +1,21 @@ use std::borrow::BorrowMut; +use std::sync::atomic::AtomicU64; use anyhow::Context; use arrow::array::{ArrayRef, UInt64Array}; -use arrow::compute::SortColumn; -use arrow::datatypes::{ArrowPrimitiveType, TimestampNanosecondType}; use arrow::record_batch::RecordBatch; use error_stack::{IntoReport, IntoReportCompat, ResultExt}; use futures::stream::BoxStream; use futures::StreamExt; -use itertools::Itertools; use sparrow_api::kaskada::v1alpha::{slice_plan, TableConfig}; use sparrow_arrow::downcast::downcast_primitive_array; use sparrow_core::TableSchema; use crate::prepare::slice_preparer::SlicePreparer; use crate::prepare::Error; +use crate::preparer::prepare_batch; use crate::RawMetadata; -use super::column_behavior::ColumnBehavior; use super::PrepareMetadata; /// Creates a stream over unordered, unprepared batches that is responsible @@ -30,7 +28,7 @@ use super::PrepareMetadata; /// 4. Computing the key-hash and key batch metadata. pub async fn prepare_input<'a>( mut reader: BoxStream<'a, error_stack::Result>, - config: &TableConfig, + config: &'a TableConfig, raw_metadata: RawMetadata, prepare_hash: u64, slice: &Option, @@ -40,39 +38,6 @@ pub async fn prepare_input<'a>( let prepared_schema = TableSchema::try_from_data_schema(raw_metadata.table_schema.clone())?; let prepared_schema = prepared_schema.schema_ref().clone(); - // Add column behaviors for each of the 3 key columns. - let mut columns = Vec::with_capacity(prepared_schema.fields().len()); - columns.push(ColumnBehavior::try_new_cast( - &raw_metadata.raw_schema, - &config.time_column_name, - &TimestampNanosecondType::DATA_TYPE, - false, - )?); - if let Some(subsort_column_name) = &config.subsort_column_name { - columns.push(ColumnBehavior::try_new_subsort( - &raw_metadata.raw_schema, - subsort_column_name, - )?); - } else { - columns.push(ColumnBehavior::try_default_subsort(prepare_hash)?); - } - - columns.push(ColumnBehavior::try_new_entity_key( - &raw_metadata.raw_schema, - &config.group_column_name, - false, - )?); - - // Add column behaviors for each column. This means we include the key columns - // redundantly, but cleaning that up is a big refactor. - // See https://github.com/riptano/kaskada/issues/90 - for field in raw_metadata.table_schema.fields() { - columns.push(ColumnBehavior::try_cast_or_reference_or_null( - &raw_metadata.raw_schema, - field, - )?); - } - // we've already checked that the group column exists so we can just unwrap it here let (entity_column_index, entity_key_column) = raw_metadata .raw_schema @@ -85,65 +50,23 @@ pub async fn prepare_input<'a>( )?; let mut metadata = PrepareMetadata::new(entity_key_column.data_type().clone()); - + let next_subsort = AtomicU64::new(prepare_hash); Ok(async_stream::try_stream! { while let Some(Ok(batch)) = reader.next().await { // 1. Slicing may reduce the number of entities to operate and sort on. let read_batch = slice_preparer.slice_batch(batch)?; - // 2. Prepare each of the columns by getting the column behavior result - let mut prepared_columns = Vec::new(); - for c in columns.iter_mut() { - let result = c - .get_result(&read_batch) - .await?; - - // update_key_metadata(c, &read_batch, &result, &mut metadata)?; - prepared_columns.push(result); - } - - // 3. Update the key hash mappings - let key_column = read_batch.column(entity_column_index); - let key_hashes = prepared_columns.get(2).expect("key column"); - update_key_metadata(key_column, key_hashes, &mut metadata)?; - // 4. Pull out the time, subsort and key hash columns to sort the record batch - let time_column = &prepared_columns[0]; - let subsort_column = &prepared_columns[1]; - let key_hash_column = &prepared_columns[2]; - let sorted_indices = arrow::compute::lexsort_to_indices( - &[ - SortColumn { - values: time_column.clone(), - options: None, - }, - SortColumn { - values: subsort_column.clone(), - options: None, - }, - SortColumn { - values: key_hash_column.clone(), - options: None, - }, - ], - None, - ) - .into_report() - .change_context(Error::SortingBatch)?; + // 2. Prepare the batch + let prepared_batch = prepare_batch(&read_batch, config, prepared_schema.clone(), &next_subsort, None).unwrap(); - // 5. Produce the fully ordered record batch by taking the indices out from the - // columns - let prepared_columns: Vec<_> = prepared_columns - .iter() - .map(|column| arrow::compute::take(column.as_ref(), &sorted_indices, None)) - .try_collect() - .into_report() - .change_context(Error::Internal)?; + // 3. Update the key inverse + let key_hash_column = prepared_batch.column(2); + let key_column = prepared_batch.column(entity_column_index + 3); + update_key_metadata(key_column, key_hash_column, &mut metadata)?; - let batch = RecordBatch::try_new(prepared_schema.clone(), prepared_columns.clone()) - .into_report() - .change_context(Error::Internal)?; + // 4. Produce the batch and associated metadata let metadata = metadata.get_flush_metadata()?; - let result = (batch, metadata); + let result = (prepared_batch, metadata); yield result; } } diff --git a/crates/sparrow-runtime/src/prepare/preparer.rs b/crates/sparrow-runtime/src/prepare/preparer.rs index 3d81d8bcb..7063f383f 100644 --- a/crates/sparrow-runtime/src/prepare/preparer.rs +++ b/crates/sparrow-runtime/src/prepare/preparer.rs @@ -166,77 +166,91 @@ impl Preparer { /// - This sorts the batch by time, subsort and key hash. /// - This adds or casts columns as needed. pub fn prepare_batch(&self, batch: RecordBatch) -> error_stack::Result { - let time_column_name = self.table_config.time_column_name.clone(); - let subsort_column_name = self.table_config.subsort_column_name.clone(); - let key_column_name = self.table_config.group_column_name.clone(); + prepare_batch( + &batch, + &self.table_config, + self.prepared_schema.clone(), + &self.next_subsort, + self.time_multiplier, + ) + } +} - let time = get_required_column(&batch, &time_column_name)?; - let time = cast_to_timestamp(time, self.time_multiplier)?; +pub fn prepare_batch( + batch: &RecordBatch, + table_config: &TableConfig, + prepared_schema: SchemaRef, + next_subsort: &AtomicU64, + time_multiplier: Option, +) -> error_stack::Result { + let time_column_name = table_config.time_column_name.clone(); + let subsort_column_name = table_config.subsort_column_name.clone(); + let key_column_name = table_config.group_column_name.clone(); + + let time = get_required_column(batch, &time_column_name)?; + let time = cast_to_timestamp(time, time_multiplier)?; + + let num_rows = batch.num_rows(); + let subsort = if let Some(subsort_column_name) = subsort_column_name.as_ref() { + let subsort = get_required_column(batch, subsort_column_name)?; + order_preserving_cast_to_u64(subsort) + .into_report() + .change_context_lazy(|| Error::ConvertSubsort(subsort.data_type().clone()))? + } else { + let subsort_start = next_subsort.fetch_add(num_rows as u64, Ordering::SeqCst); + let subsort: UInt64Array = (subsort_start..).take(num_rows).collect(); + Arc::new(subsort) + }; - let num_rows = batch.num_rows(); - let subsort = if let Some(subsort_column_name) = subsort_column_name.as_ref() { - let subsort = get_required_column(&batch, &subsort_column_name)?; - order_preserving_cast_to_u64(subsort) - .into_report() - .change_context_lazy(|| Error::ConvertSubsort(subsort.data_type().clone()))? + let key = get_required_column(batch, &key_column_name)?; + let key_hash = + sparrow_arrow::hash::hash(key.as_ref()).change_context(Error::HashingKeyArray)?; + let key_hash: ArrayRef = Arc::new(key_hash); + + let mut columns = Vec::with_capacity(prepared_schema.fields().len()); + + let indices = arrow::compute::lexsort_to_indices( + &[ + SortColumn { + values: time.clone(), + options: None, + }, + SortColumn { + values: subsort.clone(), + options: None, + }, + SortColumn { + values: key_hash.clone(), + options: None, + }, + ], + None, + ) + .into_report() + .change_context(Error::SortingBatch)?; + + let sort = |array: &ArrayRef| { + arrow::compute::take(array.as_ref(), &indices, None) + .into_report() + .change_context(Error::SortingBatch) + }; + columns.push(sort(&time)?); + columns.push(sort(&subsort)?); + columns.push(sort(&key_hash)?); + + // TODO: Slicing? + for field in prepared_schema.fields().iter().skip(3) { + let column = if let Some(column) = batch.column_by_name(field.name()) { + sort(column)? } else { - let subsort_start = self - .next_subsort - .fetch_add(num_rows as u64, Ordering::SeqCst); - let subsort: UInt64Array = (subsort_start..).take(num_rows).collect(); - Arc::new(subsort) - }; - - let key = get_required_column(&batch, &key_column_name)?; - let key_hash = - sparrow_arrow::hash::hash(key.as_ref()).change_context(Error::HashingKeyArray)?; - let key_hash: ArrayRef = Arc::new(key_hash); - - let mut columns = Vec::with_capacity(self.prepared_schema.fields().len()); - - let indices = arrow::compute::lexsort_to_indices( - &[ - SortColumn { - values: time.clone(), - options: None, - }, - SortColumn { - values: subsort.clone(), - options: None, - }, - SortColumn { - values: key_hash.clone(), - options: None, - }, - ], - None, - ) - .into_report() - .change_context(Error::SortingBatch)?; - - let sort = |array: &ArrayRef| { - arrow::compute::take(array.as_ref(), &indices, None) - .into_report() - .change_context(Error::SortingBatch) + arrow::array::new_null_array(field.data_type(), num_rows) }; - columns.push(sort(&time)?); - columns.push(sort(&subsort)?); - columns.push(sort(&key_hash)?); - - // TODO: Slicing? - for field in self.prepared_schema.fields().iter().skip(3) { - let column = if let Some(column) = batch.column_by_name(field.name()) { - sort(column)? - } else { - arrow::array::new_null_array(field.data_type(), num_rows) - }; - columns.push(column) - } - let prepared = RecordBatch::try_new(self.prepared_schema.clone(), columns) - .into_report() - .change_context(Error::CreatingBatch)?; - Ok(prepared) + columns.push(column) } + let prepared = RecordBatch::try_new(prepared_schema.clone(), columns) + .into_report() + .change_context(Error::CreatingBatch)?; + Ok(prepared) } fn get_required_column<'a>( From ff2e57c54c6b826f1dd06b762d7e7373b3f75c1c Mon Sep 17 00:00:00 2001 From: Jordan Frazier Date: Tue, 26 Sep 2023 17:34:36 -0700 Subject: [PATCH 3/7] Fix batch naming --- crates/sparrow-runtime/src/prepare/execute_input_stream.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/sparrow-runtime/src/prepare/execute_input_stream.rs b/crates/sparrow-runtime/src/prepare/execute_input_stream.rs index bdb194fba..cd29452c2 100644 --- a/crates/sparrow-runtime/src/prepare/execute_input_stream.rs +++ b/crates/sparrow-runtime/src/prepare/execute_input_stream.rs @@ -154,11 +154,11 @@ pub async fn prepare_input<'a>( // 3. Prepare the batch let next_subsort = AtomicU64::new(prepare_hash); - let prepared_batch = prepare_batch(&record_batch, &config, prepared_schema.clone(), &next_subsort, None).unwrap(); + let record_batch = prepare_batch(&record_batch, &config, prepared_schema.clone(), &next_subsort, None).unwrap(); // 4. Update the key inverse - let key_hash_column = prepared_batch.column(2); - let key_column = prepared_batch.column(entity_column_index + 3); + let key_hash_column = record_batch.column(2); + let key_column = record_batch.column(entity_column_index + 3); update_key_inverse(key_column, key_hash_column, key_hash_inverse.clone()).await?; // 5. After preparing the batch, concatenate the leftovers from the previous batch From eb7820f2491364a0b157c64740e3c2a77cc883f9 Mon Sep 17 00:00:00 2001 From: Jordan Frazier Date: Tue, 26 Sep 2023 22:11:30 -0700 Subject: [PATCH 4/7] go back to non-order-preserving cast --- crates/sparrow-runtime/src/prepare/preparer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/sparrow-runtime/src/prepare/preparer.rs b/crates/sparrow-runtime/src/prepare/preparer.rs index 7063f383f..f3a2f7d79 100644 --- a/crates/sparrow-runtime/src/prepare/preparer.rs +++ b/crates/sparrow-runtime/src/prepare/preparer.rs @@ -193,7 +193,7 @@ pub fn prepare_batch( let num_rows = batch.num_rows(); let subsort = if let Some(subsort_column_name) = subsort_column_name.as_ref() { let subsort = get_required_column(batch, subsort_column_name)?; - order_preserving_cast_to_u64(subsort) + arrow::compute::cast(subsort.as_ref(), &DataType::UInt64) .into_report() .change_context_lazy(|| Error::ConvertSubsort(subsort.data_type().clone()))? } else { From 4a306b9e0c703e7222e5c03199187ab60a66958d Mon Sep 17 00:00:00 2001 From: Jordan Frazier Date: Tue, 26 Sep 2023 22:37:08 -0700 Subject: [PATCH 5/7] update subsorts for all tests --- .../tests/e2e/aggregation_tests.rs | 654 +++++++++--------- crates/sparrow-main/tests/e2e/cast_tests.rs | 166 ++--- .../sparrow-main/tests/e2e/coalesce_tests.rs | 122 ++-- .../sparrow-main/tests/e2e/collect_tests.rs | 536 +++++++------- .../tests/e2e/comparison_tests.rs | 288 ++++---- .../tests/e2e/decoration_tests.rs | 42 +- .../tests/e2e/entity_key_output_tests.rs | 72 +- .../sparrow-main/tests/e2e/equality_tests.rs | 500 ++++++------- .../sparrow-main/tests/e2e/formula_tests.rs | 24 +- .../sparrow-main/tests/e2e/general_tests.rs | 150 ++-- crates/sparrow-main/tests/e2e/json_tests.rs | 60 +- .../sparrow-main/tests/e2e/logical_tests.rs | 354 +++++----- crates/sparrow-main/tests/e2e/lookup_tests.rs | 116 ++-- crates/sparrow-main/tests/e2e/math_tests.rs | 432 ++++++------ .../sparrow-main/tests/e2e/multiple_tables.rs | 212 +++--- .../tests/e2e/notebooks/event_data_tests.rs | 114 +-- .../sparrow-main/tests/e2e/parquet_tests.rs | 30 +- .../sparrow-main/tests/e2e/prepare_tests.rs | 12 +- crates/sparrow-main/tests/e2e/record_tests.rs | 84 +-- crates/sparrow-main/tests/e2e/shift_tests.rs | 19 +- crates/sparrow-main/tests/e2e/string_tests.rs | 48 +- crates/sparrow-main/tests/e2e/tick_tests.rs | 232 +++---- crates/sparrow-main/tests/e2e/time_tests.rs | 362 +++++----- crates/sparrow-main/tests/e2e/when_tests.rs | 38 +- .../tests/e2e/windowed_aggregation_tests.rs | 304 ++++---- .../sparrow-main/tests/e2e/with_key_tests.rs | 70 +- 26 files changed, 2520 insertions(+), 2521 deletions(-) diff --git a/crates/sparrow-main/tests/e2e/aggregation_tests.rs b/crates/sparrow-main/tests/e2e/aggregation_tests.rs index d523cd2ea..4f40b6f65 100644 --- a/crates/sparrow-main/tests/e2e/aggregation_tests.rs +++ b/crates/sparrow-main/tests/e2e/aggregation_tests.rs @@ -22,12 +22,12 @@ async fn test_sum_i64_final() { async fn test_sum_since_tick_i64() { insta::assert_snapshot!(QueryFixture::new("{ sum_field: sum(Numbers.m, window=since(daily())) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sum_field - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,22 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,22 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,34 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,34 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,22 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,22 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,34 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,34 "###); } @@ -35,48 +35,48 @@ async fn test_sum_since_tick_i64() { async fn test_since_tick_with_pipe_sum() { insta::assert_snapshot!(QueryFixture::new("{ sum_field: sum(Numbers.m, window=since( (Numbers.m | (daily() or $input > 10)) ) ) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sum_field - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,22 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,22 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A, "###); } #[tokio::test] async fn test_since_predicate_sum_i64() { insta::assert_snapshot!(QueryFixture::new("{ sum_field: sum(Numbers.m, window=since(Numbers.n > 7)) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sum_field - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,17 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,12 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,17 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,12 "###); } #[tokio::test] async fn test_sliding_tick_sum_i64() { insta::assert_snapshot!(QueryFixture::new("{ sum_field: sum(Numbers.m, window=sliding(2, daily())) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sum_field - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,22 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,22 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,34 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,34 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,22 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,22 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,34 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,34 "###); } #[tokio::test] async fn test_sliding_predicate_sum_i64() { insta::assert_snapshot!(QueryFixture::new("{ sum_field: sum(Numbers.m, window=sliding(2, Numbers.m > 1)) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sum_field - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,22 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,17 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,29 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,12 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,22 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,17 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,29 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,12 "###); } @@ -84,12 +84,12 @@ async fn test_sliding_predicate_sum_i64() { async fn test_nested_sum_i64() { insta::assert_snapshot!(QueryFixture::new("{ sum: sum(sum(Numbers.m))}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sum - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,27 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,49 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,83 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,117 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,27 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,49 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,83 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,117 "###); } @@ -97,12 +97,12 @@ async fn test_nested_sum_i64() { async fn test_sum_i64() { insta::assert_snapshot!(QueryFixture::new("{ sum: sum(Numbers.m)}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sum - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,22 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,22 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,34 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,34 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,22 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,22 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,34 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,34 "###); } @@ -110,12 +110,12 @@ async fn test_sum_i64() { async fn test_sum_f64() { insta::assert_snapshot!(QueryFixture::new("{ sum: sum(Numbers.m)}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sum - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,22.8 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,22.8 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,35.2 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,35.2 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,22.8 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,22.8 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,35.2 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,35.2 "###); } @@ -123,12 +123,12 @@ async fn test_sum_f64() { async fn test_mean_i64() { insta::assert_snapshot!(QueryFixture::new("{ mean: mean(Numbers.m)}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,mean - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.0 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,11.0 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,11.0 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,11.333333333333334 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,11.333333333333334 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.0 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,11.0 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,11.0 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,11.333333333333334 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,11.333333333333334 "###); } @@ -136,12 +136,12 @@ async fn test_mean_i64() { async fn test_mean_f64() { insta::assert_snapshot!(QueryFixture::new("{ mean: mean(Numbers.m)}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,mean - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,11.400000000000002 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,11.400000000000002 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,11.733333333333334 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,11.733333333333334 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,11.400000000000002 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,11.400000000000002 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,11.733333333333334 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,11.733333333333334 "###); } @@ -149,12 +149,12 @@ async fn test_mean_f64() { async fn test_variance_i64() { insta::assert_snapshot!(QueryFixture::new("{ variance: variance(Numbers.m)}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,variance - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,36.0 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,36.0 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,24.222222222222225 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,24.222222222222225 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,36.0 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,36.0 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,24.222222222222225 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,24.222222222222225 "###); } @@ -162,12 +162,12 @@ async fn test_variance_i64() { async fn test_variance_f64() { insta::assert_snapshot!(QueryFixture::new("{ variance: variance(Numbers.m)}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,variance - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,38.440000000000005 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,38.440000000000005 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,25.848888888888894 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,25.848888888888894 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,38.440000000000005 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,38.440000000000005 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,25.848888888888894 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,25.848888888888894 "###); } @@ -175,12 +175,12 @@ async fn test_variance_f64() { async fn test_stddev_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, stddev: stddev(Numbers.m)}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,stddev - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6.0 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,6.0 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,4.921607686744467 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,4.921607686744467 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6.0 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,6.0 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,4.921607686744467 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,4.921607686744467 "###); } @@ -188,12 +188,12 @@ async fn test_stddev_i64() { async fn test_stddev_f64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, stddev: stddev(Numbers.m)}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,stddev - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,6.2 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,5.0841802573166985 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,5.0841802573166985 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,6.2 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,5.0841802573166985 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,5.0841802573166985 "###); } @@ -201,12 +201,12 @@ async fn test_stddev_f64() { async fn test_min_f64() { insta::assert_snapshot!(QueryFixture::new("{ min: min(Numbers.m)}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,min - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,5.2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,5.2 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,5.2 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,5.2 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,5.2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,5.2 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,5.2 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,5.2 "###); } @@ -214,12 +214,12 @@ async fn test_min_f64() { async fn test_min_i64() { insta::assert_snapshot!(QueryFixture::new("{ min: min(Numbers.m)}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,min - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,5 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,5 "###); } @@ -227,12 +227,12 @@ async fn test_min_i64() { async fn test_min_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ min: min(Times.n)}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,min - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,2 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,4 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,4 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,4 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,4 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,2 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,4 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,4 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,4 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,4 "###); } @@ -240,12 +240,12 @@ async fn test_min_timestamp_ns() { async fn test_max_f64() { insta::assert_snapshot!(QueryFixture::new("{ max: max(Numbers.m)}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,max - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,17.6 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,17.6 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,17.6 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,17.6 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,17.6 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,17.6 "###); } @@ -253,12 +253,12 @@ async fn test_max_f64() { async fn test_max_i64() { insta::assert_snapshot!(QueryFixture::new("{ max: max(Numbers.m)}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,max - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,17 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,17 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,17 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,17 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,17 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,17 "###); } @@ -266,12 +266,12 @@ async fn test_max_i64() { async fn test_max_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ max: max(Times.n)}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,max - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,2 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,2 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,5 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,5 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23 "###); } @@ -279,12 +279,12 @@ async fn test_max_timestamp_ns() { async fn test_count_i64() { insta::assert_snapshot!(QueryFixture::new("{ count: count(Numbers.m)}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,count - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,2 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,3 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,3 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,1 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,2 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,3 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,3 "###); } @@ -292,12 +292,12 @@ async fn test_count_i64() { async fn test_count_f64() { insta::assert_snapshot!(QueryFixture::new("{ count: count(Numbers.m)}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,count - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,2 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,3 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,3 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,1 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,2 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,3 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,3 "###); } @@ -305,12 +305,12 @@ async fn test_count_f64() { async fn test_count_record() { insta::assert_snapshot!(QueryFixture::new("{ count: count(Numbers)}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,count - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,3 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,4 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,5 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,1 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,3 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,4 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,5 "###); } @@ -318,12 +318,12 @@ async fn test_count_record() { async fn test_count_string() { insta::assert_snapshot!(QueryFixture::new("{ count: count(Strings.s)}").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,count - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1 - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1 - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,2 - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,3 - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,4 - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,5 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1 + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,1 + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,2 + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,3 + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,4 + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,5 "###); } @@ -331,13 +331,13 @@ async fn test_count_string() { async fn test_count_boolean() { insta::assert_snapshot!(QueryFixture::new("{ count: count(Booleans.a)}").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,count - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1 - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1 - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1 - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,2 - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,3 - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,4 - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,4 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1 + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,1 + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,1 + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,2 + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,3 + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,4 + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,4 "###); } @@ -345,12 +345,12 @@ async fn test_count_boolean() { async fn test_count_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ count: count(Times.n)}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,count - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,2 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,2 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,3 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,4 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,2 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,2 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,3 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,4 "###); } @@ -358,12 +358,12 @@ async fn test_count_timestamp_ns() { async fn test_count_if_condition() { insta::assert_snapshot!(QueryFixture::new("{ count_if: count_if(Numbers.m > 10)}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,count_if - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,1 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,1 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,2 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,2 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,1 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,1 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,1 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,2 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,2 "###); } @@ -371,12 +371,12 @@ async fn test_count_if_condition() { async fn test_first_i64() { insta::assert_snapshot!(QueryFixture::new("{ first: first(Numbers.m)}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,first - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,5 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,5 "###); } @@ -384,12 +384,12 @@ async fn test_first_i64() { async fn test_first_f64() { insta::assert_snapshot!(QueryFixture::new("{ first: first(Numbers.m)}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,first - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,5.2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,5.2 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,5.2 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,5.2 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,5.2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,5.2 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,5.2 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,5.2 "###); } @@ -397,12 +397,12 @@ async fn test_first_f64() { async fn test_first_record() { insta::assert_snapshot!(QueryFixture::new("{ first: Numbers | first() | $input.m }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,first - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,5 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,5 "###); } @@ -410,12 +410,12 @@ async fn test_first_record() { async fn test_first_string() { insta::assert_snapshot!(QueryFixture::new("{ first: first(Strings.s)}").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,first - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,World - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,World - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,World - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,World + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,World + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,World + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,World + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,World "###); } @@ -423,13 +423,13 @@ async fn test_first_string() { async fn test_first_boolean() { insta::assert_snapshot!(QueryFixture::new("{ first: first(Booleans.a) }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,first - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,false + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,false + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,false + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,false "###); } @@ -437,12 +437,12 @@ async fn test_first_boolean() { async fn test_first_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ first: first(Times.n)}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,first - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,2 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,4 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,4 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,4 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,4 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,2 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,4 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,4 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,4 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,4 "###); } @@ -464,12 +464,12 @@ async fn test_last_sliding_i64() { async fn test_first_since_i64() { insta::assert_snapshot!(QueryFixture::new("{ first: first(Numbers.m, window=since(daily())) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,first - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,5 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,5 "###); } @@ -477,12 +477,12 @@ async fn test_first_since_i64() { async fn test_last_i64() { insta::assert_snapshot!(QueryFixture::new("{ last: last(Numbers.m)}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,last - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,17 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,12 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,17 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,12 "###); } @@ -514,12 +514,12 @@ async fn test_last_i64_record_finished() { async fn test_last_f64() { insta::assert_snapshot!(QueryFixture::new("{ last: last(Numbers.m)}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,last - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,17.6 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,12.4 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,17.6 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,12.4 "###); } @@ -527,12 +527,12 @@ async fn test_last_f64() { async fn test_last_record() { insta::assert_snapshot!(QueryFixture::new("{ last: Numbers | last() | $input.m }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,last - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A, "###); } @@ -540,12 +540,12 @@ async fn test_last_record() { async fn test_last_string() { insta::assert_snapshot!(QueryFixture::new("{ last: last(Strings.s)}").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,last - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye "###); } @@ -562,13 +562,13 @@ async fn test_last_string_finished() { async fn test_last_boolean() { insta::assert_snapshot!(QueryFixture::new("{ last: last(Booleans.a)}").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,last - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,false + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,false + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,false "###); } @@ -576,12 +576,12 @@ async fn test_last_boolean() { async fn test_last_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ last: last(Times.n)}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,last - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,2 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,2 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,5 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,5 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23 "###); } @@ -589,12 +589,12 @@ async fn test_last_timestamp_ns() { async fn test_count_constant() { insta::assert_snapshot!(QueryFixture::new("{ time: Times.time, agg: count(0) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,agg - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000,0 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000,0 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000,0 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000,0 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000,0 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000,0 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000,0 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000,0 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000,0 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000,0 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000,0 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000,0 "###); } @@ -602,12 +602,12 @@ async fn test_count_constant() { async fn test_count_if_constant() { insta::assert_snapshot!(QueryFixture::new("{ time: Times.time, agg: count_if(false) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,agg - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000,0 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000,0 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000,0 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000,0 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000,0 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000,0 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000,0 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000,0 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000,0 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000,0 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000,0 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000,0 "###); } @@ -615,12 +615,12 @@ async fn test_count_if_constant() { async fn test_first_constant() { insta::assert_snapshot!(QueryFixture::new("{ time: Times.time, agg: first(2) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,agg - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000, "###); } @@ -628,12 +628,12 @@ async fn test_first_constant() { async fn test_lag_constant() { insta::assert_snapshot!(QueryFixture::new("{ time: Times.time, agg: lag(2, 2) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,agg - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000, "###); } @@ -641,12 +641,12 @@ async fn test_lag_constant() { async fn test_last_constant() { insta::assert_snapshot!(QueryFixture::new("{ time: Times.time, agg: last(2) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,agg - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000, "###); } @@ -654,12 +654,12 @@ async fn test_last_constant() { async fn test_max_constant() { insta::assert_snapshot!(QueryFixture::new("{ time: Times.time, agg: max(2) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,agg - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000, "###); } @@ -667,12 +667,12 @@ async fn test_max_constant() { async fn test_mean_constant() { insta::assert_snapshot!(QueryFixture::new("{ time: Times.time, agg: mean(2) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,agg - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000, "###); } @@ -680,12 +680,12 @@ async fn test_mean_constant() { async fn test_min_constant() { insta::assert_snapshot!(QueryFixture::new("{ time: Times.time, agg: min(2) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,agg - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000, "###); } @@ -693,12 +693,12 @@ async fn test_min_constant() { async fn test_min_stddev() { insta::assert_snapshot!(QueryFixture::new("{ time: Times.time, agg: stddev(2) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,agg - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000, "###); } @@ -706,12 +706,12 @@ async fn test_min_stddev() { async fn test_min_sum() { insta::assert_snapshot!(QueryFixture::new("{ time: Times.time, agg: sum(2) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,agg - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000, "###); } @@ -719,12 +719,12 @@ async fn test_min_sum() { async fn test_min_variance() { insta::assert_snapshot!(QueryFixture::new("{ time: Times.time, agg: variance(2) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,agg - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000, "###); } @@ -732,12 +732,12 @@ async fn test_min_variance() { async fn test_first_sum_constant() { insta::assert_snapshot!(QueryFixture::new("{ time: Times.time, agg: first(sum(1)) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,agg - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000, "###); } @@ -745,12 +745,12 @@ async fn test_first_sum_constant() { async fn test_add_sum_constants() { insta::assert_snapshot!(QueryFixture::new("{ time: Times.time, agg: sum(1) + sum(2) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,agg - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000, "###); } @@ -758,11 +758,11 @@ async fn test_add_sum_constants() { async fn test_literal_agg() { insta::assert_snapshot!(QueryFixture::new("{ agg_literal: sum(5), max_output: max(Numbers.m) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,agg_literal,max_output - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,,17 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,17 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,,17 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,17 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,,17 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,17 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,,17 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,17 "###); } diff --git a/crates/sparrow-main/tests/e2e/cast_tests.rs b/crates/sparrow-main/tests/e2e/cast_tests.rs index 38ad218f3..d16869e5f 100644 --- a/crates/sparrow-main/tests/e2e/cast_tests.rs +++ b/crates/sparrow-main/tests/e2e/cast_tests.rs @@ -39,13 +39,13 @@ async fn cast_data_fixture() -> DataFixture { async fn test_implicit_cast_i64_to_f64_add() { insta::assert_snapshot!(QueryFixture::new("{ i64_field: Input.i64, f64_field: Input.f64, add: Input.i64 + Input.f64 }").run_to_csv(&cast_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,i64_field,f64_field,add - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,50,21.4,71.4 - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,,1.22, - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,25,0.0,25.0 - 1997-12-20T00:39:59.000000000,9223372036854775810,11832085162654999889,0,35,0.0,35.0 - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,25,2.2,27.2 - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,12,, - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,,, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,50,21.4,71.4 + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,,1.22, + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,25,0.0,25.0 + 1997-12-20T00:39:59.000000000,2,11832085162654999889,0,35,0.0,35.0 + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,25,2.2,27.2 + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,12,, + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,,, "###); } @@ -53,13 +53,13 @@ async fn test_implicit_cast_i64_to_f64_add() { async fn test_implicit_cast_i64_to_f64_powf() { insta::assert_snapshot!(QueryFixture::new("{ i64_field: Input.i64, f64_field: Input.f64, powf: powf(Input.i64, Input.f64) }").run_to_csv(&cast_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,i64_field,f64_field,powf - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,50,21.4,2.280122041201667e36 - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,,1.22, - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,25,0.0,1.0 - 1997-12-20T00:39:59.000000000,9223372036854775810,11832085162654999889,0,35,0.0,1.0 - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,25,2.2,1189.7837116974247 - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,12,, - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,,, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,50,21.4,2.280122041201667e36 + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,,1.22, + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,25,0.0,1.0 + 1997-12-20T00:39:59.000000000,2,11832085162654999889,0,35,0.0,1.0 + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,25,2.2,1189.7837116974247 + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,12,, + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,,, "###); } @@ -67,13 +67,13 @@ async fn test_implicit_cast_i64_to_f64_powf() { async fn test_implicit_cast_i64_to_f64_literal() { insta::assert_snapshot!(QueryFixture::new("{ i64_field: Input.i64, add: Input.i64 + 1.11 }").run_to_csv(&cast_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,i64_field,add - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,50,51.11 - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,, - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,25,26.11 - 1997-12-20T00:39:59.000000000,9223372036854775810,11832085162654999889,0,35,36.11 - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,25,26.11 - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,12,13.11 - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,50,51.11 + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,, + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,25,26.11 + 1997-12-20T00:39:59.000000000,2,11832085162654999889,0,35,36.11 + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,25,26.11 + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,12,13.11 + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,, "###); } @@ -81,13 +81,13 @@ async fn test_implicit_cast_i64_to_f64_literal() { async fn test_string_as_i64() { insta::assert_snapshot!(QueryFixture::new("{ number_string: Input.number_string, number_string_as_i64: Input.number_string as i64 }").run_to_csv(&cast_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,number_string,number_string_as_i64 - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,65,65 - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,hello, - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,73,73 - 1997-12-20T00:39:59.000000000,9223372036854775810,11832085162654999889,0,73,73 - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,82,82 - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,18,18 - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,65,65 + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,hello, + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,73,73 + 1997-12-20T00:39:59.000000000,2,11832085162654999889,0,73,73 + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,82,82 + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,18,18 + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,, "###); } @@ -95,13 +95,13 @@ async fn test_string_as_i64() { async fn test_i64_as_i32() { insta::assert_snapshot!(QueryFixture::new("{ i64: Input.i64, i64_as_i32: Input.i64 as i32 }").run_to_csv(&cast_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,i64,i64_as_i32 - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,50,50 - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,, - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,25,25 - 1997-12-20T00:39:59.000000000,9223372036854775810,11832085162654999889,0,35,35 - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,25,25 - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,12,12 - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,50,50 + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,, + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,25,25 + 1997-12-20T00:39:59.000000000,2,11832085162654999889,0,35,35 + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,25,25 + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,12,12 + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,, "###); } @@ -109,13 +109,13 @@ async fn test_i64_as_i32() { async fn test_f64_as_i64() { insta::assert_snapshot!(QueryFixture::new("{ f64: Input.f64, f64_as_i64: Input.f64 as i64 }").run_to_csv(&cast_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f64,f64_as_i64 - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,21.4,21 - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,1.22,1 - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,0.0,0 - 1997-12-20T00:39:59.000000000,9223372036854775810,11832085162654999889,0,0.0,0 - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,2.2,2 - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,, - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,21.4,21 + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,1.22,1 + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,0.0,0 + 1997-12-20T00:39:59.000000000,2,11832085162654999889,0,0.0,0 + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,2.2,2 + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,, + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,, "###); } @@ -123,13 +123,13 @@ async fn test_f64_as_i64() { async fn test_i64_as_string() { insta::assert_snapshot!(QueryFixture::new("{ i64: Input.i64, i64_as_string: Input.i64 as string }").run_to_csv(&cast_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,i64,i64_as_string - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,50,50 - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,, - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,25,25 - 1997-12-20T00:39:59.000000000,9223372036854775810,11832085162654999889,0,35,35 - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,25,25 - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,12,12 - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,50,50 + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,, + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,25,25 + 1997-12-20T00:39:59.000000000,2,11832085162654999889,0,35,35 + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,25,25 + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,12,12 + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,, "###); } @@ -139,13 +139,13 @@ async fn test_null_literal_as_string() { // are literals. insta::assert_snapshot!(QueryFixture::new("{ i64: Input.i64, null_: null, null_as_string: null as string }").run_to_csv(&cast_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,i64,null_,null_as_string - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,50,, - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,,, - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,25,, - 1997-12-20T00:39:59.000000000,9223372036854775810,11832085162654999889,0,35,, - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,25,, - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,12,, - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,,, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,50,, + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,,, + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,25,, + 1997-12-20T00:39:59.000000000,2,11832085162654999889,0,35,, + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,25,, + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,12,, + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,,, "###); } @@ -155,13 +155,13 @@ async fn test_seconds_between_as_i64() { "let duration_s = seconds_between(Input.order_time, Input.time) in { duration_s_as_i64: duration_s as i64 }").run_to_csv(&cast_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,duration_s_as_i64 - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,-283996800 - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,-126230400 - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,-126230399 - 1997-12-20T00:39:59.000000000,9223372036854775810,11832085162654999889,0,-126230398 - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,-157766400 - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,-156208802 - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,-283996800 + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,-126230400 + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,-126230399 + 1997-12-20T00:39:59.000000000,2,11832085162654999889,0,-126230398 + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,-157766400 + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,-156208802 + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0, "###); } @@ -169,13 +169,13 @@ async fn test_seconds_between_as_i64() { async fn test_days_between_as_i32() { insta::assert_snapshot!(QueryFixture::new("{ i64: Input.i64, interval_days_as_i64: days(Input.i64) as i32 }").run_to_csv(&cast_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,i64,interval_days_as_i64 - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,50,50 - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,, - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,25,25 - 1997-12-20T00:39:59.000000000,9223372036854775810,11832085162654999889,0,35,35 - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,25,25 - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,12,12 - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,50,50 + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,, + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,25,25 + 1997-12-20T00:39:59.000000000,2,11832085162654999889,0,35,35 + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,25,25 + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,12,12 + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,, "###); } @@ -183,13 +183,13 @@ async fn test_days_between_as_i32() { async fn test_months_between_as_i32() { insta::assert_snapshot!(QueryFixture::new("{ i64: Input.i64, interval_months_as_i64: months(Input.i64) as i32 }").run_to_csv(&cast_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,i64,interval_months_as_i64 - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,50,50 - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,, - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,25,25 - 1997-12-20T00:39:59.000000000,9223372036854775810,11832085162654999889,0,35,35 - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,25,25 - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,12,12 - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,50,50 + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,, + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,25,25 + 1997-12-20T00:39:59.000000000,2,11832085162654999889,0,35,35 + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,25,25 + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,12,12 + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,, "###); } @@ -199,11 +199,11 @@ async fn test_bool_as_i64() { let n = Numbers.n + 11 in { m, n, eq: (m == n) as i64 }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,21,0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,14,0 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,17,1 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,20, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,21,0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,14,0 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,17,1 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,20, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } diff --git a/crates/sparrow-main/tests/e2e/coalesce_tests.rs b/crates/sparrow-main/tests/e2e/coalesce_tests.rs index d6c93a3b3..69933d761 100644 --- a/crates/sparrow-main/tests/e2e/coalesce_tests.rs +++ b/crates/sparrow-main/tests/e2e/coalesce_tests.rs @@ -8,13 +8,13 @@ use crate::QueryFixture; async fn test_coalesce_two_boolean() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, b: Booleans.b, coalesce_a_b: Booleans.a | coalesce($input, Booleans.b) }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,b,coalesce_a_b - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,true,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,false,false - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,true,true - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,false,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,true,false - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false,,false - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,true,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false,false,false + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,,true,true + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true,false,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false,true,false + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false,,false + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,,, "###); } @@ -45,12 +45,12 @@ async fn test_coalesce_zero() { async fn test_coalesce_one_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, coalesce_m: coalesce(Numbers.m) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,coalesce_m - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,17 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,12 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,17 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,12 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -58,12 +58,12 @@ async fn test_coalesce_one_i64() { async fn test_coalesce_one_i64_one_literal_i64() { insta::assert_snapshot!(QueryFixture::new("{ n: Times.n, coalesce_n_literal: coalesce(Times.n, 12345) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,coalesce_n_literal - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,2,2 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4,4 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5,5 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,12345 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23,23 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,2,2 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4,4 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,5,5 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,12345 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23,23 "###); } @@ -71,12 +71,12 @@ async fn test_coalesce_one_i64_one_literal_i64() { async fn test_coalesce_one_i64_one_literal_f64() { insta::assert_snapshot!(QueryFixture::new("{ n: Times.n, coalesce_n_literal: coalesce(Times.n, 12345.7) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,coalesce_n_literal - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,2,2.0 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4,4.0 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5,5.0 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,12345.7 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8.0 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23,23.0 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,2,2.0 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4,4.0 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,5,5.0 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,12345.7 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8.0 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23,23.0 "###); } @@ -84,12 +84,12 @@ async fn test_coalesce_one_i64_one_literal_f64() { async fn test_coalesce_two_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, coalesce_m_n: coalesce(Numbers.m, Numbers.n) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,coalesce_m_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,17 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9,9 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,,12 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,17 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9,9 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,,12 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -97,12 +97,12 @@ async fn test_coalesce_two_i64() { async fn test_coalesce_two_i64_one_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, coalesce_m_n: coalesce(Numbers.m, Numbers.n, 42) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,coalesce_m_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,17 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9,9 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,,12 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,42 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,17 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9,9 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,,12 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,42 "###); } @@ -110,12 +110,12 @@ async fn test_coalesce_two_i64_one_literal() { async fn test_coalesce_two_f64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, coalesce_m_n: coalesce(Numbers.m, Numbers.n) }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,coalesce_m_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,5.2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,24.3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,17.6 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25,9.25 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,,12.4 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,5.2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,24.3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,17.6 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25,9.25 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,,12.4 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -123,12 +123,12 @@ async fn test_coalesce_two_f64() { async fn test_coalesce_two_string() { insta::assert_snapshot!(QueryFixture::new("{ s: Strings.s, t: Strings.t, coalesce_s_t: coalesce(Strings.s, Strings.t) }").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,t,coalesce_s_t - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hEllo,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World,world,World - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world,hello world,hello world - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,greetings, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,,salutations, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye,,goodbye + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo,hEllo,hEllo + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World,world,World + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world,hello world,hello world + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,,greetings, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,,salutations, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye,,goodbye "###); } @@ -136,12 +136,12 @@ async fn test_coalesce_two_string() { async fn test_coalesce_two_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ m: Times.m, n: Times.n, coalesce_m_n: coalesce(Times.m, Times.n) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,coalesce_m_n - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,4,2,4 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,3,4,3 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,5,5 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8,8 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,11,23,11 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,4,2,4 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,3,4,3 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,5,5 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8,8 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,11,23,11 "###); } @@ -149,12 +149,12 @@ async fn test_coalesce_two_timestamp_ns() { async fn test_coalesce_two_record() { insta::assert_snapshot!(QueryFixture::new("{ m: Times.m, n: Times.n, coalesce_times_times: coalesce(Times, Times) | $input.n }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,coalesce_times_times - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,4,2,2 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,3,4,4 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,5,5 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8,8 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,11,23,23 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,4,2,2 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,3,4,4 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,5,5 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8,8 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,11,23,23 "###); } diff --git a/crates/sparrow-main/tests/e2e/collect_tests.rs b/crates/sparrow-main/tests/e2e/collect_tests.rs index 7b148a509..15811cee6 100644 --- a/crates/sparrow-main/tests/e2e/collect_tests.rs +++ b/crates/sparrow-main/tests/e2e/collect_tests.rs @@ -45,22 +45,22 @@ pub(crate) async fn collect_data_fixture() -> DataFixture { async fn test_collect_with_null_max() { insta::assert_snapshot!(QueryFixture::new("{ f1: Collect.n | collect(max = null) | index(0), f2: Collect.b | collect(max = null) | index(0), f3: Collect.s | collect(max = null) | index(0) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f1,f2,f3 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,0,true,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,0,true,hEllo - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A,0,true,hEllo - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,0,true,hEllo - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,0,true,hEllo - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,0,true,hEllo - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,5,false,h - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5,false,h - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B,5,false,h - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,5,false,h - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,5,false,h - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,5,false,h - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C,1,true,g - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,1,true,g - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C,1,true,g - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,1,true,g + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,0,true,hEllo + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,0,true,hEllo + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A,0,true,hEllo + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,0,true,hEllo + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,0,true,hEllo + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,0,true,hEllo + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,5,false,h + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B,5,false,h + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B,5,false,h + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,5,false,h + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,5,false,h + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,5,false,h + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C,1,true,g + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,1,true,g + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C,1,true,g + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,1,true,g "###); } @@ -68,22 +68,22 @@ async fn test_collect_with_null_max() { async fn test_collect_to_list_i64() { insta::assert_snapshot!(QueryFixture::new("{ f1: Collect.n | collect(max=10) | index(0) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f1 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,0 - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,0 - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A,0 - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,0 - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,0 - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,0 - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C,1 - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,1 - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C,1 - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,1 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,0 + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,0 + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A,0 + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,0 + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,0 + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,0 + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,5 + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B,5 + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B,5 + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,5 + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,5 + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,5 + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C,1 + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,1 + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C,1 + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,1 "###); } @@ -91,22 +91,22 @@ async fn test_collect_to_list_i64() { async fn test_collect_to_list_i64_dynamic() { insta::assert_snapshot!(QueryFixture::new("{ f1: Collect.n | collect(max=10) | index(Collect.index) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f1 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,0 - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,2 - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A,9 - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,9 - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,2 - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B,-2 - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,-2 - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,-2 - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C, - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,1 - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C, - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,2 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,0 + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,2 + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A,9 + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,9 + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,2 + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A, + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,5 + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B,-2 + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B, + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,-2 + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,-2 + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,5 + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C, + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,1 + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C, + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,2 "###); } @@ -114,22 +114,22 @@ async fn test_collect_to_list_i64_dynamic() { async fn test_collect_to_small_list_i64() { insta::assert_snapshot!(QueryFixture::new("{ f1: Collect.n | collect(max=2) | index(Collect.index) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f1 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,0 - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,2 - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,-1 - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B,-2 - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,2 - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2 - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,2 - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C, - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,1 - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C, - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,4 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,0 + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,2 + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A, + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,-1 + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A, + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,5 + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B,-2 + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B, + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,2 + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,2 + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,2 + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C, + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,1 + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C, + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,4 "###); } @@ -137,22 +137,22 @@ async fn test_collect_to_small_list_i64() { async fn test_collect_to_list_string() { insta::assert_snapshot!(QueryFixture::new("{ f1: Collect.s | collect(max=10) | index(0) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f1 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,hEllo - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A,hEllo - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,hEllo - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,hEllo - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,hEllo - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,h - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B,h - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B,h - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,h - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,h - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,h - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C,g - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,g - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C,g - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,g + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,hEllo + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A,hEllo + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,hEllo + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,hEllo + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,hEllo + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,h + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B,h + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B,h + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,h + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,h + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,h + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C,g + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,g + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C,g + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,g "###); } @@ -160,22 +160,22 @@ async fn test_collect_to_list_string() { async fn test_collect_to_list_string_dynamic() { insta::assert_snapshot!(QueryFixture::new("{ f1: Collect.s | collect(max=10) | index(Collect.index) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f1 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,hi - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A,hey - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,hey - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,hi - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,h - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B,he - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,he - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,he - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,h - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C, - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,g - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C, - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,go + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,hi + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A,hey + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,hey + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,hi + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A, + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,h + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B,he + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B, + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,he + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,he + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,h + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C, + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,g + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C, + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,go "###); } @@ -183,22 +183,22 @@ async fn test_collect_to_list_string_dynamic() { async fn test_collect_to_small_list_string() { insta::assert_snapshot!(QueryFixture::new("{ f1: Collect.s | collect(max=2) | index(Collect.index) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f1 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,hi - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,ay - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,h - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B,he - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,hel - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C, - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,g - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C, - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,good + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,hi + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A, + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,ay + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A, + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,h + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B,he + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B, + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,hel + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B, + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B, + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C, + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,g + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C, + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,good "###); } @@ -206,22 +206,22 @@ async fn test_collect_to_small_list_string() { async fn test_collect_to_list_boolean() { insta::assert_snapshot!(QueryFixture::new("{ f1: Collect.b | collect(max=10) | index(0) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f1 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,true - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A,true - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,true - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,true - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,true - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C,true - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,true - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C,true - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,true + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,true + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A,true + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,true + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,true + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,true + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,false + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B,false + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B,false + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,false + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,false + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,false + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C,true + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,true + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C,true + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,true "###); } @@ -229,22 +229,22 @@ async fn test_collect_to_list_boolean() { async fn test_collect_to_list_boolean_dynamic() { insta::assert_snapshot!(QueryFixture::new("{ f1: Collect.b | collect(max=10) | index(Collect.index) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f1 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,false - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,false - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,false - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,true - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,true - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C, - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,true - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C, - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,true + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,false + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A, + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,false + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,false + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A, + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,false + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B, + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B, + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,true + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,true + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,false + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C, + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,true + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C, + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,true "###); } @@ -252,22 +252,22 @@ async fn test_collect_to_list_boolean_dynamic() { async fn test_collect_to_small_list_boolean() { insta::assert_snapshot!(QueryFixture::new("{ f1: Collect.b | collect(max=2) | index(Collect.index) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f1 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,false - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,true - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,true - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,true - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C, - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,true - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C, - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,true + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,false + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A, + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,true + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A, + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,false + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B, + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B, + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,false + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,true + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,true + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C, + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,true + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C, + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,true "###); } @@ -282,22 +282,22 @@ async fn test_collect_structs() { } ").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s0,s1,s2,s3,s4 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,,,, - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hi,,, - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hi,hey,, - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,hEllo,hi,hey,heylo, - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hi,hey,heylo,ay - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hi,hey,heylo,ay - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,h,,,, - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B,h,he,,, - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B,h,he,,, - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,h,he,,hel, - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,h,he,,hel, - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,h,he,,hel, - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C,g,,,, - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,g,go,,, - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C,g,go,goo,, - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,g,go,goo,good, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo,,,, + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,hEllo,hi,,, + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A,hEllo,hi,hey,, + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,hEllo,hi,hey,heylo, + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,hEllo,hi,hey,heylo,ay + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,hEllo,hi,hey,heylo,ay + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,h,,,, + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B,h,he,,, + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B,h,he,,, + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,h,he,,hel, + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,h,he,,hel, + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,h,he,,hel, + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C,g,,,, + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,g,go,,, + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C,g,go,goo,, + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,g,go,goo,good, "###); } @@ -309,22 +309,22 @@ async fn test_collect_with_minimum() { min2: Collect.s | collect(min=3, max=10) | index(0) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,min0,min1,min2 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,, - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hEllo, - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hEllo,hEllo - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,hEllo,hEllo,hEllo - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hEllo,hEllo - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hEllo,hEllo - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,h,, - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B,h,h, - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B,h,h,h - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,h,h,h - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,h,h,h - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,h,h,h - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C,g,, - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,g,g, - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C,g,g,g - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,g,g,g + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo,, + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,hEllo,hEllo, + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A,hEllo,hEllo,hEllo + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,hEllo,hEllo,hEllo + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,hEllo,hEllo,hEllo + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,hEllo,hEllo,hEllo + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,h,, + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B,h,h, + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B,h,h,h + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,h,h,h + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,h,h,h + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,h,h,h + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C,g,, + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,g,g, + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C,g,g,g + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,g,g,g "###); } @@ -335,22 +335,22 @@ async fn test_collect_structs_map() { in { x: x | index(0), y: x | index(1) } ").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,x,y - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo, - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hi - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hi - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,hEllo,hi - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hi - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hi - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,h, - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B,h,he - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B,h,he - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,h,he - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,h,he - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,h,he - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C,g, - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,g,go - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C,g,go - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,g,go + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo, + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,hEllo,hi + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A,hEllo,hi + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,hEllo,hi + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,hEllo,hi + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,hEllo,hi + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,h, + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B,h,he + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B,h,he + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,h,he + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,h,he + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,h,he + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C,g, + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,g,go + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C,g,go + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,g,go "###); } @@ -373,22 +373,22 @@ async fn test_collect_lag_equality() { lag: Collect.n | lag(2) }").with_dump_dot("asdf").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,collect,lag - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A,0,0 - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,2,2 - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,9,9 - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,-7,-7 - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,5,5 - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,5,5 - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,-2,-2 - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C,, - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,, - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C,1,1 - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,2,2 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,, + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A,0,0 + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,2,2 + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,9,9 + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,-7,-7 + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,, + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B,, + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B,, + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,5,5 + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,5,5 + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,-2,-2 + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C,, + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,, + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C,1,1 + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,2,2 "###); } @@ -396,32 +396,32 @@ async fn test_collect_lag_equality() { async fn test_collect_primitive_since_minutely() { insta::assert_snapshot!(QueryFixture::new("{ f1: Collect.n | collect(max=10, window=since(minutely())) | index(0) | when(is_valid($input))}").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f1 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,0 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,0 1996-12-20T00:40:00.000000000,18446744073709551615,12960666915911099378,A,0 - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,2 + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,2 1996-12-20T00:41:00.000000000,18446744073709551615,12960666915911099378,A,2 - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A,9 - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,9 + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A,9 + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,9 1996-12-20T00:42:00.000000000,18446744073709551615,12960666915911099378,A,9 - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,-1 + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,-1 1996-12-20T00:43:00.000000000,18446744073709551615,12960666915911099378,A,-1 - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,10 + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,10 1996-12-20T00:44:00.000000000,18446744073709551615,12960666915911099378,A,10 - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,5 + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,5 1996-12-21T00:41:00.000000000,18446744073709551615,2867199309159137213,B,5 - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B,-2 + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B,-2 1996-12-21T00:42:00.000000000,18446744073709551615,2867199309159137213,B,-2 - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,2 + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,2 1996-12-21T00:44:00.000000000,18446744073709551615,2867199309159137213,B,2 - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,5 + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,5 1996-12-21T01:45:00.000000000,18446744073709551615,2867199309159137213,B,5 - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C,1 + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C,1 1996-12-22T00:45:00.000000000,18446744073709551615,2521269998124177631,C,1 - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,2 + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,2 1996-12-22T00:46:00.000000000,18446744073709551615,2521269998124177631,C,2 - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C,3 + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C,3 1996-12-22T00:47:00.000000000,18446744073709551615,2521269998124177631,C,3 - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,4 + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,4 "###); } @@ -434,7 +434,7 @@ async fn test_collect_primitive_since_minutely_1() { f1_with_min: Collect.n | collect(min=3, max=10, window=since(minutely())) | index(1) | when(is_valid($input)) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f1,f1_with_min - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,-7, + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,-7, 1996-12-20T00:42:00.000000000,18446744073709551615,12960666915911099378,A,-7, "###); } @@ -444,17 +444,17 @@ async fn test_collect_string_since_hourly() { // note that `B` is empty because we collect `null` as a valid value in a list currently insta::assert_snapshot!(QueryFixture::new("{ f1: Collect.s | collect(max=10, window=since(hourly())) | index(2) | when(is_valid($input)) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f1 - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A,hey - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,hey - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,hey - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,hey + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A,hey + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,hey + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,hey + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,hey 1996-12-20T01:00:00.000000000,18446744073709551615,12960666915911099378,A,hey - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B, + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B, + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B, + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B, 1996-12-21T01:00:00.000000000,18446744073709551615,2867199309159137213,B, - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C,goo - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,goo + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C,goo + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,goo "###); } @@ -462,12 +462,12 @@ async fn test_collect_string_since_hourly() { async fn test_collect_boolean_since_hourly() { insta::assert_snapshot!(QueryFixture::new("{ f1: Collect.b | collect(max=10, window=since(hourly())) | index(3) | when(is_valid($input)) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,f1 - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,true - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,true + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,true + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,true 1996-12-20T01:00:00.000000000,18446744073709551615,12960666915911099378,A,true - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,true + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,true 1996-12-21T01:00:00.000000000,18446744073709551615,2867199309159137213,B,true - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,true + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,true "###); } @@ -486,25 +486,25 @@ async fn test_collect_struct_since_hourly() { f4: ({b: Collect.b} | collect(max=10, window=since(hourly())) | index(4)).b | when(is_valid($input)) }").run_to_csv(&collect_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,b,f0,f1,f2,f3,f4 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,true,,,, - 1996-12-20T00:40:57.000000000,9223372036854775808,12960666915911099378,A,false,true,false,,, - 1996-12-20T00:41:57.000000000,9223372036854775808,12960666915911099378,A,,true,false,,, - 1996-12-20T00:42:00.000000000,9223372036854775808,12960666915911099378,A,false,true,false,,false, - 1996-12-20T00:42:57.000000000,9223372036854775808,12960666915911099378,A,true,true,false,,false,true - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,true,true,false,,false,true + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,true,,,, + 1996-12-20T00:40:57.000000000,0,12960666915911099378,A,false,true,false,,, + 1996-12-20T00:41:57.000000000,0,12960666915911099378,A,,true,false,,, + 1996-12-20T00:42:00.000000000,0,12960666915911099378,A,false,true,false,,false, + 1996-12-20T00:42:57.000000000,0,12960666915911099378,A,true,true,false,,false,true + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,true,true,false,,false,true 1996-12-20T01:00:00.000000000,18446744073709551615,12960666915911099378,A,,true,false,,false,true - 1996-12-21T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,false,,,, - 1996-12-21T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,false,,,, - 1996-12-21T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,false,,true,, - 1996-12-21T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,false,,true,false, - 1996-12-21T00:44:57.000000000,9223372036854775808,2867199309159137213,B,true,false,,true,false,true + 1996-12-21T00:40:57.000000000,0,2867199309159137213,B,false,false,,,, + 1996-12-21T00:41:57.000000000,0,2867199309159137213,B,,false,,,, + 1996-12-21T00:42:57.000000000,0,2867199309159137213,B,true,false,,true,, + 1996-12-21T00:43:57.000000000,0,2867199309159137213,B,false,false,,true,false, + 1996-12-21T00:44:57.000000000,0,2867199309159137213,B,true,false,,true,false,true 1996-12-21T01:00:00.000000000,18446744073709551615,2867199309159137213,B,,false,,true,false,true - 1996-12-21T01:44:57.000000000,9223372036854775808,2867199309159137213,B,true,true,,,, + 1996-12-21T01:44:57.000000000,0,2867199309159137213,B,true,true,,,, 1996-12-21T02:00:00.000000000,18446744073709551615,2867199309159137213,B,,true,,,, - 1996-12-22T00:44:57.000000000,9223372036854775808,2521269998124177631,C,true,true,,,, - 1996-12-22T00:45:57.000000000,9223372036854775808,2521269998124177631,C,true,true,true,,, - 1996-12-22T00:46:57.000000000,9223372036854775808,2521269998124177631,C,true,true,true,true,, - 1996-12-22T00:47:57.000000000,9223372036854775808,2521269998124177631,C,true,true,true,true,true, + 1996-12-22T00:44:57.000000000,0,2521269998124177631,C,true,true,,,, + 1996-12-22T00:45:57.000000000,0,2521269998124177631,C,true,true,true,,, + 1996-12-22T00:46:57.000000000,0,2521269998124177631,C,true,true,true,true,, + 1996-12-22T00:47:57.000000000,0,2521269998124177631,C,true,true,true,true,true, "###); } diff --git a/crates/sparrow-main/tests/e2e/comparison_tests.rs b/crates/sparrow-main/tests/e2e/comparison_tests.rs index be53ea050..0f9244164 100644 --- a/crates/sparrow-main/tests/e2e/comparison_tests.rs +++ b/crates/sparrow-main/tests/e2e/comparison_tests.rs @@ -8,12 +8,12 @@ use crate::QueryFixture; async fn test_lt_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, lt: Numbers.m < Numbers.n }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,lt - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,false - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,false + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -21,12 +21,12 @@ async fn test_lt_i64() { async fn test_lt_f64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, lt: Numbers.m < Numbers.n}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,lt - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,false - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,false + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -34,12 +34,12 @@ async fn test_lt_f64() { async fn test_lt_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ m: Times.m, n: Times.n, lt: (Times.m as timestamp_ns) < (Times.n as timestamp_ns) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,lt - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,4,2,false - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,3,4,true - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,5, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8,false - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,11,23,true + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,4,2,false + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,3,4,true + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,5, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8,false + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,11,23,true "###); } @@ -61,12 +61,12 @@ async fn test_lt_interval() { async fn test_lt_i64_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, lt: Numbers.m < 10}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,lt - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,false - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,false - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,false + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,false + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -74,12 +74,12 @@ async fn test_lt_i64_literal() { async fn test_lt_f64_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, lt: Numbers.m < 10.0}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,lt - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,false - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,false - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,false + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,false + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -90,12 +90,12 @@ async fn test_lt_timestamp_ns_literal() { let literal = \"1970-01-01T00:00:00.000000010Z\" in { m_time, literal_time: literal as timestamp_ns, lt: m_time < literal}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m_time,literal_time,lt - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1970-01-01T00:00:00.000000004,1970-01-01T00:00:00.000000010,true - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000003,1970-01-01T00:00:00.000000010,true - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,1970-01-01T00:00:00.000000010, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,1970-01-01T00:00:00.000000010, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000008,1970-01-01T00:00:00.000000010,true - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000011,1970-01-01T00:00:00.000000010,false + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1970-01-01T00:00:00.000000004,1970-01-01T00:00:00.000000010,true + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000003,1970-01-01T00:00:00.000000010,true + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,1970-01-01T00:00:00.000000010, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,1970-01-01T00:00:00.000000010, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000008,1970-01-01T00:00:00.000000010,true + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000011,1970-01-01T00:00:00.000000010,false "###); } @@ -103,12 +103,12 @@ async fn test_lt_timestamp_ns_literal() { async fn test_gt_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, gt: Numbers.m > Numbers.n }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,gt - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -116,12 +116,12 @@ async fn test_gt_i64() { async fn test_gt_f64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, gt: Numbers.m > Numbers.n}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,gt - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -129,12 +129,12 @@ async fn test_gt_f64() { async fn test_gt_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ m: Times.m, n: Times.n, gt: (Times.m as timestamp_ns) > (Times.n as timestamp_ns) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,gt - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,4,2,true - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,3,4,false - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,5, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8,false - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,11,23,false + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,4,2,true + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,3,4,false + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,5, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8,false + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,11,23,false "###); } @@ -142,12 +142,12 @@ async fn test_gt_timestamp_ns() { async fn test_gt_i64_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, lt: Numbers.m > 10}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,lt - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,true - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,true + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -155,12 +155,12 @@ async fn test_gt_i64_literal() { async fn test_gt_f64_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, lt: Numbers.m > 10.0}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,lt - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,true - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,true + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -171,12 +171,12 @@ async fn test_gt_timestamp_ns_literal() { let literal = \"1970-01-01T00:00:00.000000010Z\" in { m_time, literal_time: literal as timestamp_ns, gt: m_time > literal}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m_time,literal_time,gt - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1970-01-01T00:00:00.000000004,1970-01-01T00:00:00.000000010,false - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000003,1970-01-01T00:00:00.000000010,false - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,1970-01-01T00:00:00.000000010, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,1970-01-01T00:00:00.000000010, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000008,1970-01-01T00:00:00.000000010,false - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000011,1970-01-01T00:00:00.000000010,true + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1970-01-01T00:00:00.000000004,1970-01-01T00:00:00.000000010,false + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000003,1970-01-01T00:00:00.000000010,false + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,1970-01-01T00:00:00.000000010, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,1970-01-01T00:00:00.000000010, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000008,1970-01-01T00:00:00.000000010,false + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000011,1970-01-01T00:00:00.000000010,true "###); } @@ -184,12 +184,12 @@ async fn test_gt_timestamp_ns_literal() { async fn test_lte_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, lte: Numbers.m <= Numbers.n }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,lte - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,false - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,false + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -197,12 +197,12 @@ async fn test_lte_i64() { async fn test_lte_f64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, lte: Numbers.m <= Numbers.n}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,lte - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,false - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,false + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -210,12 +210,12 @@ async fn test_lte_f64() { async fn test_lte_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ m: Times.m, n: Times.n, lte: (Times.m as timestamp_ns) <= (Times.n as timestamp_ns) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,lte - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,4,2,false - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,3,4,true - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,5, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8,true - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,11,23,true + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,4,2,false + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,3,4,true + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,5, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8,true + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,11,23,true "###); } @@ -223,12 +223,12 @@ async fn test_lte_timestamp_ns() { async fn test_lte_i64_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, lt: Numbers.m <= 10}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,lt - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,false - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,false - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,false + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,false + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -236,12 +236,12 @@ async fn test_lte_i64_literal() { async fn test_lte_f64_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, lt: Numbers.m <= 10.0}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,lt - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,false - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,false - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,false + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,false + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -252,12 +252,12 @@ async fn test_lte_timestamp_ns_literal() { let literal = \"1970-01-01T00:00:00.000000008Z\" in { m_time, literal_time: literal as timestamp_ns, lte: m_time <= literal}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m_time,literal_time,lte - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1970-01-01T00:00:00.000000004,1970-01-01T00:00:00.000000008,true - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000003,1970-01-01T00:00:00.000000008,true - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,1970-01-01T00:00:00.000000008, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,1970-01-01T00:00:00.000000008, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000008,1970-01-01T00:00:00.000000008,true - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000011,1970-01-01T00:00:00.000000008,false + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1970-01-01T00:00:00.000000004,1970-01-01T00:00:00.000000008,true + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000003,1970-01-01T00:00:00.000000008,true + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,1970-01-01T00:00:00.000000008, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,1970-01-01T00:00:00.000000008, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000008,1970-01-01T00:00:00.000000008,true + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000011,1970-01-01T00:00:00.000000008,false "###); } @@ -265,12 +265,12 @@ async fn test_lte_timestamp_ns_literal() { async fn test_gte_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, gte: Numbers.m >= Numbers.n }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,gte - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -278,12 +278,12 @@ async fn test_gte_i64() { async fn test_gte_f64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, gte: Numbers.m >= Numbers.n}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,gte - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -291,12 +291,12 @@ async fn test_gte_f64() { async fn test_gte_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ m: Times.m, n: Times.n, gte: Times.m >= Times.n}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,gte - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,4,2,true - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,3,4,false - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,5, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8,true - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,11,23,false + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,4,2,true + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,3,4,false + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,5, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8,true + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,11,23,false "###); } @@ -304,12 +304,12 @@ async fn test_gte_timestamp_ns() { async fn test_gte_i64_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, lt: Numbers.m >= 10}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,lt - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,true - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,true + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -317,12 +317,12 @@ async fn test_gte_i64_literal() { async fn test_gte_f64_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, lt: Numbers.m >= 10.0}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,lt - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,true - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,true + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -333,11 +333,11 @@ async fn test_gte_timestamp_ns_literal() { let literal = \"1970-01-01T00:00:00.000000008Z\" in { m_time, literal_time: literal as timestamp_ns, gte: m_time >= literal}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m_time,literal_time,gte - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1970-01-01T00:00:00.000000004,1970-01-01T00:00:00.000000008,false - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000003,1970-01-01T00:00:00.000000008,false - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,1970-01-01T00:00:00.000000008, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,1970-01-01T00:00:00.000000008, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000008,1970-01-01T00:00:00.000000008,true - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000011,1970-01-01T00:00:00.000000008,true + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1970-01-01T00:00:00.000000004,1970-01-01T00:00:00.000000008,false + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000003,1970-01-01T00:00:00.000000008,false + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,1970-01-01T00:00:00.000000008, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,1970-01-01T00:00:00.000000008, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000008,1970-01-01T00:00:00.000000008,true + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000011,1970-01-01T00:00:00.000000008,true "###); } diff --git a/crates/sparrow-main/tests/e2e/decoration_tests.rs b/crates/sparrow-main/tests/e2e/decoration_tests.rs index d2d0040ef..3e223bb98 100644 --- a/crates/sparrow-main/tests/e2e/decoration_tests.rs +++ b/crates/sparrow-main/tests/e2e/decoration_tests.rs @@ -14,12 +14,12 @@ use crate::QueryFixture; async fn test_last_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ last: last(Times.n)}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,last - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,2 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,2 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,5 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,5 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23 "###); } @@ -46,11 +46,11 @@ async fn test_last_timestamp_ns_changed_since() { let changed_since = NaiveDateTime::new(date_for_test(1995, 1, 1), time_for_test(0, 0, 0)); insta::assert_snapshot!(QueryFixture::new("{ last: last(Times.n) }").with_changed_since(changed_since).run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,last - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,5 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,5 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23 "###); } @@ -74,9 +74,9 @@ async fn test_last_timestamp_ns_changed_since_equal_to_event_time() { let changed_since = NaiveDateTime::new(date_for_test(1997, 12, 12), time_for_test(0, 42, 57)); insta::assert_snapshot!(QueryFixture::new("{ last: last(Times.n) }").with_changed_since(changed_since).run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,last - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,5 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23 "###); } @@ -94,7 +94,7 @@ async fn test_last_timestamp_ns_windowed_changed_since() { 2003-01-01T00:00:00.000000000,18446744073709551615,12960666915911099378,A, 2004-01-01T00:00:00.000000000,18446744073709551615,2867199309159137213,B, 2004-01-01T00:00:00.000000000,18446744073709551615,12960666915911099378,A, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23 "###); } @@ -145,7 +145,7 @@ async fn test_last_timestamp_ns_changed_since_expect_filtered_results() { insta::assert_snapshot!(QueryFixture::new(FILTERED_RESULTS) .with_changed_since(changed_since).run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,key,time,last - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,B,2004-12-06T00:44:57.000000000,23 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,B,2004-12-06T00:44:57.000000000,23 "###); } @@ -168,11 +168,11 @@ async fn test_last_timestamp_filtered_results() { insta::assert_snapshot!(QueryFixture::new(FILTERED_RESULTS).run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,key,time,last - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,B,1995-10-20T00:40:57.000000000,4 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,B,1996-08-20T00:41:57.000000000,5 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,B,1997-12-12T00:42:57.000000000,5 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,B,1998-12-13T00:43:57.000000000,8 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,B,2004-12-06T00:44:57.000000000,23 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,B,1995-10-20T00:40:57.000000000,4 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,B,1996-08-20T00:41:57.000000000,5 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,B,1997-12-12T00:42:57.000000000,5 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,B,1998-12-13T00:43:57.000000000,8 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,B,2004-12-06T00:44:57.000000000,23 "###); } diff --git a/crates/sparrow-main/tests/e2e/entity_key_output_tests.rs b/crates/sparrow-main/tests/e2e/entity_key_output_tests.rs index 571d24300..0a7a9800c 100644 --- a/crates/sparrow-main/tests/e2e/entity_key_output_tests.rs +++ b/crates/sparrow-main/tests/e2e/entity_key_output_tests.rs @@ -57,12 +57,12 @@ pub(crate) async fn multiple_table_fixture() -> DataFixture { async fn test_entity_keys_numbers() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A, "###); } @@ -70,18 +70,18 @@ async fn test_entity_keys_numbers() { async fn test_multiple_tables_entity_keys() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers2.n }").run_to_csv(&multiple_table_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6, - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:03.000000000,9223372036854775808,2521269998124177631,C,,10.0 - 1996-12-20T00:40:04.000000000,9223372036854775808,1021973589662386405,D,,3.9 - 1996-12-20T00:40:05.000000000,9223372036854775808,2521269998124177631,C,,6.2 - 1996-12-20T00:40:06.000000000,9223372036854775808,2521269998124177631,C,,9.25 - 1996-12-20T00:40:07.000000000,9223372036854775808,2521269998124177631,C,, - 1996-12-20T00:40:08.000000000,9223372036854775808,2521269998124177631,C,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6, + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:03.000000000,0,2521269998124177631,C,,10.0 + 1996-12-20T00:40:04.000000000,0,1021973589662386405,D,,3.9 + 1996-12-20T00:40:05.000000000,0,2521269998124177631,C,,6.2 + 1996-12-20T00:40:06.000000000,0,2521269998124177631,C,,9.25 + 1996-12-20T00:40:07.000000000,0,2521269998124177631,C,, + 1996-12-20T00:40:08.000000000,0,2521269998124177631,C,, "###); } @@ -90,12 +90,12 @@ async fn test_lookup_entity_keys() { insta::assert_snapshot!(QueryFixture::new("{ m: lookup(Numbers.key, sum(Numbers2.n)) }").run_to_csv(&multiple_table_fixture().await).await .unwrap(), @r###" _time,_subsort,_key_hash,_key,m - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A, "###); } @@ -103,12 +103,12 @@ async fn test_lookup_entity_keys() { async fn test_with_key() { insta::assert_snapshot!(QueryFixture::new("Numbers | with_key($input.n, grouping='other_key')").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,subsort,key,m,n - 1996-12-20T00:39:57.000000000,9223372036854775808,10780876405615667760,10,1996-12-20T00:39:57.000000000,0,A,5,10 - 1996-12-20T00:39:58.000000000,9223372036854775808,5496774745203840792,3,1996-12-20T00:39:58.000000000,0,B,24,3 - 1996-12-20T00:39:59.000000000,9223372036854775808,1360592371395427998,6,1996-12-20T00:39:59.000000000,0,A,17,6 - 1996-12-20T00:40:00.000000000,9223372036854775808,15653042715643359010,9,1996-12-20T00:40:00.000000000,0,A,,9 - 1996-12-20T00:40:01.000000000,9223372036854775808,0,,1996-12-20T00:40:01.000000000,0,A,12, - 1996-12-20T00:40:02.000000000,9223372036854775808,0,,1996-12-20T00:40:02.000000000,0,A,, + 1996-12-20T00:39:57.000000000,0,10780876405615667760,10,1996-12-20T00:39:57.000000000,0,A,5,10 + 1996-12-20T00:39:58.000000000,0,5496774745203840792,3,1996-12-20T00:39:58.000000000,0,B,24,3 + 1996-12-20T00:39:59.000000000,0,1360592371395427998,6,1996-12-20T00:39:59.000000000,0,A,17,6 + 1996-12-20T00:40:00.000000000,0,15653042715643359010,9,1996-12-20T00:40:00.000000000,0,A,,9 + 1996-12-20T00:40:01.000000000,0,0,,1996-12-20T00:40:01.000000000,0,A,12, + 1996-12-20T00:40:02.000000000,0,0,,1996-12-20T00:40:02.000000000,0,A,, "###); } @@ -117,11 +117,11 @@ async fn test_lookup_with_key_entity_keys() { insta::assert_snapshot!(QueryFixture::new("{ m: lookup(Numbers.key, with_key(Numbers.key, sum(Numbers.m))) }").run_to_csv(&multiple_table_fixture().await).await .unwrap(), @r###" _time,_subsort,_key_hash,_key,m - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,22.8 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,22.8 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,35.2 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,35.2 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,22.8 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,22.8 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,35.2 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,35.2 "###); } diff --git a/crates/sparrow-main/tests/e2e/equality_tests.rs b/crates/sparrow-main/tests/e2e/equality_tests.rs index 7b59c449d..0d82a626f 100644 --- a/crates/sparrow-main/tests/e2e/equality_tests.rs +++ b/crates/sparrow-main/tests/e2e/equality_tests.rs @@ -11,12 +11,12 @@ use crate::QueryFixture; async fn test_eq_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, eq: Numbers.m == Numbers.n }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,false - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,false + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -24,12 +24,12 @@ async fn test_eq_i64() { async fn test_eq_f64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, eq: Numbers.m == Numbers.n }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,false - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,false + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -37,12 +37,12 @@ async fn test_eq_f64() { async fn test_eq_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ m: Times.m, n: Times.n, eq: (Times.m as timestamp_ns) == (Times.n as timestamp_ns) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,eq - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,4,2,false - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,3,4,false - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,5, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8,true - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,11,23,false + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,4,2,false + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,3,4,false + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,5, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8,true + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,11,23,false "###); } @@ -50,13 +50,13 @@ async fn test_eq_timestamp_ns() { async fn test_eq_boolean() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, b: Booleans.b, eq: Booleans.a == Booleans.b }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,b,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,true,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,false,true - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,true, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,false,false - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,true,false - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false,, - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,true,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false,false,true + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,,true, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true,false,false + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false,true,false + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false,, + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,,, "###); } @@ -64,12 +64,12 @@ async fn test_eq_boolean() { async fn test_eq_string() { insta::assert_snapshot!(QueryFixture::new("{ s: Strings.s, t: Strings.t, eq: Strings.s == Strings.t }").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,t,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hEllo,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World,world,false - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world,hello world,true - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,greetings,false - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,,salutations,false - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye,,false + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo,hEllo,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World,world,false + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world,hello world,true + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,,greetings,false + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,,salutations,false + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye,,false "###); } @@ -91,12 +91,12 @@ async fn test_eq_record() { async fn test_eq_i64_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, eq: Numbers.m == 10 }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,false - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,false - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,false + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,false + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -104,12 +104,12 @@ async fn test_eq_i64_literal() { async fn test_eq_i64_literal_converse() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, eq: 10 == Numbers.m }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,false - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,false - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,false + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,false + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -117,12 +117,12 @@ async fn test_eq_i64_literal_converse() { async fn test_eq_i64_literal_null() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, eq: Numbers.m == null }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17, - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17, + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -130,12 +130,12 @@ async fn test_eq_i64_literal_null() { async fn test_eq_i64_literal_null_converse() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, eq: null == Numbers.m }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17, - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17, + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -143,12 +143,12 @@ async fn test_eq_i64_literal_null_converse() { async fn test_eq_f64_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, eq: Numbers.m == 24.3 }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,false - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,false + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -156,12 +156,12 @@ async fn test_eq_f64_literal() { async fn test_eq_f64_literal_zero() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, eq: Numbers.m == 0.0 }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,false - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,false - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,false + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,false + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -172,12 +172,12 @@ async fn test_eq_timestamp_ns_literal() { let literal = \"1970-01-01T00:00:00.000000008Z\" in { m_time, literal_time: literal as timestamp_ns, lt: m_time == literal}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m_time,literal_time,lt - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1970-01-01T00:00:00.000000004,1970-01-01T00:00:00.000000008,false - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000003,1970-01-01T00:00:00.000000008,false - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,1970-01-01T00:00:00.000000008, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,1970-01-01T00:00:00.000000008, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000008,1970-01-01T00:00:00.000000008,true - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000011,1970-01-01T00:00:00.000000008,false + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1970-01-01T00:00:00.000000004,1970-01-01T00:00:00.000000008,false + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000003,1970-01-01T00:00:00.000000008,false + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,1970-01-01T00:00:00.000000008, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,1970-01-01T00:00:00.000000008, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000008,1970-01-01T00:00:00.000000008,true + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000011,1970-01-01T00:00:00.000000008,false "###); } @@ -185,13 +185,13 @@ async fn test_eq_timestamp_ns_literal() { async fn test_eq_boolean_literal() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, eq: Booleans.a == true }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,false - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,false - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false,false - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false,false + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false,false + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false,false + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,, "###); } @@ -199,13 +199,13 @@ async fn test_eq_boolean_literal() { async fn test_eq_boolean_literal_null() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, eq: Booleans.a == null }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true, - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true, + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,, "###); } @@ -213,13 +213,13 @@ async fn test_eq_boolean_literal_null() { async fn test_eq_boolean_literal_converse() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, eq: true == Booleans.a }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,false - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,false - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false,false - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false,false + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false,false + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false,false + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,, "###); } @@ -227,13 +227,13 @@ async fn test_eq_boolean_literal_converse() { async fn test_eq_boolean_literal_null_converse() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, eq: null == Booleans.a }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true, - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true, + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,, "###); } @@ -241,12 +241,12 @@ async fn test_eq_boolean_literal_null_converse() { async fn test_eq_string_literal() { insta::assert_snapshot!(QueryFixture::new("let eq = Strings.s == \"hello world\" in { s: Strings.s, eq, is_valid: is_valid(eq)}").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,eq,is_valid - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,false,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World,false,true - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world,true,true - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,false,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,,false,true - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye,false,true + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo,false,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World,false,true + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world,true,true + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,,false,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,,false,true + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye,false,true "###); } @@ -254,12 +254,12 @@ async fn test_eq_string_literal() { async fn test_eq_string_literal_converse() { insta::assert_snapshot!(QueryFixture::new("let eq = \"hello world\" == Strings.s in { s: Strings.s, eq, is_valid: is_valid(eq)}").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,eq,is_valid - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,false,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World,false,true - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world,true,true - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,false,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,,false,true - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye,false,true + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo,false,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World,false,true + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world,true,true + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,,false,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,,false,true + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye,false,true "###); } @@ -267,12 +267,12 @@ async fn test_eq_string_literal_converse() { async fn test_eq_string_literal_null() { insta::assert_snapshot!(QueryFixture::new("{ s: Strings.s, eq: Strings.s == null }").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo, - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World, - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo, + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World, + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye, "###); } @@ -280,12 +280,12 @@ async fn test_eq_string_literal_null() { async fn test_eq_string_literal_null_converse() { insta::assert_snapshot!(QueryFixture::new("{ s: Strings.s, eq: null == Strings.s }").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,eq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo, - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World, - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo, + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World, + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye, "###); } @@ -307,12 +307,12 @@ async fn test_eq_record_literal() { async fn test_neq_i64_old() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, neq: Numbers.m <> Numbers.n }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -320,12 +320,12 @@ async fn test_neq_i64_old() { async fn test_neq_i64_new() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, neq: Numbers.m != Numbers.n }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -333,12 +333,12 @@ async fn test_neq_i64_new() { async fn test_neq_f64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, neq: Numbers.m <> Numbers.n}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -346,12 +346,12 @@ async fn test_neq_f64() { async fn test_neq_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ m: Times.m, n: Times.n, neq: (Times.m as timestamp_ns) <> (Times.n as timestamp_ns) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,neq - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,4,2,true - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,3,4,true - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,5, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8,false - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,11,23,true + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,4,2,true + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,3,4,true + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,5, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8,false + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,11,23,true "###); } @@ -359,13 +359,13 @@ async fn test_neq_timestamp_ns() { async fn test_neq_boolean() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, b: Booleans.b, neq: Booleans.a <> Booleans.b}").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,b,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,true,false - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,false,false - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,true, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,false,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,true,true - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false,, - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,true,false + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false,false,false + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,,true, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true,false,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false,true,true + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false,, + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,,, "###); } @@ -373,12 +373,12 @@ async fn test_neq_boolean() { async fn test_neq_string() { insta::assert_snapshot!(QueryFixture::new("{ s: Strings.s, t: Strings.t, neq: Strings.s <> Strings.t}").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,t,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hEllo,false - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World,world,true - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world,hello world,false - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,greetings,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,,salutations,true - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye,,true + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo,hEllo,false + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World,world,true + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world,hello world,false + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,,greetings,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,,salutations,true + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye,,true "###); } @@ -400,12 +400,12 @@ async fn test_neq_record() { async fn test_neq_i64_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, neq: Numbers.m <> 5 }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,true - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,true + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -413,12 +413,12 @@ async fn test_neq_i64_literal() { async fn test_neq_i64_literal_converse() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, neq: 5 != Numbers.m }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,true - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,true + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -426,12 +426,12 @@ async fn test_neq_i64_literal_converse() { async fn test_neq_i64_literal_null() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, neq: Numbers.m != null }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17, - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17, + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -439,12 +439,12 @@ async fn test_neq_i64_literal_null() { async fn test_neq_i64_literal_null_converse() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, neq: null != Numbers.m }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17, - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17, + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -452,12 +452,12 @@ async fn test_neq_i64_literal_null_converse() { async fn test_neq_f64_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, neq: Numbers.m <> 5.2}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,false - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,true - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,false + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,true + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -468,12 +468,12 @@ async fn test_neq_timestamp_ns_literal() { let literal = \"1970-01-01T00:00:00.000000008Z\" in { m_time, literal_time: literal as timestamp_ns, neq: m_time <> literal}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m_time,literal_time,neq - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1970-01-01T00:00:00.000000004,1970-01-01T00:00:00.000000008,true - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000003,1970-01-01T00:00:00.000000008,true - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,1970-01-01T00:00:00.000000008, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,1970-01-01T00:00:00.000000008, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000008,1970-01-01T00:00:00.000000008,false - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,1970-01-01T00:00:00.000000011,1970-01-01T00:00:00.000000008,true + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1970-01-01T00:00:00.000000004,1970-01-01T00:00:00.000000008,true + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000003,1970-01-01T00:00:00.000000008,true + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,1970-01-01T00:00:00.000000008, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,1970-01-01T00:00:00.000000008, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000008,1970-01-01T00:00:00.000000008,false + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,1970-01-01T00:00:00.000000011,1970-01-01T00:00:00.000000008,true "###); } @@ -481,13 +481,13 @@ async fn test_neq_timestamp_ns_literal() { async fn test_neq_boolean_literal() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, neq: Booleans.a <> true }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,false - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,true - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,false - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,true - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false,true - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,false + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false,true + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true,false + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false,true + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false,true + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,, "###); } @@ -495,13 +495,13 @@ async fn test_neq_boolean_literal() { async fn test_neq_boolean_literal_converse() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, neq: true != Booleans.a }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,false - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,true - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,false - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,true - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false,true - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,false + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false,true + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true,false + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false,true + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false,true + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,, "###); } @@ -509,13 +509,13 @@ async fn test_neq_boolean_literal_converse() { async fn test_neq_boolean_literal_null() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, neq: Booleans.a <> null }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true, - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true, + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,, "###); } @@ -523,13 +523,13 @@ async fn test_neq_boolean_literal_null() { async fn test_neq_boolean_literal_null_converse() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, neq: null != Booleans.a }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true, - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true, + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,, "###); } @@ -537,12 +537,12 @@ async fn test_neq_boolean_literal_null_converse() { async fn test_neq_string_literal() { insta::assert_snapshot!(QueryFixture::new("{ s: Strings.s, neq: Strings.s <> \"hello world\" }").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World,true - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world,false - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,,true - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye,true + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World,true + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world,false + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,,true + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye,true "###); } @@ -550,12 +550,12 @@ async fn test_neq_string_literal() { async fn test_neq_string_literal_converse() { insta::assert_snapshot!(QueryFixture::new("{ s: Strings.s, neq: \"hello world\" != Strings.s }").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World,true - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world,false - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,,true - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye,true + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World,true + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world,false + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,,true + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye,true "###); } @@ -563,12 +563,12 @@ async fn test_neq_string_literal_converse() { async fn test_neq_string_literal_null() { insta::assert_snapshot!(QueryFixture::new("{ s: Strings.s, neq: Strings.s <> null }").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo, - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World, - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo, + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World, + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye, "###); } @@ -576,12 +576,12 @@ async fn test_neq_string_literal_null() { async fn test_neq_string_literal_null_converse() { insta::assert_snapshot!(QueryFixture::new("{ s: Strings.s, neq: null != Strings.s }").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,neq - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo, - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World, - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo, + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World, + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye, "###); } diff --git a/crates/sparrow-main/tests/e2e/formula_tests.rs b/crates/sparrow-main/tests/e2e/formula_tests.rs index f515feedb..b9b1872f8 100644 --- a/crates/sparrow-main/tests/e2e/formula_tests.rs +++ b/crates/sparrow-main/tests/e2e/formula_tests.rs @@ -7,12 +7,12 @@ async fn test_formulas_out_of_order() { .with_formula("n_plus_m", "numbers_m + Numbers.n") .with_formula("numbers_m", "Numbers.m").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,add - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,15 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,27 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,23 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,15 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,27 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,23 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -103,11 +103,11 @@ async fn test_unused_formula_does_not_report_query_error() { insta::assert_snapshot!(QueryFixture::new("{ n: Numbers.n } ") .with_formula("Invalid", "Numbers.n + $$").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,6 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,9 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,6 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,9 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A, "###); } diff --git a/crates/sparrow-main/tests/e2e/general_tests.rs b/crates/sparrow-main/tests/e2e/general_tests.rs index 93c60b075..2d5709bef 100644 --- a/crates/sparrow-main/tests/e2e/general_tests.rs +++ b/crates/sparrow-main/tests/e2e/general_tests.rs @@ -11,12 +11,12 @@ use crate::QueryFixture; async fn test_is_valid_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, is_valid: is_valid(Numbers.m)}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,is_valid - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,false - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,true - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,false + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,false + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,true + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,false "###); } @@ -24,12 +24,12 @@ async fn test_is_valid_i64() { async fn test_is_valid_f64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, is_valid: is_valid(Numbers.m)}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,is_valid - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,false - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,true - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,false + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,false + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,true + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,false "###); } @@ -37,12 +37,12 @@ async fn test_is_valid_f64() { async fn test_is_valid_string() { insta::assert_snapshot!(QueryFixture::new("{ s: Strings.s, is_valid: is_valid(Strings.s)}").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,is_valid - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World,true - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world,true - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,,true - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye,true + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World,true + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world,true + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,,true + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye,true "###); } @@ -50,13 +50,13 @@ async fn test_is_valid_string() { async fn test_is_valid_boolean() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, is_valid: is_valid(Booleans.a)}").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,is_valid - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,true - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,false - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,true - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false,true - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,,false + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false,true + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,,false + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false,true + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false,true + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,,false "###); } @@ -64,12 +64,12 @@ async fn test_is_valid_boolean() { async fn test_is_valid_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ n: Times.n, is_valid: is_valid(Times.n)}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,is_valid - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,2,true - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4,true - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5,true - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,false - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,true - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23,true + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,2,true + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4,true + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,5,true + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,false + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,true + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23,true "###); } @@ -77,12 +77,12 @@ async fn test_is_valid_timestamp_ns() { async fn test_is_valid_record() { insta::assert_snapshot!(QueryFixture::new("{ is_valid: is_valid(Times)}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,is_valid - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,true - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,true - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,true - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,true + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,true + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,true + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,true + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,true + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,true + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,true "###); } @@ -90,12 +90,12 @@ async fn test_is_valid_record() { async fn test_hash_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, hash: hash(Numbers.m)}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,hash - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,16461383214845928621 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,11274228027825807126 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,322098188319043992 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,0 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,2287927947190353380 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,0 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,16461383214845928621 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,11274228027825807126 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,322098188319043992 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,0 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,2287927947190353380 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,0 "###); } @@ -103,12 +103,12 @@ async fn test_hash_i64() { async fn test_hash_string() { insta::assert_snapshot!(QueryFixture::new("{ s: Strings.s, hash: hash(Strings.s)}").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,hash - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,7011413575603941612 - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World,13226470954278774291 - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world,10229417672155185436 - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,5663277146615294718 - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,,5663277146615294718 - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye,12405021407607093536 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo,7011413575603941612 + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World,13226470954278774291 + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world,10229417672155185436 + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,,5663277146615294718 + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,,5663277146615294718 + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye,12405021407607093536 "###); } @@ -116,12 +116,12 @@ async fn test_hash_string() { async fn test_hash_struct() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, hash: hash({m: Numbers.m})}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,hash - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,328624516884178922 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,11318067407944751383 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,11917632967804650977 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,0 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,10866357751204891869 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,0 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,328624516884178922 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,11318067407944751383 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,11917632967804650977 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,0 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,10866357751204891869 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,0 "###); } @@ -129,13 +129,13 @@ async fn test_hash_struct() { async fn test_hash_boolean() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, hash: hash(Booleans.a)}").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,hash - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,18433805721903975440 - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,11832085162654999889 - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,0 - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,18433805721903975440 - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,11832085162654999889 - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false,11832085162654999889 - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,,0 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,18433805721903975440 + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false,11832085162654999889 + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,,0 + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true,18433805721903975440 + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false,11832085162654999889 + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false,11832085162654999889 + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,,0 "###); } @@ -143,12 +143,12 @@ async fn test_hash_boolean() { async fn test_hash_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ n: Times.n, hash: hash(Times.n)}").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,hash - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,2,2694864431690786590 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4,17062639839782733832 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5,16461383214845928621 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,0 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,6794973171266502674 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23,5700754754056540783 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,2,2694864431690786590 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4,17062639839782733832 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,5,16461383214845928621 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,0 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,6794973171266502674 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23,5700754754056540783 "###); } @@ -163,7 +163,7 @@ async fn test_hash_record() { async fn test_basic_limit_rows_to_1() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, is_valid: is_valid(Numbers.m)}").with_preview_rows(1).run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,is_valid - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,true + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,true "###); } @@ -173,12 +173,12 @@ async fn test_basic_limit_rows_to_1() { async fn test_basic_limit_rows_all() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, is_valid: is_valid(Numbers.m)}").with_preview_rows(100).run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,is_valid - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,false - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,true - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,false + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,false + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,true + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,false "###); } diff --git a/crates/sparrow-main/tests/e2e/json_tests.rs b/crates/sparrow-main/tests/e2e/json_tests.rs index de50ff09e..077706e57 100644 --- a/crates/sparrow-main/tests/e2e/json_tests.rs +++ b/crates/sparrow-main/tests/e2e/json_tests.rs @@ -68,12 +68,12 @@ pub(crate) async fn invalid_json_data_fixture() -> DataFixture { async fn test_json_parses_field() { insta::assert_snapshot!(QueryFixture::new("let json = json(Json.json) in { a_test: json.a as i64, b_test: json(Json.json).b }").run_to_csv(&json_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a_test,b_test - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10,dog - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4,lizard - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,12,cat - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,34, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,6,dog + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10,dog + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,4,lizard + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,1, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,12,cat + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,34, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,6,dog "###) } @@ -81,12 +81,12 @@ async fn test_json_parses_field() { async fn test_json_string_field_usable_in_string_functions() { insta::assert_snapshot!(QueryFixture::new("let json = json(Json.json) in { string: json.b, len: len(json.b) }").run_to_csv(&json_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,string,len - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,dog,3 - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,lizard,6 - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,cat,3 - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,dog,3 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,dog,3 + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,lizard,6 + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,cat,3 + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,dog,3 "###) } @@ -94,12 +94,12 @@ async fn test_json_string_field_usable_in_string_functions() { async fn test_json_field_number_as_string() { insta::assert_snapshot!(QueryFixture::new("let json = json(Json.json) in { num_as_str: json.a as string, len: len(json.a as string) }").run_to_csv(&json_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,num_as_str,len - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10,2 - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4,1 - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1,1 - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,12,2 - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,34,2 - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,6,1 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10,2 + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,4,1 + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,1,1 + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,12,2 + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,34,2 + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,6,1 "###) } @@ -107,12 +107,12 @@ async fn test_json_field_number_as_string() { async fn test_json_field_as_number_with_addition() { insta::assert_snapshot!(QueryFixture::new("let json = json(Json.json) in { a: json.a, plus_one: (json.a as i64) + 1 }").run_to_csv(&json_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,plus_one - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10,11 - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4,5 - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1,2 - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,12,13 - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,34,35 - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,6,7 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10,11 + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,4,5 + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,1,2 + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,12,13 + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,34,35 + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,6,7 "###) } @@ -123,12 +123,12 @@ async fn test_incorrect_json_format_produces_null() { // print "null" in other fields. insta::assert_snapshot!(QueryFixture::new("let json = json(Json.json) in { a_test: json.a as i64, b_test: json(Json.json).b }").run_to_csv(&invalid_json_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a_test,b_test - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,12,cat - 1996-12-20T00:43:57.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,6,dog + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,1, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,12,cat + 1996-12-20T00:43:57.000000000,0,12960666915911099378,A,, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,6,dog "###) } diff --git a/crates/sparrow-main/tests/e2e/logical_tests.rs b/crates/sparrow-main/tests/e2e/logical_tests.rs index e59e2f7b5..5d0221579 100644 --- a/crates/sparrow-main/tests/e2e/logical_tests.rs +++ b/crates/sparrow-main/tests/e2e/logical_tests.rs @@ -10,13 +10,13 @@ use crate::QueryFixture; async fn test_not_boolean() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, not_a: !Booleans.a }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,not_a - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,false - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,true - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,false - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,true - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false,true - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,false + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false,true + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true,false + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false,true + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false,true + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,, "###); } @@ -24,13 +24,13 @@ async fn test_not_boolean() { async fn test_logical_or_boolean() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, b: Booleans.b, logical_or: Booleans.a or Booleans.b }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,b,logical_or - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,true,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,false,false - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,true,true - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,false,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,true,true - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false,, - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,true,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false,false,false + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,,true,true + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true,false,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false,true,true + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false,, + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,,, "###); } @@ -38,13 +38,13 @@ async fn test_logical_or_boolean() { async fn test_logical_and_boolean() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, b: Booleans.b, logical_and: Booleans.a and Booleans.b }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,b,logical_and - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,true,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,false,false - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,true, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,false,false - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,true,false - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false,,false - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,true,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false,false,false + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,,true, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true,false,false + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false,true,false + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false,,false + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,,, "###); } @@ -52,13 +52,13 @@ async fn test_logical_and_boolean() { async fn test_if_boolean() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, if_bool: Booleans.a | if(Booleans.a) }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,if_bool - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false, - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false, + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,, "###); } @@ -66,12 +66,12 @@ async fn test_if_boolean() { async fn test_if_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, if_i64: Numbers.m | if(Numbers.m == 5) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,if_i64 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17, - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17, + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -79,12 +79,12 @@ async fn test_if_i64() { async fn test_if_f64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, if_f64: Numbers.m | if(Numbers.m == 5.2) }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,if_f64 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,5.2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6, - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,5.2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6, + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -92,12 +92,12 @@ async fn test_if_f64() { async fn test_if_string() { insta::assert_snapshot!(QueryFixture::new("{ s: Strings.s, if_string: Strings.s | if(Strings.s == \"hEllo\") }").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,if_string - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World, - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo,hEllo + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World, + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye, "###); } @@ -105,12 +105,12 @@ async fn test_if_string() { async fn test_if_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ t: Times.n, if_ts: Times.n | if(Times.key == \"B\") }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,t,if_ts - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,2, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4,4 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5,5 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23,23 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,2, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4,4 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,5,5 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23,23 "###); } @@ -118,12 +118,12 @@ async fn test_if_timestamp_ns() { async fn test_if_record() { insta::assert_snapshot!(QueryFixture::new("{ t: Times.n, if_record: Times | if(Times.key == \"B\") | $input.n }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,t,if_record - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,2, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4,4 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5,5 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23,23 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,2, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4,4 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,5,5 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23,23 "###); } @@ -131,12 +131,12 @@ async fn test_if_record() { async fn test_if_literal() { insta::assert_snapshot!(QueryFixture::new("{ if_literal: 1 | if(Times.key == \"B\") }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,if_literal - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,1 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,1 "###); } @@ -144,13 +144,13 @@ async fn test_if_literal() { async fn test_null_if_boolean() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, null_if_bool: Booleans.a | null_if(Booleans.a) }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,null_if_bool - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true, - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,false - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,false - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false,false - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true, + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false,false + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false,false + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false,false + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,, "###); } @@ -158,12 +158,12 @@ async fn test_null_if_boolean() { async fn test_null_if_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, null_if_i64: Numbers.m | null_if(Numbers.m == 5) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,null_if_i64 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,17 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,12 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,17 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,12 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -171,12 +171,12 @@ async fn test_null_if_i64() { async fn test_null_if_f64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, null_if_f64: Numbers.m | null_if(Numbers.m == 5.2) }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,null_if_f64 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,24.3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,17.6 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,12.4 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,24.3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,17.6 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,12.4 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -184,12 +184,12 @@ async fn test_null_if_f64() { async fn test_null_if_string() { insta::assert_snapshot!(QueryFixture::new("{ s: Strings.s, null_if_string: Strings.s | null_if(Strings.s == \"hEllo\") }").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,null_if_string - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo, - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World,World - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world,hello world - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye,goodbye + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo, + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World,World + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world,hello world + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye,goodbye "###); } @@ -197,12 +197,12 @@ async fn test_null_if_string() { async fn test_null_if_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ t: Times.n, null_if_ts: Times.n | null_if(Times.key == \"B\") }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,t,null_if_ts - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,2,2 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,2,2 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,5, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23, "###); } @@ -210,12 +210,12 @@ async fn test_null_if_timestamp_ns() { async fn test_null_if_record() { insta::assert_snapshot!(QueryFixture::new("{ n: Times.n, null_if_record: Times | null_if(Times.key == \"B\") | $input.n }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,null_if_record - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,2,2 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,2,2 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,5, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23, "###); } @@ -223,12 +223,12 @@ async fn test_null_if_record() { async fn test_null_if_literal() { insta::assert_snapshot!(QueryFixture::new("{ null_if_literal: 1 | null_if(Times.key == \"B\") }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,null_if_literal - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B, "###); } @@ -236,13 +236,13 @@ async fn test_null_if_literal() { async fn test_else_boolean() { insta::assert_snapshot!(QueryFixture::new("{ a: Booleans.a, b: Booleans.b, a_else_b: Booleans.a | else(Booleans.b) }").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,a,b,a_else_b - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,true,true - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,false,false,false - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,true,true - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,true,false,true - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,false,true,false - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,false,,false - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,true,true + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,false,false,false + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,,true,true + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,true,false,true + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,false,true,false + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,false,,false + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,,, "###); } @@ -250,12 +250,12 @@ async fn test_else_boolean() { async fn test_else_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, m_else_n: Numbers.m | else(Numbers.n) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,m_else_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,24 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,17 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9,9 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,,12 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,24 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,17 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9,9 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,,12 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -263,12 +263,12 @@ async fn test_else_i64() { async fn test_else_f64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, m_else_n: Numbers.m | else(Numbers.n) }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,m_else_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,5.2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,24.3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,17.6 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25,9.25 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,,12.4 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,5.2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,24.3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,17.6 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25,9.25 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,,12.4 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -276,12 +276,12 @@ async fn test_else_f64() { async fn test_else_string() { insta::assert_snapshot!(QueryFixture::new("{ s: Strings.s, t: Strings.t, s_else_t: Strings.s | else(Strings.t) }").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,s,t,s_else_t - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hEllo,hEllo,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,World,world,World - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world,hello world,hello world - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,greetings, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,,salutations, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye,,goodbye + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hEllo,hEllo,hEllo + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,World,world,World + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world,hello world,hello world + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,,greetings, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,,salutations, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye,,goodbye "###); } @@ -289,12 +289,12 @@ async fn test_else_string() { async fn test_else_timestamp_ns() { insta::assert_snapshot!(QueryFixture::new("{ m: Times.m, n: Times.n, m_else_n: Times.m | else(Times.n) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,m_else_n - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,4,2,4 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,3,4,3 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,5,5 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8,8 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,11,23,11 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,4,2,4 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,3,4,3 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,5,5 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8,8 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,11,23,11 "###); } @@ -302,12 +302,12 @@ async fn test_else_timestamp_ns() { async fn test_else_record() { insta::assert_snapshot!(QueryFixture::new("{ m: Times.m, n: Times.n, times_else_times: Times | else(Times) | $input.n }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,times_else_times - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,4,2,2 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,3,4,4 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,5,5 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8,8 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,11,23,23 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,4,2,2 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,3,4,4 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,5,5 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8,8 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,11,23,23 "###); } @@ -315,12 +315,12 @@ async fn test_else_record() { async fn test_else_literal() { insta::assert_snapshot!(QueryFixture::new("{ n: Times.n, n_else_literal: Times.n | else(12345) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,n_else_literal - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,2,2 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,4,4 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5,5 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,12345 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,8,8 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,23,23 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,2,2 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,4,4 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,5,5 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,12345 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,8,8 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,23,23 "###); } @@ -329,8 +329,8 @@ async fn test_if_record_nested_is_valid_num_eq() { // Test for https://gitlab.com/kaskada/kaskada/-/issues/342 insta::assert_snapshot!(QueryFixture::new("Times | extend({gr_5: $input.n > 5}) | if(Times.n > 5) | when(is_valid($input.key))").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,gr_5,time,subsort,key,n,m,other_time,fruit - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,true,1998-12-13T00:43:57.000000000,0,B,8,8,, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,true,2004-12-06T00:44:57.000000000,0,B,23,11,1994-12-20T00:39:57.000000000,mango + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,true,1998-12-13T00:43:57.000000000,0,B,8,8,, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,true,2004-12-06T00:44:57.000000000,0,B,23,11,1994-12-20T00:39:57.000000000,mango "###); } @@ -338,7 +338,7 @@ async fn test_if_record_nested_is_valid_num_eq() { async fn test_if_record_nested_is_valid_string_eq() { insta::assert_snapshot!(QueryFixture::new("Times | extend({eq_A: len($input.fruit) > 6}) | if (len(Times.fruit) > 6) | when(is_valid($input.key))").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,eq_A,time,subsort,key,n,m,other_time,fruit - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,true,1995-10-20T00:40:57.000000000,0,B,4,3,1994-11-20T00:39:57.000000000,watermelon + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,true,1995-10-20T00:40:57.000000000,0,B,4,3,1994-11-20T00:39:57.000000000,watermelon "###); } @@ -346,12 +346,12 @@ async fn test_if_record_nested_is_valid_string_eq() { async fn test_if_null_condition_number() { insta::assert_snapshot!(QueryFixture::new("Times | if ($input.n > 0) | extend({ cond: Times.n > 0 })").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,cond,time,subsort,key,n,m,other_time,fruit - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,1994-12-20T00:39:57.000000000,0,A,2,4,2003-12-20T00:39:57.000000000,pear - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,true,1995-10-20T00:40:57.000000000,0,B,4,3,1994-11-20T00:39:57.000000000,watermelon - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,true,1996-08-20T00:41:57.000000000,0,B,5,,1998-12-20T00:39:57.000000000,mango - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,,,,,,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,true,1998-12-13T00:43:57.000000000,0,B,8,8,, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,true,2004-12-06T00:44:57.000000000,0,B,23,11,1994-12-20T00:39:57.000000000,mango + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,true,1994-12-20T00:39:57.000000000,0,A,2,4,2003-12-20T00:39:57.000000000,pear + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,true,1995-10-20T00:40:57.000000000,0,B,4,3,1994-11-20T00:39:57.000000000,watermelon + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,true,1996-08-20T00:41:57.000000000,0,B,5,,1998-12-20T00:39:57.000000000,mango + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,,,,,,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,true,1998-12-13T00:43:57.000000000,0,B,8,8,, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,true,2004-12-06T00:44:57.000000000,0,B,23,11,1994-12-20T00:39:57.000000000,mango "###); } @@ -359,12 +359,12 @@ async fn test_if_null_condition_number() { async fn test_if_null_condition_string_equality() { insta::assert_snapshot!(QueryFixture::new("Times | if ($input.fruit == \"mango\")").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,subsort,key,n,m,other_time,fruit - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,,,,,,, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,,,,,,, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000,0,B,5,,1998-12-20T00:39:57.000000000,mango - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,,,,,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,,,,,,, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000,0,B,23,11,1994-12-20T00:39:57.000000000,mango + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,,,,,,, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,,,,,,, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000,0,B,5,,1998-12-20T00:39:57.000000000,mango + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,,,,,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,,,,,,, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000,0,B,23,11,1994-12-20T00:39:57.000000000,mango "###); } @@ -372,12 +372,12 @@ async fn test_if_null_condition_string_equality() { async fn test_null_if_null_condition() { insta::assert_snapshot!(QueryFixture::new("Times | null_if ($input.n > 6)").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,subsort,key,n,m,other_time,fruit - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000,0,A,2,4,2003-12-20T00:39:57.000000000,pear - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000,0,B,4,3,1994-11-20T00:39:57.000000000,watermelon - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000,0,B,5,,1998-12-20T00:39:57.000000000,mango - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,,,,,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,,,,,,, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,,,,,,, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000,0,A,2,4,2003-12-20T00:39:57.000000000,pear + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000,0,B,4,3,1994-11-20T00:39:57.000000000,watermelon + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000,0,B,5,,1998-12-20T00:39:57.000000000,mango + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,,,,,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,,,,,,, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,,,,,,, "###); } @@ -386,11 +386,11 @@ async fn test_null_if_condition_null_values() { // Ensure that null rows are produced insta::assert_snapshot!(QueryFixture::new("Times | if ($input.n < 0)").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,subsort,key,n,m,other_time,fruit - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,,,,,,, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,,,,,,, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,,,,,,, - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,,,,,, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,,,,,,, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,,,,,,, + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,,,,,,, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,,,,,,, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,,,,,,, + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,,,,,,, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,,,,,,, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,,,,,,, "###); } diff --git a/crates/sparrow-main/tests/e2e/lookup_tests.rs b/crates/sparrow-main/tests/e2e/lookup_tests.rs index 086afc081..626514b27 100644 --- a/crates/sparrow-main/tests/e2e/lookup_tests.rs +++ b/crates/sparrow-main/tests/e2e/lookup_tests.rs @@ -78,12 +78,12 @@ async fn test_lookup_self_i64() { let last_sender_sum_sent = lookup(last(Received.from), sum_sent) in { last_sender, last_sender_sum_sent }").run_to_csv(&lookup_account_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,last_sender,last_sender_sum_sent - 1996-12-20T00:39:57.000000000,9223372036854775808,2694864431690786590,2,0,50 - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,0,61 - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,2,25 - 1998-12-20T00:39:57.000000000,9223372036854775811,18433805721903975440,1,0,86 - 1999-12-20T00:39:58.000000000,9223372036854775812,18433805721903975440,1,0,98 - 1999-12-20T00:39:58.000000000,9223372036854775813,18433805721903975440,1,0,98 + 1996-12-20T00:39:57.000000000,0,2694864431690786590,2,0,50 + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,0,61 + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,2,25 + 1998-12-20T00:39:57.000000000,3,18433805721903975440,1,0,86 + 1999-12-20T00:39:58.000000000,4,18433805721903975440,1,0,98 + 1999-12-20T00:39:58.000000000,5,18433805721903975440,1,0,98 "###); } @@ -94,17 +94,17 @@ async fn test_lookup_self_i64_with_merge_interpolation() { let last_sender_sum_sent = lookup(last(Received.from), sum_sent) in { sum_sent, last_sender, last_sender_sum_sent }").run_to_csv(&lookup_account_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sum_sent,last_sender,last_sender_sum_sent - 1996-12-20T00:39:57.000000000,9223372036854775808,2694864431690786590,2,,0,50 - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,50,, - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,61,0,61 - 1997-12-20T00:39:58.000000000,9223372036854775810,2694864431690786590,2,25,0,50 - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,61,2,25 - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,86,2,25 - 1998-12-20T00:39:57.000000000,9223372036854775811,18433805721903975440,1,,0,86 - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,98,2,25 - 1999-12-20T00:39:58.000000000,9223372036854775812,18433805721903975440,1,,0,98 - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,98,2,25 - 1999-12-20T00:39:58.000000000,9223372036854775813,18433805721903975440,1,,0,98 + 1996-12-20T00:39:57.000000000,0,2694864431690786590,2,,0,50 + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,50,, + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,61,0,61 + 1997-12-20T00:39:58.000000000,2,2694864431690786590,2,25,0,50 + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,61,2,25 + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,86,2,25 + 1998-12-20T00:39:57.000000000,3,18433805721903975440,1,,0,86 + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,98,2,25 + 1999-12-20T00:39:58.000000000,4,18433805721903975440,1,,0,98 + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,98,2,25 + 1999-12-20T00:39:58.000000000,5,18433805721903975440,1,,0,98 "###); } @@ -114,12 +114,12 @@ async fn test_lookup_self_string() { let last_sender_description = lookup(last_sender, last(Sent.description)) in { description: Received.description, last_sender, last_sender_description }").run_to_csv(&lookup_account_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,description,last_sender,last_sender_description - 1996-12-20T00:39:57.000000000,9223372036854775808,2694864431690786590,2,food,0,food - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,gas,0,gas - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,food,2,food - 1998-12-20T00:39:57.000000000,9223372036854775811,18433805721903975440,1,gas,0,gas - 1999-12-20T00:39:58.000000000,9223372036854775812,18433805721903975440,1,MOVIe,0,MOVIe - 1999-12-20T00:39:58.000000000,9223372036854775813,18433805721903975440,1,null_amount,0,null_amount + 1996-12-20T00:39:57.000000000,0,2694864431690786590,2,food,0,food + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,gas,0,gas + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,food,2,food + 1998-12-20T00:39:57.000000000,3,18433805721903975440,1,gas,0,gas + 1999-12-20T00:39:58.000000000,4,18433805721903975440,1,MOVIe,0,MOVIe + 1999-12-20T00:39:58.000000000,5,18433805721903975440,1,null_amount,0,null_amount "###); } @@ -129,17 +129,17 @@ async fn test_lookup_self_record() { let last_sender_sent = lookup(last(Received.to), Sent.description) in Sent | extend({ received_description: Received.description, last_sender, last_sender_sent })").run_to_csv(&lookup_account_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,received_description,last_sender,last_sender_sent,from,to,time,subsort,amount,description,order_time,code - 1996-12-20T00:39:57.000000000,9223372036854775808,2694864431690786590,2,food,0,,,,,,,,, - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,,,,0,2,1996-12-20T00:39:57.000000000,0,50,food,2005-12-19T16:39:57-08:00,5 - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,gas,0,gas,0,0,1997-12-20T00:39:57.000000000,1,11,gas,2001-12-19T16:39:57-08:00,6 - 1997-12-20T00:39:58.000000000,9223372036854775810,2694864431690786590,2,,0,,2,0,1997-12-20T00:39:58.000000000,2,25,food,2001-12-19T16:39:57-08:00,5 - 1997-12-20T00:39:58.000000000,9223372036854775810,11832085162654999889,0,food,2,,,,,,,,, - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,,2,,0,1,1998-12-20T00:39:57.000000000,3,25,gas,2003-12-19T16:39:57-08:00,6 - 1998-12-20T00:39:57.000000000,9223372036854775811,18433805721903975440,1,gas,0,,,,,,,,, - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,,2,,0,1,1999-12-20T00:39:58.000000000,4,12,MOVIe,2004-12-1,7 - 1999-12-20T00:39:58.000000000,9223372036854775812,18433805721903975440,1,MOVIe,0,,,,,,,,, - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,,2,,0,1,1999-12-20T00:39:58.000000000,5,,null_amount,2005-12-1, - 1999-12-20T00:39:58.000000000,9223372036854775813,18433805721903975440,1,null_amount,0,,,,,,,,, + 1996-12-20T00:39:57.000000000,0,2694864431690786590,2,food,0,,,,,,,,, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,,,,0,2,1996-12-20T00:39:57.000000000,0,50,food,2005-12-19T16:39:57-08:00,5 + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,gas,0,gas,0,0,1997-12-20T00:39:57.000000000,1,11,gas,2001-12-19T16:39:57-08:00,6 + 1997-12-20T00:39:58.000000000,2,2694864431690786590,2,,0,,2,0,1997-12-20T00:39:58.000000000,2,25,food,2001-12-19T16:39:57-08:00,5 + 1997-12-20T00:39:58.000000000,2,11832085162654999889,0,food,2,,,,,,,,, + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,,2,,0,1,1998-12-20T00:39:57.000000000,3,25,gas,2003-12-19T16:39:57-08:00,6 + 1998-12-20T00:39:57.000000000,3,18433805721903975440,1,gas,0,,,,,,,,, + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,,2,,0,1,1999-12-20T00:39:58.000000000,4,12,MOVIe,2004-12-1,7 + 1999-12-20T00:39:58.000000000,4,18433805721903975440,1,MOVIe,0,,,,,,,,, + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,,2,,0,1,1999-12-20T00:39:58.000000000,5,,null_amount,2005-12-1, + 1999-12-20T00:39:58.000000000,5,18433805721903975440,1,null_amount,0,,,,,,,,, "###); } @@ -148,12 +148,12 @@ async fn test_lookup_code_name() { insta::assert_snapshot!(QueryFixture::new("{ code: Sent.code, code_name: lookup(Sent.code, CodeName.name | last()) }").run_to_csv(&lookup_account_data_fixture().await).await .unwrap(), @r###" _time,_subsort,_key_hash,_key,code,code_name - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,5,FiveA - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,6,Six - 1997-12-20T00:39:58.000000000,9223372036854775810,2694864431690786590,2,5,FiveB - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,6,Six - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,7, - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,5,FiveA + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,6,Six + 1997-12-20T00:39:58.000000000,2,2694864431690786590,2,5,FiveB + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,6,Six + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,7, + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,, "###); } @@ -164,12 +164,12 @@ async fn test_lookup_code_name_wacky_unused() { { code: Sent.code, code_name: lookup(Sent.code, CodeName.name | last()) }").run_to_csv(&lookup_account_data_fixture().await).await .unwrap(), @r###" _time,_subsort,_key_hash,_key,code,code_name - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,5,FiveA - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,6,Six - 1997-12-20T00:39:58.000000000,9223372036854775810,2694864431690786590,2,5,FiveB - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,6,Six - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,7, - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,5,FiveA + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,6,Six + 1997-12-20T00:39:58.000000000,2,2694864431690786590,2,5,FiveB + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,6,Six + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,7, + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,, "###); } @@ -327,12 +327,12 @@ async fn test_lookup_with_key() { .run_to_csv(&lookup_account_data_fixture().await).await .unwrap(), @r###" _time,_subsort,_key_hash,_key,code_name - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,1 - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0, - 1997-12-20T00:39:58.000000000,9223372036854775810,2694864431690786590,2,1 - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0, - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0, - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0, + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,1 + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0, + 1997-12-20T00:39:58.000000000,2,2694864431690786590,2,1 + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0, + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0, + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0, "###); } @@ -450,12 +450,12 @@ async fn test_lookup_only_includes_primary_entites() { // there should only be 2 entities in the rows. insta::assert_snapshot!(QueryFixture::new("{ description: lookup(last(Sent.to), Received.description) }").run_to_csv(&lookup_account_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,description - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,food - 1997-12-20T00:39:57.000000000,9223372036854775809,11832085162654999889,0,gas - 1997-12-20T00:39:58.000000000,9223372036854775810,2694864431690786590,2,food - 1998-12-20T00:39:57.000000000,9223372036854775811,11832085162654999889,0,gas - 1999-12-20T00:39:58.000000000,9223372036854775812,11832085162654999889,0,MOVIe - 1999-12-20T00:39:58.000000000,9223372036854775813,11832085162654999889,0,null_amount + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,food + 1997-12-20T00:39:57.000000000,1,11832085162654999889,0,gas + 1997-12-20T00:39:58.000000000,2,2694864431690786590,2,food + 1998-12-20T00:39:57.000000000,3,11832085162654999889,0,gas + 1999-12-20T00:39:58.000000000,4,11832085162654999889,0,MOVIe + 1999-12-20T00:39:58.000000000,5,11832085162654999889,0,null_amount "###); } diff --git a/crates/sparrow-main/tests/e2e/math_tests.rs b/crates/sparrow-main/tests/e2e/math_tests.rs index f98fed8f8..374df37e8 100644 --- a/crates/sparrow-main/tests/e2e/math_tests.rs +++ b/crates/sparrow-main/tests/e2e/math_tests.rs @@ -7,12 +7,12 @@ use crate::QueryFixture; async fn test_i64_add() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, add: Numbers.m + Numbers.n}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,add - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,15 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,27 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,23 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,15 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,27 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,23 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -20,12 +20,12 @@ async fn test_i64_add() { async fn test_f64_add() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, add: Numbers.m + Numbers.n}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,add - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,15.2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,28.2 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,23.8 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,15.2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,28.2 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,23.8 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,, "###); } @@ -33,12 +33,12 @@ async fn test_f64_add() { async fn test_i64_add_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, add: Numbers.m + 1}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,add - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,6 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,25 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,18 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,13 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,6 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,25 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,18 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,13 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -46,12 +46,12 @@ async fn test_i64_add_literal() { async fn test_f64_add_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, add: Numbers.m + 1}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,add - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,6.2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,25.3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,18.6 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,13.4 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,6.2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,25.3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,18.6 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,13.4 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -59,12 +59,12 @@ async fn test_f64_add_literal() { async fn test_i64_sub() { insta::assert_snapshot!(QueryFixture::new("{ sub: Numbers.m - Numbers.n}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sub - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,-5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,21 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,11 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,-5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,21 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,11 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A, "###); } @@ -72,12 +72,12 @@ async fn test_i64_sub() { async fn test_f64_sub() { insta::assert_snapshot!(QueryFixture::new("{ sub: Numbers.m - Numbers.n}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sub - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,-4.8 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,20.400000000000002 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,11.400000000000002 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,-4.8 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,20.400000000000002 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,11.400000000000002 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A, "###); } @@ -85,12 +85,12 @@ async fn test_f64_sub() { async fn test_i64_sub_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, sub: Numbers.m - 1}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,sub - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,4 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,23 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,16 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,11 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,4 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,23 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,16 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,11 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -98,12 +98,12 @@ async fn test_i64_sub_literal() { async fn test_f64_sub_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, sub: Numbers.m - 1}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,sub - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,4.2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,23.3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,16.6 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,11.4 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,4.2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,23.3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,16.6 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,11.4 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -111,12 +111,12 @@ async fn test_f64_sub_literal() { async fn test_i64_mul() { insta::assert_snapshot!(QueryFixture::new("{ mul: Numbers.m * Numbers.n}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,mul - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,50 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,72 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,102 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,50 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,72 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,102 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A, "###); } @@ -124,12 +124,12 @@ async fn test_i64_mul() { async fn test_f64_mul() { insta::assert_snapshot!(QueryFixture::new("{ mul: Numbers.m * Numbers.n}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,mul - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,52.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,94.77 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,109.12000000000002 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,52.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,94.77 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,109.12000000000002 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A, "###); } @@ -137,12 +137,12 @@ async fn test_f64_mul() { async fn test_i64_mul_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, mul: Numbers.m * 2}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,mul - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,48 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,34 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,24 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,48 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,34 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,24 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -150,12 +150,12 @@ async fn test_i64_mul_literal() { async fn test_f64_mul_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, mul: Numbers.m * 2}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,mul - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.4 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,48.6 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,35.2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,24.8 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.4 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,48.6 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,35.2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,24.8 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -163,12 +163,12 @@ async fn test_f64_mul_literal() { async fn test_i64_div() { insta::assert_snapshot!(QueryFixture::new("{ div: Numbers.m / Numbers.n}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,div - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,8 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,8 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A, "###); } @@ -176,12 +176,12 @@ async fn test_i64_div() { async fn test_f64_div() { insta::assert_snapshot!(QueryFixture::new("{ div: Numbers.m / Numbers.n }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,div - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,0.52 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,6.230769230769231 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,2.838709677419355 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,0.52 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,6.230769230769231 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,2.838709677419355 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A, "###); } @@ -189,12 +189,12 @@ async fn test_f64_div() { async fn test_i64_div_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, div: Numbers.m / 2}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,div - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,12 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,8 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,6 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,12 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,8 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,6 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -202,12 +202,12 @@ async fn test_i64_div_literal() { async fn test_f64_div_literal() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, div: Numbers.m / 2}").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,div - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,2.6 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,12.15 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,8.8 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,6.2 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,2.6 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,12.15 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,8.8 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,6.2 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -215,12 +215,12 @@ async fn test_f64_div_literal() { async fn test_i64_neg() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, neg_m: -Numbers.m, neg_n: neg(Numbers.n) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,neg_m,neg_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,-5,-10 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,-24,-3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,-17,-6 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9,,-9 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,,-12, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,-5,-10 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,-24,-3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,-17,-6 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9,,-9 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,,-12, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,, "###); } @@ -228,12 +228,12 @@ async fn test_i64_neg() { async fn test_f64_neg() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, neg_m: -Numbers.m, neg_n: neg(Numbers.n) }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,neg_m,neg_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,-5.2,-10.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,-24.3,-3.9 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,-17.6,-6.2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25,,-9.25 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,,-12.4, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,-5.2,-10.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,-24.3,-3.9 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,-17.6,-6.2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25,,-9.25 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,,-12.4, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,, "###); } @@ -241,12 +241,12 @@ async fn test_f64_neg() { async fn test_i64_ceil() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, ceil_m: ceil(Numbers.m), ceil_n: ceil(Numbers.n) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,ceil_m,ceil_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,5,10 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,24,3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,17,6 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9,,9 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,,12, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,5,10 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,24,3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,17,6 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9,,9 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,,12, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,, "###); } @@ -254,12 +254,12 @@ async fn test_i64_ceil() { async fn test_f64_ceil() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, ceil_m: ceil(Numbers.m), ceil_n: ceil(Numbers.n) }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,ceil_m,ceil_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,6.0,10.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,25.0,4.0 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,18.0,7.0 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25,,10.0 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,,13.0, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,6.0,10.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,25.0,4.0 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,18.0,7.0 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25,,10.0 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,,13.0, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,, "###); } @@ -267,12 +267,12 @@ async fn test_f64_ceil() { async fn test_i64_round() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, round_m: round(Numbers.m), round_n: round(Numbers.n) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,round_m,round_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,5,10 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,24,3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,17,6 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9,,9 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,,12, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,5,10 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,24,3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,17,6 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9,,9 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,,12, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,, "###); } @@ -280,12 +280,12 @@ async fn test_i64_round() { async fn test_f64_round() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, round_m: round(Numbers.m), round_n: round(Numbers.n) }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,round_m,round_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,5.0,10.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,24.0,4.0 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,18.0,6.0 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25,,9.0 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,,12.0, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,5.0,10.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,24.0,4.0 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,18.0,6.0 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25,,9.0 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,,12.0, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,, "###); } @@ -293,12 +293,12 @@ async fn test_f64_round() { async fn test_i64_floor() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, floor_m: floor(Numbers.m), floor_n: floor(Numbers.n) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,floor_m,floor_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,5,10 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,24,3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,17,6 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9,,9 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,,12, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,5,10 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,24,3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,17,6 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9,,9 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,,12, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,, "###); } @@ -306,12 +306,12 @@ async fn test_i64_floor() { async fn test_f64_floor() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, n: Numbers.n, floor_m: floor(Numbers.m), floor_n: floor(Numbers.n) }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,floor_m,floor_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,5.0,10.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,24.0,3.0 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,17.0,6.0 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25,,9.0 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,,12.0, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,5.0,10.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,24.0,3.0 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,17.0,6.0 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25,,9.0 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,,12.0, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,, "###); } @@ -323,12 +323,12 @@ async fn test_i64_zip_min() { , zip_min_2_n: zip_min(2, Numbers.n) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,zip_min,zip_min_2_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,5,2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,3,2 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,6,2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9,,2 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,5,2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,3,2 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,6,2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9,,2 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,, "###); } @@ -340,12 +340,12 @@ async fn test_f64_zip_min() { , zip_min_2_n: zip_min(2, Numbers.n) }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,zip_min,zip_min_2_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,5.2,2.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,3.9,2.0 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,6.2,2.0 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25,,2.0 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,5.2,2.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,3.9,2.0 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,6.2,2.0 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25,,2.0 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,, "###); } @@ -357,12 +357,12 @@ async fn test_i64_zip_max() { , zip_max_2_n: zip_max(2, Numbers.n) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,zip_max,zip_max_2_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,10,10 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,24,3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,17,6 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9,,9 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,10,10 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,24,3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,17,6 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9,,9 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,, "###); } @@ -374,12 +374,12 @@ async fn test_f64_zip_max() { , zip_max_2_n: zip_max(2, Numbers.n) }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,zip_max,zip_max_2_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,10.0,10.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,24.3,3.9 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,17.6,6.2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25,,9.25 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,10.0,10.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,24.3,3.9 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,17.6,6.2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25,,9.25 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,, "###); } @@ -392,12 +392,12 @@ async fn test_i64_powf() { , pow_n_2: powf(Numbers.n, 2) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,pow_m_n,pow_2_m,pow_n_2 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,9765625.0,32.0,100.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,13824.0,16777216.0,9.0 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,24137569.0,131072.0,36.0 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9,,,81.0 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,,,4096.0, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,9765625.0,32.0,100.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,13824.0,16777216.0,9.0 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,24137569.0,131072.0,36.0 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9,,,81.0 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,,,4096.0, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,,, "###); } @@ -410,12 +410,12 @@ async fn test_f64_powf() { , pow_n_2: powf(Numbers.n, 2) }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,pow_m_n,pow_2_m,pow_n_2 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,14455510.594905708,36.75834735990512,100.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,253433.76006548494,20655175.749880955,15.209999999999999 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,52744665.25889734,198668.0018056511,38.440000000000005 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25,,,85.5625 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,,,5404.7044025257765, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,14455510.594905708,36.75834735990512,100.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,253433.76006548494,20655175.749880955,15.209999999999999 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,52744665.25889734,198668.0018056511,38.440000000000005 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25,,,85.5625 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,,,5404.7044025257765, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,,, "###); } @@ -427,12 +427,12 @@ async fn test_i64_exp() { , exp_n: exp(Numbers.n) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,exp_m,exp_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,148.4131591025766,22026.465794806718 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,2.648912212984347e10,20.085536923187668 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,24154952.7535753,403.4287934927351 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9,,8103.083927575384 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,,162754.79141900392, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,148.4131591025766,22026.465794806718 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,2.648912212984347e10,20.085536923187668 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,24154952.7535753,403.4287934927351 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9,,8103.083927575384 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,,162754.79141900392, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,, "###); } @@ -444,12 +444,12 @@ async fn test_f64_exp() { , exp_n: exp(Numbers.n) }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,exp_m,exp_n - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,181.27224187515122,22026.465794806718 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,3.575657481192565e10,49.40244910553017 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,44013193.53483411,492.7490410932563 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25,,10404.565716560723 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,,242801.61749832364, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,181.27224187515122,22026.465794806718 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,3.575657481192565e10,49.40244910553017 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,44013193.53483411,492.7490410932563 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25,,10404.565716560723 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,,242801.61749832364, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,, "###); } @@ -463,12 +463,12 @@ async fn test_i64_clamp() { , clamp_m_n_max: clamp(Numbers.m, max = Numbers.n) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,clamp_m_2_n,clamp_m_2_10,clamp_m_n_min,clamp_m_n_max - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,10,5,5,10,5 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,3,3,10,24,3 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,6,6,10,17,6 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9,,,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,,12,10,12,12 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,10,5,5,10,5 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,3,3,10,24,3 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,6,6,10,17,6 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9,,,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,,12,10,12,12 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,,,, "###); } @@ -482,12 +482,12 @@ async fn test_f64_clamp() { , clamp_m_n_max: clamp(Numbers.m, max = Numbers.n) }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,n,clamp_m_2_n,clamp_m_2_10,clamp_m_n_min,clamp_m_n_max - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,10.0,5.2,5.2,10.0,5.2 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,3.9,3.9,10.2,24.3,3.9 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,6.2,6.2,10.2,17.6,6.2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,9.25,,,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,,12.4,10.2,12.4,12.4 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,,,,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,10.0,5.2,5.2,10.0,5.2 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,3.9,3.9,10.2,24.3,3.9 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,6.2,6.2,10.2,17.6,6.2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,9.25,,,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,,12.4,10.2,12.4,12.4 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,,,,, "###); } @@ -495,12 +495,12 @@ async fn test_f64_clamp() { async fn test_f64_sqrt() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, sqrt: sqrt(Numbers.m) }").run_to_csv(&f64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,sqrt - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5.2,2.280350850198276 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24.3,4.929503017546495 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17.6,4.1952353926806065 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.4,3.521363372331802 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5.2,2.280350850198276 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24.3,4.929503017546495 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17.6,4.1952353926806065 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.4,3.521363372331802 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } @@ -508,11 +508,11 @@ async fn test_f64_sqrt() { async fn test_i64_sqrt() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, sqrt: sqrt(Numbers.m) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,sqrt - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,2.23606797749979 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24,4.898979485566356 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,4.123105625617661 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,3.4641016151377544 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,2.23606797749979 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24,4.898979485566356 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,4.123105625617661 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,3.4641016151377544 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,, "###); } diff --git a/crates/sparrow-main/tests/e2e/multiple_tables.rs b/crates/sparrow-main/tests/e2e/multiple_tables.rs index aee311dbf..4cb330eaa 100644 --- a/crates/sparrow-main/tests/e2e/multiple_tables.rs +++ b/crates/sparrow-main/tests/e2e/multiple_tables.rs @@ -148,13 +148,13 @@ async fn test_identical_table_join() { .run_to_csv(&test_data).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,t1_val,t2_val,sum - 2022-01-03T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1,1,2 - 2022-01-03T01:39:57.000000000,9223372036854775808,2867199309159137213,B,10,10,20 - 2022-01-03T02:39:57.000000000,9223372036854775808,12960666915911099378,A,2,2,4 - 2022-01-03T03:39:57.000000000,9223372036854775808,2867199309159137213,B,20,20,40 - 2022-01-03T04:39:57.000000000,9223372036854775808,2521269998124177631,C,100,100,200 - 2022-01-03T05:39:57.000000000,9223372036854775808,2521269998124177631,C,200,200,400 - 2022-01-03T06:39:57.000000000,9223372036854775808,12960666915911099378,A,3,3,6 + 2022-01-03T00:39:57.000000000,0,12960666915911099378,A,1,1,2 + 2022-01-03T01:39:57.000000000,0,2867199309159137213,B,10,10,20 + 2022-01-03T02:39:57.000000000,0,12960666915911099378,A,2,2,4 + 2022-01-03T03:39:57.000000000,0,2867199309159137213,B,20,20,40 + 2022-01-03T04:39:57.000000000,0,2521269998124177631,C,100,100,200 + 2022-01-03T05:39:57.000000000,0,2521269998124177631,C,200,200,400 + 2022-01-03T06:39:57.000000000,0,12960666915911099378,A,3,3,6 "### ); } @@ -168,20 +168,20 @@ async fn test_tables_different_dates() { .run_to_csv(&purchase_tables_two_days().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,t1_val,t2_val,sum - 2022-01-03T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1,, - 2022-01-03T01:39:57.000000000,9223372036854775808,2867199309159137213,B,10,, - 2022-01-03T02:39:57.000000000,9223372036854775808,12960666915911099378,A,2,, - 2022-01-03T03:39:57.000000000,9223372036854775808,2867199309159137213,B,20,, - 2022-01-03T04:39:57.000000000,9223372036854775808,2521269998124177631,C,100,, - 2022-01-03T05:39:57.000000000,9223372036854775808,2521269998124177631,C,200,, - 2022-01-03T06:39:57.000000000,9223372036854775808,12960666915911099378,A,3,, - 2022-01-03T18:39:57.000000000,9223372036854775808,2867199309159137213,B,,5, - 2022-01-03T20:39:57.000000000,9223372036854775808,12960666915911099378,A,,4, - 2022-01-03T21:39:57.000000000,9223372036854775808,2867199309159137213,B,,15, - 2022-01-03T22:39:57.000000000,9223372036854775808,12960666915911099378,A,,8, - 2022-01-03T23:39:57.000000000,9223372036854775808,2521269998124177631,C,,100, - 2022-01-04T00:39:57.000000000,9223372036854775808,2521269998124177631,C,,200, - 2022-01-04T01:39:57.000000000,9223372036854775808,2867199309159137213,B,,20, + 2022-01-03T00:39:57.000000000,0,12960666915911099378,A,1,, + 2022-01-03T01:39:57.000000000,0,2867199309159137213,B,10,, + 2022-01-03T02:39:57.000000000,0,12960666915911099378,A,2,, + 2022-01-03T03:39:57.000000000,0,2867199309159137213,B,20,, + 2022-01-03T04:39:57.000000000,0,2521269998124177631,C,100,, + 2022-01-03T05:39:57.000000000,0,2521269998124177631,C,200,, + 2022-01-03T06:39:57.000000000,0,12960666915911099378,A,3,, + 2022-01-03T18:39:57.000000000,0,2867199309159137213,B,,5, + 2022-01-03T20:39:57.000000000,0,12960666915911099378,A,,4, + 2022-01-03T21:39:57.000000000,0,2867199309159137213,B,,15, + 2022-01-03T22:39:57.000000000,0,12960666915911099378,A,,8, + 2022-01-03T23:39:57.000000000,0,2521269998124177631,C,,100, + 2022-01-04T00:39:57.000000000,0,2521269998124177631,C,,200, + 2022-01-04T01:39:57.000000000,0,2867199309159137213,B,,20, "###); } @@ -194,20 +194,20 @@ async fn test_tables_no_overlapping_keys() { .run_to_csv(&purchase_tables_different_keys().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,t1_val,t2_val,sum - 2022-01-03T00:39:57.000000000,9223372036854775808,5844668342709334339,X,,1, - 2022-01-03T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1,, - 2022-01-03T01:39:57.000000000,9223372036854775808,2867199309159137213,B,10,, - 2022-01-03T01:39:57.000000000,9223372036854775808,8493950773958210388,Y,,10, - 2022-01-03T02:39:57.000000000,9223372036854775808,5844668342709334339,X,,2, - 2022-01-03T02:39:57.000000000,9223372036854775808,12960666915911099378,A,2,, - 2022-01-03T03:39:57.000000000,9223372036854775808,2867199309159137213,B,20,, - 2022-01-03T03:39:57.000000000,9223372036854775808,8493950773958210388,Y,,20, - 2022-01-03T04:39:57.000000000,9223372036854775808,2521269998124177631,C,100,, - 2022-01-03T04:39:57.000000000,9223372036854775808,5050198837546418057,Z,,100, - 2022-01-03T05:39:57.000000000,9223372036854775808,2521269998124177631,C,200,, - 2022-01-03T05:39:57.000000000,9223372036854775808,5050198837546418057,Z,,200, - 2022-01-03T06:39:57.000000000,9223372036854775808,5844668342709334339,X,,3, - 2022-01-03T06:39:57.000000000,9223372036854775808,12960666915911099378,A,3,, + 2022-01-03T00:39:57.000000000,0,5844668342709334339,X,,1, + 2022-01-03T00:39:57.000000000,0,12960666915911099378,A,1,, + 2022-01-03T01:39:57.000000000,0,2867199309159137213,B,10,, + 2022-01-03T01:39:57.000000000,0,8493950773958210388,Y,,10, + 2022-01-03T02:39:57.000000000,0,5844668342709334339,X,,2, + 2022-01-03T02:39:57.000000000,0,12960666915911099378,A,2,, + 2022-01-03T03:39:57.000000000,0,2867199309159137213,B,20,, + 2022-01-03T03:39:57.000000000,0,8493950773958210388,Y,,20, + 2022-01-03T04:39:57.000000000,0,2521269998124177631,C,100,, + 2022-01-03T04:39:57.000000000,0,5050198837546418057,Z,,100, + 2022-01-03T05:39:57.000000000,0,2521269998124177631,C,200,, + 2022-01-03T05:39:57.000000000,0,5050198837546418057,Z,,200, + 2022-01-03T06:39:57.000000000,0,5844668342709334339,X,,3, + 2022-01-03T06:39:57.000000000,0,12960666915911099378,A,3,, "###); } @@ -220,17 +220,17 @@ async fn test_tables_overlapping_keys() { .run_to_csv(&purchase_tables_overlapping_keys().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,t1_val,t2_val,sum - 2022-01-03T00:39:57.000000000,9223372036854775808,5844668342709334339,X,,1, - 2022-01-03T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1,, - 2022-01-03T01:39:57.000000000,9223372036854775808,2867199309159137213,B,10,10,20 - 2022-01-03T02:39:57.000000000,9223372036854775808,12960666915911099378,A,2,2,4 - 2022-01-03T03:39:57.000000000,9223372036854775808,2867199309159137213,B,20,, - 2022-01-03T03:39:57.000000000,9223372036854775808,8493950773958210388,Y,,20, - 2022-01-03T04:39:57.000000000,9223372036854775808,2521269998124177631,C,100,100,200 - 2022-01-03T05:39:57.000000000,9223372036854775808,2521269998124177631,C,200,, - 2022-01-03T05:39:57.000000000,9223372036854775808,5050198837546418057,Z,,200, - 2022-01-03T06:39:57.000000000,9223372036854775808,5844668342709334339,X,,3, - 2022-01-03T06:39:57.000000000,9223372036854775808,12960666915911099378,A,3,, + 2022-01-03T00:39:57.000000000,0,5844668342709334339,X,,1, + 2022-01-03T00:39:57.000000000,0,12960666915911099378,A,1,, + 2022-01-03T01:39:57.000000000,0,2867199309159137213,B,10,10,20 + 2022-01-03T02:39:57.000000000,0,12960666915911099378,A,2,2,4 + 2022-01-03T03:39:57.000000000,0,2867199309159137213,B,20,, + 2022-01-03T03:39:57.000000000,0,8493950773958210388,Y,,20, + 2022-01-03T04:39:57.000000000,0,2521269998124177631,C,100,100,200 + 2022-01-03T05:39:57.000000000,0,2521269998124177631,C,200,, + 2022-01-03T05:39:57.000000000,0,5050198837546418057,Z,,200, + 2022-01-03T06:39:57.000000000,0,5844668342709334339,X,,3, + 2022-01-03T06:39:57.000000000,0,12960666915911099378,A,3,, "###); } @@ -243,21 +243,21 @@ async fn test_tables_superset() { .run_to_csv(&purchase_tables_superset().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,t1_val,t2_val,sum - 2022-01-03T00:37:57.000000000,9223372036854775808,2867199309159137213,B,,1, - 2022-01-03T00:38:57.000000000,9223372036854775808,12960666915911099378,A,,1, - 2022-01-03T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1,1,2 - 2022-01-03T00:39:58.000000000,9223372036854775808,12960666915911099378,A,,666, - 2022-01-03T01:39:57.000000000,9223372036854775808,2867199309159137213,B,10,10,20 - 2022-01-03T01:40:57.000000000,9223372036854775808,2867199309159137213,B,,1000, - 2022-01-03T02:38:57.000000000,9223372036854775808,12960666915911099378,A,,777, - 2022-01-03T02:39:57.000000000,9223372036854775808,12960666915911099378,A,2,2,4 - 2022-01-03T02:44:57.000000000,9223372036854775808,12960666915911099378,A,,888, - 2022-01-03T03:39:57.000000000,9223372036854775808,2867199309159137213,B,20,20,40 - 2022-01-03T04:39:57.000000000,9223372036854775808,2521269998124177631,C,100,100,200 - 2022-01-03T05:39:57.000000000,9223372036854775808,2521269998124177631,C,200,200,400 - 2022-01-03T06:39:57.000000000,9223372036854775808,12960666915911099378,A,3,3,6 - 2022-01-03T06:43:57.000000000,9223372036854775808,2867199309159137213,B,,3, - 2022-01-03T06:45:57.000000000,9223372036854775808,2521269998124177631,C,,3, + 2022-01-03T00:37:57.000000000,0,2867199309159137213,B,,1, + 2022-01-03T00:38:57.000000000,0,12960666915911099378,A,,1, + 2022-01-03T00:39:57.000000000,0,12960666915911099378,A,1,1,2 + 2022-01-03T00:39:58.000000000,0,12960666915911099378,A,,666, + 2022-01-03T01:39:57.000000000,0,2867199309159137213,B,10,10,20 + 2022-01-03T01:40:57.000000000,0,2867199309159137213,B,,1000, + 2022-01-03T02:38:57.000000000,0,12960666915911099378,A,,777, + 2022-01-03T02:39:57.000000000,0,12960666915911099378,A,2,2,4 + 2022-01-03T02:44:57.000000000,0,12960666915911099378,A,,888, + 2022-01-03T03:39:57.000000000,0,2867199309159137213,B,20,20,40 + 2022-01-03T04:39:57.000000000,0,2521269998124177631,C,100,100,200 + 2022-01-03T05:39:57.000000000,0,2521269998124177631,C,200,200,400 + 2022-01-03T06:39:57.000000000,0,12960666915911099378,A,3,3,6 + 2022-01-03T06:43:57.000000000,0,2867199309159137213,B,,3, + 2022-01-03T06:45:57.000000000,0,2521269998124177631,C,,3, "###); } @@ -275,13 +275,13 @@ async fn test_triple_add_same_table() { .run_to_csv(&test_data).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,t1_val,t2_val,sum - 2022-01-03T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1,1,3 - 2022-01-03T01:39:57.000000000,9223372036854775808,2867199309159137213,B,10,10,30 - 2022-01-03T02:39:57.000000000,9223372036854775808,12960666915911099378,A,2,2,6 - 2022-01-03T03:39:57.000000000,9223372036854775808,2867199309159137213,B,20,20,60 - 2022-01-03T04:39:57.000000000,9223372036854775808,2521269998124177631,C,100,100,300 - 2022-01-03T05:39:57.000000000,9223372036854775808,2521269998124177631,C,200,200,600 - 2022-01-03T06:39:57.000000000,9223372036854775808,12960666915911099378,A,3,3,9 + 2022-01-03T00:39:57.000000000,0,12960666915911099378,A,1,1,3 + 2022-01-03T01:39:57.000000000,0,2867199309159137213,B,10,10,30 + 2022-01-03T02:39:57.000000000,0,12960666915911099378,A,2,2,6 + 2022-01-03T03:39:57.000000000,0,2867199309159137213,B,20,20,60 + 2022-01-03T04:39:57.000000000,0,2521269998124177631,C,100,100,300 + 2022-01-03T05:39:57.000000000,0,2521269998124177631,C,200,200,600 + 2022-01-03T06:39:57.000000000,0,12960666915911099378,A,3,3,9 "###); } @@ -303,25 +303,25 @@ async fn test_triple_add_different_tables() { .run_to_csv(&test_data).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,t1_val,t2_val,t3_val,sum - 2022-01-03T00:37:57.000000000,9223372036854775808,2867199309159137213,B,,1,, - 2022-01-03T00:38:57.000000000,9223372036854775808,12960666915911099378,A,,1,, - 2022-01-03T00:39:57.000000000,9223372036854775808,5844668342709334339,X,,,1, - 2022-01-03T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1,1,, - 2022-01-03T00:39:58.000000000,9223372036854775808,12960666915911099378,A,,666,, - 2022-01-03T01:39:57.000000000,9223372036854775808,2867199309159137213,B,10,10,10,30 - 2022-01-03T01:40:57.000000000,9223372036854775808,2867199309159137213,B,,1000,, - 2022-01-03T02:38:57.000000000,9223372036854775808,12960666915911099378,A,,777,, - 2022-01-03T02:39:57.000000000,9223372036854775808,12960666915911099378,A,2,2,2,6 - 2022-01-03T02:44:57.000000000,9223372036854775808,12960666915911099378,A,,888,, - 2022-01-03T03:39:57.000000000,9223372036854775808,2867199309159137213,B,20,20,, - 2022-01-03T03:39:57.000000000,9223372036854775808,8493950773958210388,Y,,,20, - 2022-01-03T04:39:57.000000000,9223372036854775808,2521269998124177631,C,100,100,100,300 - 2022-01-03T05:39:57.000000000,9223372036854775808,2521269998124177631,C,200,200,, - 2022-01-03T05:39:57.000000000,9223372036854775808,5050198837546418057,Z,,,200, - 2022-01-03T06:39:57.000000000,9223372036854775808,5844668342709334339,X,,,3, - 2022-01-03T06:39:57.000000000,9223372036854775808,12960666915911099378,A,3,3,, - 2022-01-03T06:43:57.000000000,9223372036854775808,2867199309159137213,B,,3,, - 2022-01-03T06:45:57.000000000,9223372036854775808,2521269998124177631,C,,3,, + 2022-01-03T00:37:57.000000000,0,2867199309159137213,B,,1,, + 2022-01-03T00:38:57.000000000,0,12960666915911099378,A,,1,, + 2022-01-03T00:39:57.000000000,0,5844668342709334339,X,,,1, + 2022-01-03T00:39:57.000000000,0,12960666915911099378,A,1,1,, + 2022-01-03T00:39:58.000000000,0,12960666915911099378,A,,666,, + 2022-01-03T01:39:57.000000000,0,2867199309159137213,B,10,10,10,30 + 2022-01-03T01:40:57.000000000,0,2867199309159137213,B,,1000,, + 2022-01-03T02:38:57.000000000,0,12960666915911099378,A,,777,, + 2022-01-03T02:39:57.000000000,0,12960666915911099378,A,2,2,2,6 + 2022-01-03T02:44:57.000000000,0,12960666915911099378,A,,888,, + 2022-01-03T03:39:57.000000000,0,2867199309159137213,B,20,20,, + 2022-01-03T03:39:57.000000000,0,8493950773958210388,Y,,,20, + 2022-01-03T04:39:57.000000000,0,2521269998124177631,C,100,100,100,300 + 2022-01-03T05:39:57.000000000,0,2521269998124177631,C,200,200,, + 2022-01-03T05:39:57.000000000,0,5050198837546418057,Z,,,200, + 2022-01-03T06:39:57.000000000,0,5844668342709334339,X,,,3, + 2022-01-03T06:39:57.000000000,0,12960666915911099378,A,3,3,, + 2022-01-03T06:43:57.000000000,0,2867199309159137213,B,,3,, + 2022-01-03T06:45:57.000000000,0,2521269998124177631,C,,3,, "###); } @@ -343,24 +343,24 @@ async fn test_3_tables_with_3_additions_with_1_common_operand() { .run_to_csv(&test_data).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,t1_val,t2_val,t3_val,sum - 2022-01-03T00:37:57.000000000,9223372036854775808,2867199309159137213,B,,1,, - 2022-01-03T00:38:57.000000000,9223372036854775808,12960666915911099378,A,,1,, - 2022-01-03T00:39:57.000000000,9223372036854775808,5844668342709334339,X,,,1, - 2022-01-03T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1,1,, - 2022-01-03T00:39:58.000000000,9223372036854775808,12960666915911099378,A,,666,, - 2022-01-03T01:39:57.000000000,9223372036854775808,2867199309159137213,B,10,10,10,40 - 2022-01-03T01:40:57.000000000,9223372036854775808,2867199309159137213,B,,1000,, - 2022-01-03T02:38:57.000000000,9223372036854775808,12960666915911099378,A,,777,, - 2022-01-03T02:39:57.000000000,9223372036854775808,12960666915911099378,A,2,2,2,8 - 2022-01-03T02:44:57.000000000,9223372036854775808,12960666915911099378,A,,888,, - 2022-01-03T03:39:57.000000000,9223372036854775808,2867199309159137213,B,20,20,, - 2022-01-03T03:39:57.000000000,9223372036854775808,8493950773958210388,Y,,,20, - 2022-01-03T04:39:57.000000000,9223372036854775808,2521269998124177631,C,100,100,100,400 - 2022-01-03T05:39:57.000000000,9223372036854775808,2521269998124177631,C,200,200,, - 2022-01-03T05:39:57.000000000,9223372036854775808,5050198837546418057,Z,,,200, - 2022-01-03T06:39:57.000000000,9223372036854775808,5844668342709334339,X,,,3, - 2022-01-03T06:39:57.000000000,9223372036854775808,12960666915911099378,A,3,3,, - 2022-01-03T06:43:57.000000000,9223372036854775808,2867199309159137213,B,,3,, - 2022-01-03T06:45:57.000000000,9223372036854775808,2521269998124177631,C,,3,, + 2022-01-03T00:37:57.000000000,0,2867199309159137213,B,,1,, + 2022-01-03T00:38:57.000000000,0,12960666915911099378,A,,1,, + 2022-01-03T00:39:57.000000000,0,5844668342709334339,X,,,1, + 2022-01-03T00:39:57.000000000,0,12960666915911099378,A,1,1,, + 2022-01-03T00:39:58.000000000,0,12960666915911099378,A,,666,, + 2022-01-03T01:39:57.000000000,0,2867199309159137213,B,10,10,10,40 + 2022-01-03T01:40:57.000000000,0,2867199309159137213,B,,1000,, + 2022-01-03T02:38:57.000000000,0,12960666915911099378,A,,777,, + 2022-01-03T02:39:57.000000000,0,12960666915911099378,A,2,2,2,8 + 2022-01-03T02:44:57.000000000,0,12960666915911099378,A,,888,, + 2022-01-03T03:39:57.000000000,0,2867199309159137213,B,20,20,, + 2022-01-03T03:39:57.000000000,0,8493950773958210388,Y,,,20, + 2022-01-03T04:39:57.000000000,0,2521269998124177631,C,100,100,100,400 + 2022-01-03T05:39:57.000000000,0,2521269998124177631,C,200,200,, + 2022-01-03T05:39:57.000000000,0,5050198837546418057,Z,,,200, + 2022-01-03T06:39:57.000000000,0,5844668342709334339,X,,,3, + 2022-01-03T06:39:57.000000000,0,12960666915911099378,A,3,3,, + 2022-01-03T06:43:57.000000000,0,2867199309159137213,B,,3,, + 2022-01-03T06:45:57.000000000,0,2521269998124177631,C,,3,, "###); } diff --git a/crates/sparrow-main/tests/e2e/notebooks/event_data_tests.rs b/crates/sparrow-main/tests/e2e/notebooks/event_data_tests.rs index a21919171..cbf4dc74b 100644 --- a/crates/sparrow-main/tests/e2e/notebooks/event_data_tests.rs +++ b/crates/sparrow-main/tests/e2e/notebooks/event_data_tests.rs @@ -38,7 +38,7 @@ async fn test_initial_query() { // Regression test for a take on a null array assert_eq!(no_simplifier, simplifier); - insta::assert_snapshot!(no_simplifier, @"D50CA894B17571359A5E0F4527E3E26A2A74CC48CAC97065E9331E72"); + insta::assert_snapshot!(no_simplifier, @"CDEE886652EA1E2D00DEEBABB69BA127029600B931509CFB097A2D55"); } const EVENTS: &str = indoc! {" @@ -111,52 +111,52 @@ async fn test_events() { insta::assert_snapshot!(QueryFixture::new(EVENTS).run_to_csv(&sample_event_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,id,count_today,locale_score_sliding - 2020-10-27T16:03:28.331000000,9223372036854775808,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,1,1 - 2020-10-27T17:24:17.956000000,9223372036854775811,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,2,2 - 2020-10-27T17:24:17.967000000,9223372036854775813,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,3,3 - 2020-10-27T17:24:17.967000000,9223372036854775814,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,4,4 - 2020-10-27T17:25:45.242000000,9223372036854775829,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,5,5 - 2020-10-27T17:25:45.248000000,9223372036854775830,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,6,6 - 2020-10-27T17:25:53.717000000,9223372036854775839,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,7,7 - 2020-10-27T17:26:25.213000000,9223372036854775854,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,8,8 - 2020-10-27T17:26:35.816000000,9223372036854775855,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,9,9 - 2020-10-27T17:26:49.665000000,9223372036854775856,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,10,10 - 2020-10-27T17:29:35.525000000,9223372036854775857,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,11,11 - 2020-10-27T17:30:21.233000000,9223372036854775859,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,12,12 - 2020-10-27T17:32:36.646000000,9223372036854775860,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,13,13 - 2020-10-27T17:33:55.353000000,9223372036854775867,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,14,14 - 2020-10-27T17:34:03.546000000,9223372036854775868,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,15,15 - 2020-10-27T17:35:39.310000000,9223372036854775869,1279197888909376308,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,1,1 - 2020-10-27T17:35:39.311000000,9223372036854775870,1279197888909376308,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,2,2 - 2020-10-27T17:35:47.195000000,9223372036854775881,1279197888909376308,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,3,3 - 2020-10-27T17:35:47.201000000,9223372036854775882,1279197888909376308,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,4,4 - 2020-10-27T17:36:30.940000000,9223372036854775897,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,16,16 - 2020-10-27T17:36:31.894000000,9223372036854775898,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,1,1 - 2020-10-27T17:36:31.894000000,9223372036854775899,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,2,2 - 2020-10-27T17:36:31.895000000,9223372036854775900,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,3,3 - 2020-10-27T17:36:35.873000000,9223372036854775909,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,4,4 - 2020-10-27T17:36:36.031000000,9223372036854775918,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,5,5 - 2020-10-27T17:36:37.360000000,9223372036854775919,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,6,6 - 2020-10-27T17:36:37.453000000,9223372036854775920,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,7,7 - 2020-10-27T17:36:38.193000000,9223372036854775921,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,8,8 - 2020-10-27T17:36:38.259000000,9223372036854775922,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,9,9 - 2020-10-27T17:36:38.923000000,9223372036854775923,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,10,10 - 2020-10-27T17:36:39.012000000,9223372036854775924,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,11,11 - 2020-10-27T17:36:41.397000000,9223372036854775925,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,12,12 - 2020-10-27T17:36:41.916000000,9223372036854775926,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,13,13 - 2020-10-27T17:36:41.980000000,9223372036854775927,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,14,14 - 2020-10-27T17:36:42.939000000,9223372036854775928,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,15,15 - 2020-10-27T17:36:43.652000000,9223372036854775929,1279197888909376308,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,5,5 - 2020-10-27T17:36:43.862000000,9223372036854775930,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,16,16 - 2020-10-27T17:36:43.927000000,9223372036854775931,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,17,17 - 2020-10-27T17:36:47.068000000,9223372036854775934,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,18,18 - 2020-10-27T17:36:48.517000000,9223372036854775935,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,19,19 - 2020-10-27T17:36:52.086000000,9223372036854775938,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,20,20 - 2020-10-27T17:36:52.145000000,9223372036854775939,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,21,21 - 2020-10-27T17:36:52.548000000,9223372036854775940,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,22,22 - 2020-10-27T17:36:52.629000000,9223372036854775941,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,23,23 - 2020-10-27T17:36:57.093000000,9223372036854775942,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,17,17 - 2020-10-27T17:36:57.104000000,9223372036854775943,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,18,18 + 2020-10-27T16:03:28.331000000,0,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,1,1 + 2020-10-27T17:24:17.956000000,3,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,2,2 + 2020-10-27T17:24:17.967000000,5,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,3,3 + 2020-10-27T17:24:17.967000000,6,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,4,4 + 2020-10-27T17:25:45.242000000,21,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,5,5 + 2020-10-27T17:25:45.248000000,22,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,6,6 + 2020-10-27T17:25:53.717000000,31,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,7,7 + 2020-10-27T17:26:25.213000000,46,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,8,8 + 2020-10-27T17:26:35.816000000,47,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,9,9 + 2020-10-27T17:26:49.665000000,48,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,10,10 + 2020-10-27T17:29:35.525000000,49,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,11,11 + 2020-10-27T17:30:21.233000000,51,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,12,12 + 2020-10-27T17:32:36.646000000,52,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,13,13 + 2020-10-27T17:33:55.353000000,59,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,14,14 + 2020-10-27T17:34:03.546000000,60,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,15,15 + 2020-10-27T17:35:39.310000000,61,1279197888909376308,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,1,1 + 2020-10-27T17:35:39.311000000,62,1279197888909376308,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,2,2 + 2020-10-27T17:35:47.195000000,73,1279197888909376308,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,3,3 + 2020-10-27T17:35:47.201000000,74,1279197888909376308,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,4,4 + 2020-10-27T17:36:30.940000000,89,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,16,16 + 2020-10-27T17:36:31.894000000,90,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,1,1 + 2020-10-27T17:36:31.894000000,91,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,2,2 + 2020-10-27T17:36:31.895000000,92,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,3,3 + 2020-10-27T17:36:35.873000000,101,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,4,4 + 2020-10-27T17:36:36.031000000,110,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,5,5 + 2020-10-27T17:36:37.360000000,111,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,6,6 + 2020-10-27T17:36:37.453000000,112,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,7,7 + 2020-10-27T17:36:38.193000000,113,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,8,8 + 2020-10-27T17:36:38.259000000,114,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,9,9 + 2020-10-27T17:36:38.923000000,115,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,10,10 + 2020-10-27T17:36:39.012000000,116,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,11,11 + 2020-10-27T17:36:41.397000000,117,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,12,12 + 2020-10-27T17:36:41.916000000,118,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,13,13 + 2020-10-27T17:36:41.980000000,119,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,14,14 + 2020-10-27T17:36:42.939000000,120,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,15,15 + 2020-10-27T17:36:43.652000000,121,1279197888909376308,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,5,5 + 2020-10-27T17:36:43.862000000,122,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,16,16 + 2020-10-27T17:36:43.927000000,123,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,17,17 + 2020-10-27T17:36:47.068000000,126,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,18,18 + 2020-10-27T17:36:48.517000000,127,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,19,19 + 2020-10-27T17:36:52.086000000,130,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,20,20 + 2020-10-27T17:36:52.145000000,131,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,21,21 + 2020-10-27T17:36:52.548000000,132,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,22,22 + 2020-10-27T17:36:52.629000000,133,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,23,23 + 2020-10-27T17:36:57.093000000,134,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,17,17 + 2020-10-27T17:36:57.104000000,135,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,18,18 "###); } @@ -176,15 +176,15 @@ async fn test_page_event() { insta::assert_snapshot!(QueryFixture::new(PAGE_EVENTS).run_to_csv(&sample_event_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,id,locale_score_sliding - 2020-10-27T16:03:28.331000000,9223372036854775808,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,2.0 - 2020-10-27T17:24:17.967000000,9223372036854775813,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,1.5 - 2020-10-27T17:25:45.242000000,9223372036854775829,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,1.0 - 2020-10-27T17:25:53.717000000,9223372036854775839,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,0.75 - 2020-10-27T17:35:39.310000000,9223372036854775869,1279197888909376308,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,0.0 - 2020-10-27T17:35:47.195000000,9223372036854775881,1279197888909376308,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,0.0 - 2020-10-27T17:36:31.894000000,9223372036854775898,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0.0 - 2020-10-27T17:36:31.894000000,9223372036854775899,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0.0 - 2020-10-27T17:36:57.093000000,9223372036854775942,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,0.8 + 2020-10-27T16:03:28.331000000,0,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,2.0 + 2020-10-27T17:24:17.967000000,5,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,1.5 + 2020-10-27T17:25:45.242000000,21,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,1.0 + 2020-10-27T17:25:53.717000000,31,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,0.75 + 2020-10-27T17:35:39.310000000,61,1279197888909376308,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,0.0 + 2020-10-27T17:35:47.195000000,73,1279197888909376308,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,02b9152e-3b25-45cc-b7bb-0d8f98bf7524,0.0 + 2020-10-27T17:36:31.894000000,90,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0.0 + 2020-10-27T17:36:31.894000000,91,17703029354039803950,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0b00083c-5c1e-47f5-abba-f89b12ae3cf4,0.0 + 2020-10-27T17:36:57.093000000,134,17552223493047837804,8a16beda-c07a-4625-a805-2d28f5934107,8a16beda-c07a-4625-a805-2d28f5934107,0.8 "###); } @@ -248,7 +248,7 @@ async fn test_multiple_distinct_partitions() { insta::assert_snapshot!( hash, - @"F79E4F750EFA4A66D751CE4EE52A7B460FC6B9C4C8D8E09B90DCF464" + @"ABED6CADF1ABD5F2F3E10888E4365F32835D3A401E36F269C743E8D9" ); } diff --git a/crates/sparrow-main/tests/e2e/parquet_tests.rs b/crates/sparrow-main/tests/e2e/parquet_tests.rs index 8e9308718..ec0e5e17c 100644 --- a/crates/sparrow-main/tests/e2e/parquet_tests.rs +++ b/crates/sparrow-main/tests/e2e/parquet_tests.rs @@ -146,20 +146,20 @@ async fn test_multi_file_purchases() { }").run_to_csv(&data_fixture).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,entity,max_amount - 2020-01-01T00:00:00.000000000,9223372036854775808,4674756217206002200,karen,2020-01-01T00:00:00.000000000,karen,9 - 2020-01-01T00:00:00.000000000,9223372036854775809,14576041771120212628,patrick,2020-01-01T00:00:00.000000000,patrick,3 - 2020-01-02T00:00:00.000000000,9223372036854775810,4674756217206002200,karen,2020-01-02T00:00:00.000000000,karen,9 - 2020-01-02T00:00:00.000000000,9223372036854775811,14576041771120212628,patrick,2020-01-02T00:00:00.000000000,patrick,5 - 2020-01-03T00:00:00.000000000,9223372036854775812,4674756217206002200,karen,2020-01-03T00:00:00.000000000,karen,9 - 2020-01-03T00:00:00.000000000,9223372036854775813,14576041771120212628,patrick,2020-01-03T00:00:00.000000000,patrick,12 - 2020-01-04T00:00:00.000000000,9223372036854775814,14576041771120212628,patrick,2020-01-04T00:00:00.000000000,patrick,5000 - 2020-01-04T00:00:00.000000000,9223372036854775815,4674756217206002200,karen,2020-01-04T00:00:00.000000000,karen,9 - 2020-01-05T00:00:00.000000000,9223372036854775816,4674756217206002200,karen,2020-01-05T00:00:00.000000000,karen,9 - 2020-01-05T00:00:00.000000000,9223372036854775817,14576041771120212628,patrick,2020-01-05T00:00:00.000000000,patrick,5000 - 2020-01-06T00:00:00.000000000,9223372036854775808,14576041771120212628,patrick,2020-01-06T00:00:00.000000000,patrick,5000 - 2020-01-06T00:00:00.000000000,9223372036854775809,6566809397636161383,spongebob,2020-01-06T00:00:00.000000000,spongebob,7 - 2020-01-07T00:00:00.000000000,9223372036854775810,6566809397636161383,spongebob,2020-01-07T00:00:00.000000000,spongebob,34 - 2020-01-08T00:00:00.000000000,9223372036854775811,4674756217206002200,karen,2020-01-08T00:00:00.000000000,karen,9 - 2020-01-08T00:00:00.000000000,9223372036854775812,14576041771120212628,patrick,2020-01-08T00:00:00.000000000,patrick,5000 + 2020-01-01T00:00:00.000000000,0,4674756217206002200,karen,2020-01-01T00:00:00.000000000,karen,9 + 2020-01-01T00:00:00.000000000,1,14576041771120212628,patrick,2020-01-01T00:00:00.000000000,patrick,3 + 2020-01-02T00:00:00.000000000,2,4674756217206002200,karen,2020-01-02T00:00:00.000000000,karen,9 + 2020-01-02T00:00:00.000000000,3,14576041771120212628,patrick,2020-01-02T00:00:00.000000000,patrick,5 + 2020-01-03T00:00:00.000000000,4,4674756217206002200,karen,2020-01-03T00:00:00.000000000,karen,9 + 2020-01-03T00:00:00.000000000,5,14576041771120212628,patrick,2020-01-03T00:00:00.000000000,patrick,12 + 2020-01-04T00:00:00.000000000,6,14576041771120212628,patrick,2020-01-04T00:00:00.000000000,patrick,5000 + 2020-01-04T00:00:00.000000000,7,4674756217206002200,karen,2020-01-04T00:00:00.000000000,karen,9 + 2020-01-05T00:00:00.000000000,8,4674756217206002200,karen,2020-01-05T00:00:00.000000000,karen,9 + 2020-01-05T00:00:00.000000000,9,14576041771120212628,patrick,2020-01-05T00:00:00.000000000,patrick,5000 + 2020-01-06T00:00:00.000000000,0,14576041771120212628,patrick,2020-01-06T00:00:00.000000000,patrick,5000 + 2020-01-06T00:00:00.000000000,1,6566809397636161383,spongebob,2020-01-06T00:00:00.000000000,spongebob,7 + 2020-01-07T00:00:00.000000000,2,6566809397636161383,spongebob,2020-01-07T00:00:00.000000000,spongebob,34 + 2020-01-08T00:00:00.000000000,3,4674756217206002200,karen,2020-01-08T00:00:00.000000000,karen,9 + 2020-01-08T00:00:00.000000000,4,14576041771120212628,patrick,2020-01-08T00:00:00.000000000,patrick,5000 "###); } diff --git a/crates/sparrow-main/tests/e2e/prepare_tests.rs b/crates/sparrow-main/tests/e2e/prepare_tests.rs index 3f3bc1979..525e486be 100644 --- a/crates/sparrow-main/tests/e2e/prepare_tests.rs +++ b/crates/sparrow-main/tests/e2e/prepare_tests.rs @@ -64,12 +64,12 @@ async fn test_prepare_key_columns_parquet() { .unwrap(); insta::assert_snapshot!(QueryFixture::new("Numbers").run_to_csv(&data_fixture).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,subsort,key,m,n - 1996-12-20T00:39:57.000000000,9223372036854775809,12960666915911099378,A,1996-12-20T00:39:57.000000000,1,A,5,10 - 1996-12-20T00:39:58.000000000,9223372036854775810,2867199309159137213,B,1996-12-20T00:39:58.000000000,2,B,24,3 - 1996-12-20T00:39:59.000000000,9223372036854775811,12960666915911099378,A,1996-12-20T00:39:59.000000000,3,A,17,6 - 1996-12-20T00:40:00.000000000,9223372036854775812,12960666915911099378,A,1996-12-20T00:40:00.000000000,4,A,,9 - 1996-12-20T00:40:01.000000000,9223372036854775813,12960666915911099378,A,1996-12-20T00:40:01.000000000,5,A,12, - 1996-12-20T00:40:02.000000000,9223372036854775814,12960666915911099378,A,1996-12-20T00:40:02.000000000,6,A,, + 1996-12-20T00:39:57.000000000,1,12960666915911099378,A,1996-12-20T00:39:57.000000000,1,A,5,10 + 1996-12-20T00:39:58.000000000,2,2867199309159137213,B,1996-12-20T00:39:58.000000000,2,B,24,3 + 1996-12-20T00:39:59.000000000,3,12960666915911099378,A,1996-12-20T00:39:59.000000000,3,A,17,6 + 1996-12-20T00:40:00.000000000,4,12960666915911099378,A,1996-12-20T00:40:00.000000000,4,A,,9 + 1996-12-20T00:40:01.000000000,5,12960666915911099378,A,1996-12-20T00:40:01.000000000,5,A,12, + 1996-12-20T00:40:02.000000000,6,12960666915911099378,A,1996-12-20T00:40:02.000000000,6,A,, "###); } diff --git a/crates/sparrow-main/tests/e2e/record_tests.rs b/crates/sparrow-main/tests/e2e/record_tests.rs index 1de83c745..77ea9a8dc 100644 --- a/crates/sparrow-main/tests/e2e/record_tests.rs +++ b/crates/sparrow-main/tests/e2e/record_tests.rs @@ -15,12 +15,12 @@ async fn test_record_creation() { let z = record.z in { n: record.x + record.n, y, z }").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,y,z - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,hello,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,10,hello,World - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,3,hello,hello world - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,3,hello, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,7,hello, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,,hello,goodbye + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,hello,hEllo + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,10,hello,World + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,3,hello,hello world + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,3,hello, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,7,hello, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,,hello,goodbye "###); } @@ -32,12 +32,12 @@ async fn test_record_extension() { let z = record.s in { n: record.x + record.n, y, z }").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,y,z - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,hello,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,10,hello,World - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,3,hello,hello world - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,3,hello, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,7,hello, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,,hello,goodbye + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,hello,hEllo + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,10,hello,World + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,3,hello,hello world + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,3,hello, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,7,hello, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,,hello,goodbye "###); } @@ -68,12 +68,12 @@ async fn test_record_extension_error() { async fn test_record_extension_ordering() { insta::assert_snapshot!(QueryFixture::new("Strings | extend({ x: 5, y: \"hello\"})").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,x,y,time,subsort,key,s,n,t - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5,hello,1996-12-20T00:39:57.000000000,0,A,hEllo,0,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,5,hello,1996-12-20T00:40:57.000000000,0,B,World,5,world - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,5,hello,1996-12-20T00:41:57.000000000,0,B,hello world,-2,hello world - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,5,hello,1996-12-20T00:42:57.000000000,0,B,,-2,greetings - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,5,hello,1996-12-20T00:43:57.000000000,0,B,,2,salutations - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,5,hello,1996-12-20T00:44:57.000000000,0,B,goodbye,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5,hello,1996-12-20T00:39:57.000000000,0,A,hEllo,0,hEllo + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,5,hello,1996-12-20T00:40:57.000000000,0,B,World,5,world + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,5,hello,1996-12-20T00:41:57.000000000,0,B,hello world,-2,hello world + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,5,hello,1996-12-20T00:42:57.000000000,0,B,,-2,greetings + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,5,hello,1996-12-20T00:43:57.000000000,0,B,,2,salutations + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,5,hello,1996-12-20T00:44:57.000000000,0,B,goodbye,, "###); } @@ -81,12 +81,12 @@ async fn test_record_extension_ordering() { async fn test_record_removal() { insta::assert_snapshot!(QueryFixture::new("remove_fields(Strings, \"time\", \"subsort\")").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,key,s,n,t - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,A,hEllo,0,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,B,World,5,world - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,B,hello world,-2,hello world - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,B,,-2,greetings - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,B,,2,salutations - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,B,goodbye,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,A,hEllo,0,hEllo + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,B,World,5,world + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,B,hello world,-2,hello world + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,B,,-2,greetings + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,B,,2,salutations + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,B,goodbye,, "###) } @@ -94,12 +94,12 @@ async fn test_record_removal() { async fn test_record_removal_pipe() { insta::assert_snapshot!(QueryFixture::new("Strings | remove_fields($input, \"time\", \"subsort\")").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,key,s,n,t - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,A,hEllo,0,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,B,World,5,world - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,B,hello world,-2,hello world - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,B,,-2,greetings - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,B,,2,salutations - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,B,goodbye,, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,A,hEllo,0,hEllo + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,B,World,5,world + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,B,hello world,-2,hello world + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,B,,-2,greetings + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,B,,2,salutations + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,B,goodbye,, "###) } @@ -107,12 +107,12 @@ async fn test_record_removal_pipe() { async fn test_record_select() { insta::assert_snapshot!(QueryFixture::new("select_fields(Strings, \"time\", \"s\")").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,s - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:39:57.000000000,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:40:57.000000000,World - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:41:57.000000000,hello world - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:42:57.000000000, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:43:57.000000000, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:44:57.000000000,goodbye + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1996-12-20T00:39:57.000000000,hEllo + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,1996-12-20T00:40:57.000000000,World + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,1996-12-20T00:41:57.000000000,hello world + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,1996-12-20T00:42:57.000000000, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,1996-12-20T00:43:57.000000000, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,1996-12-20T00:44:57.000000000,goodbye "###) } @@ -144,12 +144,12 @@ async fn test_record_select_unused_key() { async fn test_record_select_pipe() { insta::assert_snapshot!(QueryFixture::new("Strings | select_fields($input, \"time\", \"s\")").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,s - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:39:57.000000000,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:40:57.000000000,World - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:41:57.000000000,hello world - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:42:57.000000000, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:43:57.000000000, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:44:57.000000000,goodbye + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1996-12-20T00:39:57.000000000,hEllo + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,1996-12-20T00:40:57.000000000,World + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,1996-12-20T00:41:57.000000000,hello world + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,1996-12-20T00:42:57.000000000, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,1996-12-20T00:43:57.000000000, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,1996-12-20T00:44:57.000000000,goodbye "###) } diff --git a/crates/sparrow-main/tests/e2e/shift_tests.rs b/crates/sparrow-main/tests/e2e/shift_tests.rs index 764ab404e..ab6a7304b 100644 --- a/crates/sparrow-main/tests/e2e/shift_tests.rs +++ b/crates/sparrow-main/tests/e2e/shift_tests.rs @@ -227,9 +227,9 @@ async fn test_shift_until_false() { let shift_until_gt_75 = ShiftFixture.string | shift_until(gt_75) in { gt_10, shift_until_gt_10, gt_75, shift_until_gt_75 } | when(gt_10 or gt_75)").run_to_csv(&shift_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,gt_10,shift_until_gt_10,gt_75,shift_until_gt_75 - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,,false, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,true,,false, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,true,,false, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,hello,false, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,true,,false, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,true,,false, "###) } @@ -243,15 +243,14 @@ async fn test_shift_until_false_sum() { let shift_until_gt_75 = ShiftFixture.string | shift_until(gt_75) in { gt_10, shift_until_gt_10, gt_75, shift_until_gt_75 }").run_to_csv(&shift_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,gt_10,shift_until_gt_10,gt_75,shift_until_gt_75 - 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,,57,, - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,57,false, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,57,false, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,true,,false, 1996-12-20T00:39:58.000000000,1,2867199309159137213,B,,58,, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,true,58,false, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,true,57,false, 1996-12-20T00:39:59.000000000,2,12960666915911099378,A,,116,, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,true,116,false, - 1996-12-20T00:40:00.000000000,9223372036854775808,2867199309159137213,B,,58,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,,116,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,false,116,false, + 1996-12-20T00:40:00.000000000,0,2867199309159137213,B,,58,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,,116,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,false,116,false, "###) } diff --git a/crates/sparrow-main/tests/e2e/string_tests.rs b/crates/sparrow-main/tests/e2e/string_tests.rs index 598de0d3f..970f2cd5c 100644 --- a/crates/sparrow-main/tests/e2e/string_tests.rs +++ b/crates/sparrow-main/tests/e2e/string_tests.rs @@ -7,12 +7,12 @@ use crate::QueryFixture; async fn test_len() { insta::assert_snapshot!(QueryFixture::new("{ len: len(Strings.s)}").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,len - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5 - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,5 - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,11 - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,0 - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,0 - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,7 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5 + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,5 + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,11 + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,0 + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,0 + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,7 "###); } @@ -20,12 +20,12 @@ async fn test_len() { async fn test_upper_len() { insta::assert_snapshot!(QueryFixture::new("{ upper: upper(Strings.s)}").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,upper - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,HELLO - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,WORLD - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,HELLO WORLD - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,GOODBYE + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,HELLO + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,WORLD + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,HELLO WORLD + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,GOODBYE "###); } @@ -33,12 +33,12 @@ async fn test_upper_len() { async fn test_lower_len() { insta::assert_snapshot!(QueryFixture::new("{ lower: lower(Strings.s)}").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,lower - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hello - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,world - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,hello world - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,goodbye + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hello + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,world + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,hello world + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,goodbye "###); } @@ -50,11 +50,11 @@ async fn test_substring() { , substring_i: substring(Strings.s, start=Strings.n), }").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,substring_0_2,substring_1,substring_0_i,substring_i - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hE,Ello,,hEllo - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,Wo,orld,World, - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,he,ello world,hello wor,ld - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,,,, - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,,,, - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,go,oodbye,goodbye,goodbye + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hE,Ello,,hEllo + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,Wo,orld,World, + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,he,ello world,hello wor,ld + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,,,, + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,,,, + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,go,oodbye,goodbye,goodbye "###); } diff --git a/crates/sparrow-main/tests/e2e/tick_tests.rs b/crates/sparrow-main/tests/e2e/tick_tests.rs index bed3a2819..fd761995c 100644 --- a/crates/sparrow-main/tests/e2e/tick_tests.rs +++ b/crates/sparrow-main/tests/e2e/tick_tests.rs @@ -274,20 +274,20 @@ async fn test_since_tick_when_tick() { async fn test_since_minutely() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, minutely: count(Foo, window=since(minutely())) }").run_to_csv(&data_fixture_over_minutes().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,minutely - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,1 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,6.2,2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,9.25,3 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10.0,1 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,3.9,1 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,6.2,2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,9.25,3 1996-12-20T00:40:00.000000000,18446744073709551615,2867199309159137213,B,,1 1996-12-20T00:40:00.000000000,18446744073709551615,12960666915911099378,A,,3 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,3.0,1 - 1996-12-20T00:40:02.000000000,9223372036854775808,2867199309159137213,B,8.0,1 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,8.0,2 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,,3 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,4 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,3.0,1 + 1996-12-20T00:40:02.000000000,0,2867199309159137213,B,8.0,1 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,8.0,2 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,,3 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,10.0,4 1996-12-20T00:41:00.000000000,18446744073709551615,2867199309159137213,B,,1 1996-12-20T00:41:00.000000000,18446744073709551615,12960666915911099378,A,,4 - 1996-12-20T00:41:04.000000000,9223372036854775808,12960666915911099378,A,10.0,1 + 1996-12-20T00:41:04.000000000,0,12960666915911099378,A,10.0,1 "###); } @@ -295,21 +295,21 @@ async fn test_since_minutely() { async fn test_if_hourly() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, n_if_hourly: Foo.n | if(hourly()) }").run_to_csv(&data_fixture_over_hours_end_on_hour().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,n_if_hourly - 1996-12-20T07:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0, - 1996-12-20T07:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9, - 1996-12-20T08:00:00.000000000,9223372036854775808,12960666915911099378,A,6.2, + 1996-12-20T07:39:57.000000000,0,12960666915911099378,A,10.0, + 1996-12-20T07:39:58.000000000,0,2867199309159137213,B,3.9, + 1996-12-20T08:00:00.000000000,0,12960666915911099378,A,6.2, 1996-12-20T08:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1996-12-20T08:00:00.000000000,18446744073709551615,12960666915911099378,A,, - 1996-12-20T08:44:00.000000000,9223372036854775808,12960666915911099378,A,9.25, - 1996-12-20T08:45:01.000000000,9223372036854775808,12960666915911099378,A,3.0, + 1996-12-20T08:44:00.000000000,0,12960666915911099378,A,9.25, + 1996-12-20T08:45:01.000000000,0,12960666915911099378,A,3.0, 1996-12-20T09:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1996-12-20T09:00:00.000000000,18446744073709551615,12960666915911099378,A,, - 1996-12-20T09:20:02.000000000,9223372036854775808,12960666915911099378,A,8.0, - 1996-12-20T09:25:02.000000000,9223372036854775808,2867199309159137213,B,23.9, + 1996-12-20T09:20:02.000000000,0,12960666915911099378,A,8.0, + 1996-12-20T09:25:02.000000000,0,2867199309159137213,B,23.9, 1996-12-20T10:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1996-12-20T10:00:00.000000000,18446744073709551615,12960666915911099378,A,, - 1996-12-20T10:30:03.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T11:00:00.000000000,9223372036854775808,12960666915911099378,A,10.0, + 1996-12-20T10:30:03.000000000,0,12960666915911099378,A,, + 1996-12-20T11:00:00.000000000,0,12960666915911099378,A,10.0, 1996-12-20T11:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1996-12-20T11:00:00.000000000,18446744073709551615,12960666915911099378,A,, "###); @@ -319,22 +319,22 @@ async fn test_if_hourly() { async fn test_daily_else() { insta::assert_snapshot!(QueryFixture::new("{ sum_since: sum(Foo.n, window=since(daily())) | else(0) }").run_to_csv(&data_days_for_else().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sum_since - 1996-12-19T20:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0 - 1996-12-19T20:39:58.000000000,9223372036854775808,2867199309159137213,B,0.0 + 1996-12-19T20:39:57.000000000,0,12960666915911099378,A,10.0 + 1996-12-19T20:39:58.000000000,0,2867199309159137213,B,0.0 1996-12-20T00:00:00.000000000,18446744073709551615,2867199309159137213,B,0.0 1996-12-20T00:00:00.000000000,18446744073709551615,12960666915911099378,A,10.0 1996-12-21T00:00:00.000000000,18446744073709551615,2867199309159137213,B,0.0 1996-12-21T00:00:00.000000000,18446744073709551615,12960666915911099378,A,0.0 - 1996-12-21T00:32:59.000000000,9223372036854775808,12960666915911099378,A,6.2 - 1996-12-21T00:44:00.000000000,9223372036854775808,12960666915911099378,A,15.45 - 1996-12-21T00:45:01.000000000,9223372036854775808,12960666915911099378,A,18.45 - 1996-12-21T08:00:00.000000000,9223372036854775808,12960666915911099378,A,26.45 + 1996-12-21T00:32:59.000000000,0,12960666915911099378,A,6.2 + 1996-12-21T00:44:00.000000000,0,12960666915911099378,A,15.45 + 1996-12-21T00:45:01.000000000,0,12960666915911099378,A,18.45 + 1996-12-21T08:00:00.000000000,0,12960666915911099378,A,26.45 1996-12-22T00:00:00.000000000,18446744073709551615,2867199309159137213,B,0.0 1996-12-22T00:00:00.000000000,18446744073709551615,12960666915911099378,A,26.45 - 1996-12-22T00:30:03.000000000,9223372036854775808,12960666915911099378,A,0.0 + 1996-12-22T00:30:03.000000000,0,12960666915911099378,A,0.0 1996-12-23T00:00:00.000000000,18446744073709551615,2867199309159137213,B,0.0 1996-12-23T00:00:00.000000000,18446744073709551615,12960666915911099378,A,0.0 - 1996-12-23T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0 + 1996-12-23T00:40:04.000000000,0,12960666915911099378,A,10.0 "###); } @@ -342,22 +342,22 @@ async fn test_daily_else() { async fn test_daily_else_to_last() { insta::assert_snapshot!(QueryFixture::new("{ sum_since: sum(Foo.n, window=since(daily())) | else(0) | last() }").run_to_csv(&data_days_for_else().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sum_since - 1996-12-19T20:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0 - 1996-12-19T20:39:58.000000000,9223372036854775808,2867199309159137213,B,0.0 + 1996-12-19T20:39:57.000000000,0,12960666915911099378,A,10.0 + 1996-12-19T20:39:58.000000000,0,2867199309159137213,B,0.0 1996-12-20T00:00:00.000000000,18446744073709551615,2867199309159137213,B,0.0 1996-12-20T00:00:00.000000000,18446744073709551615,12960666915911099378,A,10.0 1996-12-21T00:00:00.000000000,18446744073709551615,2867199309159137213,B,0.0 1996-12-21T00:00:00.000000000,18446744073709551615,12960666915911099378,A,0.0 - 1996-12-21T00:32:59.000000000,9223372036854775808,12960666915911099378,A,6.2 - 1996-12-21T00:44:00.000000000,9223372036854775808,12960666915911099378,A,15.45 - 1996-12-21T00:45:01.000000000,9223372036854775808,12960666915911099378,A,18.45 - 1996-12-21T08:00:00.000000000,9223372036854775808,12960666915911099378,A,26.45 + 1996-12-21T00:32:59.000000000,0,12960666915911099378,A,6.2 + 1996-12-21T00:44:00.000000000,0,12960666915911099378,A,15.45 + 1996-12-21T00:45:01.000000000,0,12960666915911099378,A,18.45 + 1996-12-21T08:00:00.000000000,0,12960666915911099378,A,26.45 1996-12-22T00:00:00.000000000,18446744073709551615,2867199309159137213,B,0.0 1996-12-22T00:00:00.000000000,18446744073709551615,12960666915911099378,A,26.45 - 1996-12-22T00:30:03.000000000,9223372036854775808,12960666915911099378,A,0.0 + 1996-12-22T00:30:03.000000000,0,12960666915911099378,A,0.0 1996-12-23T00:00:00.000000000,18446744073709551615,2867199309159137213,B,0.0 1996-12-23T00:00:00.000000000,18446744073709551615,12960666915911099378,A,0.0 - 1996-12-23T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0 + 1996-12-23T00:40:04.000000000,0,12960666915911099378,A,10.0 "###); } @@ -365,23 +365,23 @@ async fn test_daily_else_to_last() { async fn test_since_daily() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, sum_since: sum(Foo.n, window=since(daily())) }").run_to_csv(&data_fixture_over_days().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,sum_since - 1996-12-19T20:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,10.0 - 1996-12-19T20:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,3.9 + 1996-12-19T20:39:57.000000000,0,12960666915911099378,A,10.0,10.0 + 1996-12-19T20:39:58.000000000,0,2867199309159137213,B,3.9,3.9 1996-12-20T00:00:00.000000000,18446744073709551615,2867199309159137213,B,,3.9 1996-12-20T00:00:00.000000000,18446744073709551615,12960666915911099378,A,,10.0 1996-12-21T00:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1996-12-21T00:00:00.000000000,18446744073709551615,12960666915911099378,A,, - 1996-12-21T00:32:59.000000000,9223372036854775808,12960666915911099378,A,6.2,6.2 - 1996-12-21T00:44:00.000000000,9223372036854775808,12960666915911099378,A,9.25,15.45 - 1996-12-21T00:45:01.000000000,9223372036854775808,12960666915911099378,A,3.0,18.45 - 1996-12-21T08:00:00.000000000,9223372036854775808,12960666915911099378,A,8.0,26.45 + 1996-12-21T00:32:59.000000000,0,12960666915911099378,A,6.2,6.2 + 1996-12-21T00:44:00.000000000,0,12960666915911099378,A,9.25,15.45 + 1996-12-21T00:45:01.000000000,0,12960666915911099378,A,3.0,18.45 + 1996-12-21T08:00:00.000000000,0,12960666915911099378,A,8.0,26.45 1996-12-22T00:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1996-12-22T00:00:00.000000000,18446744073709551615,12960666915911099378,A,,26.45 - 1996-12-22T00:25:02.000000000,9223372036854775808,2867199309159137213,B,23.9,23.9 - 1996-12-22T00:30:03.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-22T00:25:02.000000000,0,2867199309159137213,B,23.9,23.9 + 1996-12-22T00:30:03.000000000,0,12960666915911099378,A,, 1996-12-23T00:00:00.000000000,18446744073709551615,2867199309159137213,B,,23.9 1996-12-23T00:00:00.000000000,18446744073709551615,12960666915911099378,A,, - 1996-12-23T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,10.0 + 1996-12-23T00:40:04.000000000,0,12960666915911099378,A,10.0,10.0 "###); } @@ -389,23 +389,23 @@ async fn test_since_daily() { async fn test_since_hourly() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, sum_since: sum(Foo.n, window=since(hourly())) }").run_to_csv(&data_fixture_over_hours().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,sum_since - 1996-12-20T07:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,10.0 - 1996-12-20T07:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,3.9 - 1996-12-20T08:00:00.000000000,9223372036854775808,12960666915911099378,A,6.2,16.2 + 1996-12-20T07:39:57.000000000,0,12960666915911099378,A,10.0,10.0 + 1996-12-20T07:39:58.000000000,0,2867199309159137213,B,3.9,3.9 + 1996-12-20T08:00:00.000000000,0,12960666915911099378,A,6.2,16.2 1996-12-20T08:00:00.000000000,18446744073709551615,2867199309159137213,B,,3.9 1996-12-20T08:00:00.000000000,18446744073709551615,12960666915911099378,A,,16.2 - 1996-12-20T08:44:00.000000000,9223372036854775808,12960666915911099378,A,9.25,9.25 - 1996-12-20T08:45:01.000000000,9223372036854775808,12960666915911099378,A,3.0,12.25 + 1996-12-20T08:44:00.000000000,0,12960666915911099378,A,9.25,9.25 + 1996-12-20T08:45:01.000000000,0,12960666915911099378,A,3.0,12.25 1996-12-20T09:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1996-12-20T09:00:00.000000000,18446744073709551615,12960666915911099378,A,,12.25 - 1996-12-20T09:20:02.000000000,9223372036854775808,12960666915911099378,A,8.0,8.0 - 1996-12-20T09:25:02.000000000,9223372036854775808,2867199309159137213,B,23.9,23.9 + 1996-12-20T09:20:02.000000000,0,12960666915911099378,A,8.0,8.0 + 1996-12-20T09:25:02.000000000,0,2867199309159137213,B,23.9,23.9 1996-12-20T10:00:00.000000000,18446744073709551615,2867199309159137213,B,,23.9 1996-12-20T10:00:00.000000000,18446744073709551615,12960666915911099378,A,,8.0 - 1996-12-20T10:30:03.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-20T10:30:03.000000000,0,12960666915911099378,A,, 1996-12-20T11:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1996-12-20T11:00:00.000000000,18446744073709551615,12960666915911099378,A,, - 1996-12-20T11:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,10.0 + 1996-12-20T11:40:04.000000000,0,12960666915911099378,A,10.0,10.0 "###); } @@ -413,21 +413,21 @@ async fn test_since_hourly() { async fn test_since_hourly_end_on_hour() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, sum_since: sum(Foo.n, window=since(hourly())) }").run_to_csv(&data_fixture_over_hours_end_on_hour().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,sum_since - 1996-12-20T07:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,10.0 - 1996-12-20T07:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,3.9 - 1996-12-20T08:00:00.000000000,9223372036854775808,12960666915911099378,A,6.2,16.2 + 1996-12-20T07:39:57.000000000,0,12960666915911099378,A,10.0,10.0 + 1996-12-20T07:39:58.000000000,0,2867199309159137213,B,3.9,3.9 + 1996-12-20T08:00:00.000000000,0,12960666915911099378,A,6.2,16.2 1996-12-20T08:00:00.000000000,18446744073709551615,2867199309159137213,B,,3.9 1996-12-20T08:00:00.000000000,18446744073709551615,12960666915911099378,A,,16.2 - 1996-12-20T08:44:00.000000000,9223372036854775808,12960666915911099378,A,9.25,9.25 - 1996-12-20T08:45:01.000000000,9223372036854775808,12960666915911099378,A,3.0,12.25 + 1996-12-20T08:44:00.000000000,0,12960666915911099378,A,9.25,9.25 + 1996-12-20T08:45:01.000000000,0,12960666915911099378,A,3.0,12.25 1996-12-20T09:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1996-12-20T09:00:00.000000000,18446744073709551615,12960666915911099378,A,,12.25 - 1996-12-20T09:20:02.000000000,9223372036854775808,12960666915911099378,A,8.0,8.0 - 1996-12-20T09:25:02.000000000,9223372036854775808,2867199309159137213,B,23.9,23.9 + 1996-12-20T09:20:02.000000000,0,12960666915911099378,A,8.0,8.0 + 1996-12-20T09:25:02.000000000,0,2867199309159137213,B,23.9,23.9 1996-12-20T10:00:00.000000000,18446744073709551615,2867199309159137213,B,,23.9 1996-12-20T10:00:00.000000000,18446744073709551615,12960666915911099378,A,,8.0 - 1996-12-20T10:30:03.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T11:00:00.000000000,9223372036854775808,12960666915911099378,A,10.0,10.0 + 1996-12-20T10:30:03.000000000,0,12960666915911099378,A,, + 1996-12-20T11:00:00.000000000,0,12960666915911099378,A,10.0,10.0 1996-12-20T11:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1996-12-20T11:00:00.000000000,18446744073709551615,12960666915911099378,A,,10.0 "###); @@ -462,23 +462,23 @@ async fn test_when_hourly_end_on_hour_final_results() { async fn test_since_daily_over_span_of_days() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, sum_since: sum(Foo.n, window=since(daily())) }").run_to_csv(&data_fixture_over_days().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,sum_since - 1996-12-19T20:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,10.0 - 1996-12-19T20:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,3.9 + 1996-12-19T20:39:57.000000000,0,12960666915911099378,A,10.0,10.0 + 1996-12-19T20:39:58.000000000,0,2867199309159137213,B,3.9,3.9 1996-12-20T00:00:00.000000000,18446744073709551615,2867199309159137213,B,,3.9 1996-12-20T00:00:00.000000000,18446744073709551615,12960666915911099378,A,,10.0 1996-12-21T00:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1996-12-21T00:00:00.000000000,18446744073709551615,12960666915911099378,A,, - 1996-12-21T00:32:59.000000000,9223372036854775808,12960666915911099378,A,6.2,6.2 - 1996-12-21T00:44:00.000000000,9223372036854775808,12960666915911099378,A,9.25,15.45 - 1996-12-21T00:45:01.000000000,9223372036854775808,12960666915911099378,A,3.0,18.45 - 1996-12-21T08:00:00.000000000,9223372036854775808,12960666915911099378,A,8.0,26.45 + 1996-12-21T00:32:59.000000000,0,12960666915911099378,A,6.2,6.2 + 1996-12-21T00:44:00.000000000,0,12960666915911099378,A,9.25,15.45 + 1996-12-21T00:45:01.000000000,0,12960666915911099378,A,3.0,18.45 + 1996-12-21T08:00:00.000000000,0,12960666915911099378,A,8.0,26.45 1996-12-22T00:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1996-12-22T00:00:00.000000000,18446744073709551615,12960666915911099378,A,,26.45 - 1996-12-22T00:25:02.000000000,9223372036854775808,2867199309159137213,B,23.9,23.9 - 1996-12-22T00:30:03.000000000,9223372036854775808,12960666915911099378,A,, + 1996-12-22T00:25:02.000000000,0,2867199309159137213,B,23.9,23.9 + 1996-12-22T00:30:03.000000000,0,12960666915911099378,A,, 1996-12-23T00:00:00.000000000,18446744073709551615,2867199309159137213,B,,23.9 1996-12-23T00:00:00.000000000,18446744073709551615,12960666915911099378,A,, - 1996-12-23T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,10.0 + 1996-12-23T00:40:04.000000000,0,12960666915911099378,A,10.0,10.0 "###); } @@ -486,25 +486,25 @@ async fn test_since_daily_over_span_of_days() { async fn test_since_monthly() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, sum_since: sum(Foo.n, window=since(monthly())) }").run_to_csv(&data_fixture_over_months().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,sum_since - 1996-12-19T20:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,10.0 - 1996-12-19T20:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,3.9 + 1996-12-19T20:39:57.000000000,0,12960666915911099378,A,10.0,10.0 + 1996-12-19T20:39:58.000000000,0,2867199309159137213,B,3.9,3.9 1997-01-01T00:00:00.000000000,18446744073709551615,2867199309159137213,B,,3.9 1997-01-01T00:00:00.000000000,18446744073709551615,12960666915911099378,A,,10.0 - 1997-01-21T00:32:59.000000000,9223372036854775808,12960666915911099378,A,6.2,6.2 - 1997-01-21T00:44:00.000000000,9223372036854775808,12960666915911099378,A,9.25,15.45 + 1997-01-21T00:32:59.000000000,0,12960666915911099378,A,6.2,6.2 + 1997-01-21T00:44:00.000000000,0,12960666915911099378,A,9.25,15.45 1997-02-01T00:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1997-02-01T00:00:00.000000000,18446744073709551615,12960666915911099378,A,,15.45 - 1997-02-21T00:45:01.000000000,9223372036854775808,12960666915911099378,A,3.0,3.0 + 1997-02-21T00:45:01.000000000,0,12960666915911099378,A,3.0,3.0 1997-03-01T00:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1997-03-01T00:00:00.000000000,18446744073709551615,12960666915911099378,A,,3.0 1997-04-01T00:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1997-04-01T00:00:00.000000000,18446744073709551615,12960666915911099378,A,, - 1997-04-22T00:20:02.000000000,9223372036854775808,12960666915911099378,A,8.0,8.0 - 1997-04-22T00:25:02.000000000,9223372036854775808,2867199309159137213,B,23.9,23.9 - 1997-04-22T00:30:03.000000000,9223372036854775808,12960666915911099378,A,,8.0 + 1997-04-22T00:20:02.000000000,0,12960666915911099378,A,8.0,8.0 + 1997-04-22T00:25:02.000000000,0,2867199309159137213,B,23.9,23.9 + 1997-04-22T00:30:03.000000000,0,12960666915911099378,A,,8.0 1997-05-01T00:00:00.000000000,18446744073709551615,2867199309159137213,B,,23.9 1997-05-01T00:00:00.000000000,18446744073709551615,12960666915911099378,A,,8.0 - 1997-05-23T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,10.0 + 1997-05-23T00:40:04.000000000,0,12960666915911099378,A,10.0,10.0 "###); } @@ -512,25 +512,25 @@ async fn test_since_monthly() { async fn test_since_yearly() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, sum_since: sum(Foo.n, window=since(yearly())) }").run_to_csv(&data_fixture_over_years().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,sum_since - 1996-12-19T20:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,10.0 - 1996-12-19T20:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,3.9 + 1996-12-19T20:39:57.000000000,0,12960666915911099378,A,10.0,10.0 + 1996-12-19T20:39:58.000000000,0,2867199309159137213,B,3.9,3.9 1997-01-01T00:00:00.000000000,18446744073709551615,2867199309159137213,B,,3.9 1997-01-01T00:00:00.000000000,18446744073709551615,12960666915911099378,A,,10.0 1998-01-01T00:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1998-01-01T00:00:00.000000000,18446744073709551615,12960666915911099378,A,, - 1998-01-21T00:32:59.000000000,9223372036854775808,12960666915911099378,A,6.2,6.2 - 1998-01-21T00:44:00.000000000,9223372036854775808,12960666915911099378,A,9.25,15.45 + 1998-01-21T00:32:59.000000000,0,12960666915911099378,A,6.2,6.2 + 1998-01-21T00:44:00.000000000,0,12960666915911099378,A,9.25,15.45 1999-01-01T00:00:00.000000000,18446744073709551615,2867199309159137213,B,, 1999-01-01T00:00:00.000000000,18446744073709551615,12960666915911099378,A,,15.45 - 1999-02-21T00:45:01.000000000,9223372036854775808,12960666915911099378,A,3.0,3.0 - 1999-04-22T00:20:02.000000000,9223372036854775808,12960666915911099378,A,8.0,11.0 + 1999-02-21T00:45:01.000000000,0,12960666915911099378,A,3.0,3.0 + 1999-04-22T00:20:02.000000000,0,12960666915911099378,A,8.0,11.0 2000-01-01T00:00:00.000000000,18446744073709551615,2867199309159137213,B,, 2000-01-01T00:00:00.000000000,18446744073709551615,12960666915911099378,A,,11.0 2001-01-01T00:00:00.000000000,18446744073709551615,2867199309159137213,B,, 2001-01-01T00:00:00.000000000,18446744073709551615,12960666915911099378,A,, - 2001-04-22T00:25:02.000000000,9223372036854775808,2867199309159137213,B,23.9,23.9 - 2001-04-22T00:30:03.000000000,9223372036854775808,12960666915911099378,A,, - 2001-05-23T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,10.0 + 2001-04-22T00:25:02.000000000,0,2867199309159137213,B,23.9,23.9 + 2001-04-22T00:30:03.000000000,0,12960666915911099378,A,, + 2001-05-23T00:40:04.000000000,0,12960666915911099378,A,10.0,10.0 "###); } @@ -566,23 +566,23 @@ async fn test_tick_when_finished() { async fn test_count_sliding_tick_daily() { insta::assert_snapshot!(QueryFixture::new("{ count: count(Foo), sliding_count: count(Foo, window=sliding(2, daily())) }").run_to_csv(&data_fixture_over_days().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,count,sliding_count - 1996-12-19T20:39:57.000000000,9223372036854775808,12960666915911099378,A,1,1 - 1996-12-19T20:39:58.000000000,9223372036854775808,2867199309159137213,B,1,1 + 1996-12-19T20:39:57.000000000,0,12960666915911099378,A,1,1 + 1996-12-19T20:39:58.000000000,0,2867199309159137213,B,1,1 1996-12-20T00:00:00.000000000,18446744073709551615,2867199309159137213,B,1,1 1996-12-20T00:00:00.000000000,18446744073709551615,12960666915911099378,A,1,1 1996-12-21T00:00:00.000000000,18446744073709551615,2867199309159137213,B,1,1 1996-12-21T00:00:00.000000000,18446744073709551615,12960666915911099378,A,1,1 - 1996-12-21T00:32:59.000000000,9223372036854775808,12960666915911099378,A,2,1 - 1996-12-21T00:44:00.000000000,9223372036854775808,12960666915911099378,A,3,2 - 1996-12-21T00:45:01.000000000,9223372036854775808,12960666915911099378,A,4,3 - 1996-12-21T08:00:00.000000000,9223372036854775808,12960666915911099378,A,5,4 + 1996-12-21T00:32:59.000000000,0,12960666915911099378,A,2,1 + 1996-12-21T00:44:00.000000000,0,12960666915911099378,A,3,2 + 1996-12-21T00:45:01.000000000,0,12960666915911099378,A,4,3 + 1996-12-21T08:00:00.000000000,0,12960666915911099378,A,5,4 1996-12-22T00:00:00.000000000,18446744073709551615,2867199309159137213,B,1,0 1996-12-22T00:00:00.000000000,18446744073709551615,12960666915911099378,A,5,4 - 1996-12-22T00:25:02.000000000,9223372036854775808,2867199309159137213,B,2,1 - 1996-12-22T00:30:03.000000000,9223372036854775808,12960666915911099378,A,6,5 + 1996-12-22T00:25:02.000000000,0,2867199309159137213,B,2,1 + 1996-12-22T00:30:03.000000000,0,12960666915911099378,A,6,5 1996-12-23T00:00:00.000000000,18446744073709551615,2867199309159137213,B,2,1 1996-12-23T00:00:00.000000000,18446744073709551615,12960666915911099378,A,6,5 - 1996-12-23T00:40:04.000000000,9223372036854775808,12960666915911099378,A,7,2 + 1996-12-23T00:40:04.000000000,0,12960666915911099378,A,7,2 "###); } @@ -590,23 +590,23 @@ async fn test_count_sliding_tick_daily() { async fn test_count_daily_sliding_equivalent_to_since() { insta::assert_snapshot!(QueryFixture::new("{ since: count(Foo, window=since(daily())), sliding: count(Foo, window=sliding(1, daily())) }").run_to_csv(&data_fixture_over_days().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,since,sliding - 1996-12-19T20:39:57.000000000,9223372036854775808,12960666915911099378,A,1,1 - 1996-12-19T20:39:58.000000000,9223372036854775808,2867199309159137213,B,1,1 + 1996-12-19T20:39:57.000000000,0,12960666915911099378,A,1,1 + 1996-12-19T20:39:58.000000000,0,2867199309159137213,B,1,1 1996-12-20T00:00:00.000000000,18446744073709551615,2867199309159137213,B,1,1 1996-12-20T00:00:00.000000000,18446744073709551615,12960666915911099378,A,1,1 1996-12-21T00:00:00.000000000,18446744073709551615,2867199309159137213,B,0,0 1996-12-21T00:00:00.000000000,18446744073709551615,12960666915911099378,A,0,0 - 1996-12-21T00:32:59.000000000,9223372036854775808,12960666915911099378,A,1,1 - 1996-12-21T00:44:00.000000000,9223372036854775808,12960666915911099378,A,2,2 - 1996-12-21T00:45:01.000000000,9223372036854775808,12960666915911099378,A,3,3 - 1996-12-21T08:00:00.000000000,9223372036854775808,12960666915911099378,A,4,4 + 1996-12-21T00:32:59.000000000,0,12960666915911099378,A,1,1 + 1996-12-21T00:44:00.000000000,0,12960666915911099378,A,2,2 + 1996-12-21T00:45:01.000000000,0,12960666915911099378,A,3,3 + 1996-12-21T08:00:00.000000000,0,12960666915911099378,A,4,4 1996-12-22T00:00:00.000000000,18446744073709551615,2867199309159137213,B,0,0 1996-12-22T00:00:00.000000000,18446744073709551615,12960666915911099378,A,4,4 - 1996-12-22T00:25:02.000000000,9223372036854775808,2867199309159137213,B,1,1 - 1996-12-22T00:30:03.000000000,9223372036854775808,12960666915911099378,A,1,1 + 1996-12-22T00:25:02.000000000,0,2867199309159137213,B,1,1 + 1996-12-22T00:30:03.000000000,0,12960666915911099378,A,1,1 1996-12-23T00:00:00.000000000,18446744073709551615,2867199309159137213,B,1,1 1996-12-23T00:00:00.000000000,18446744073709551615,12960666915911099378,A,1,1 - 1996-12-23T00:40:04.000000000,9223372036854775808,12960666915911099378,A,1,1 + 1996-12-23T00:40:04.000000000,0,12960666915911099378,A,1,1 "###); } @@ -614,23 +614,23 @@ async fn test_count_daily_sliding_equivalent_to_since() { async fn test_max_subsort_input_merges_correctly() { insta::assert_snapshot!(QueryFixture::new("{ field: count(Foo, window=since(daily())) | count(window=since(daily())) }").run_to_csv(&data_fixture_over_days().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,field - 1996-12-19T20:39:57.000000000,9223372036854775808,12960666915911099378,A,1 - 1996-12-19T20:39:58.000000000,9223372036854775808,2867199309159137213,B,1 + 1996-12-19T20:39:57.000000000,0,12960666915911099378,A,1 + 1996-12-19T20:39:58.000000000,0,2867199309159137213,B,1 1996-12-20T00:00:00.000000000,18446744073709551615,2867199309159137213,B,2 1996-12-20T00:00:00.000000000,18446744073709551615,12960666915911099378,A,2 1996-12-21T00:00:00.000000000,18446744073709551615,2867199309159137213,B,1 1996-12-21T00:00:00.000000000,18446744073709551615,12960666915911099378,A,1 - 1996-12-21T00:32:59.000000000,9223372036854775808,12960666915911099378,A,1 - 1996-12-21T00:44:00.000000000,9223372036854775808,12960666915911099378,A,2 - 1996-12-21T00:45:01.000000000,9223372036854775808,12960666915911099378,A,3 - 1996-12-21T08:00:00.000000000,9223372036854775808,12960666915911099378,A,4 + 1996-12-21T00:32:59.000000000,0,12960666915911099378,A,1 + 1996-12-21T00:44:00.000000000,0,12960666915911099378,A,2 + 1996-12-21T00:45:01.000000000,0,12960666915911099378,A,3 + 1996-12-21T08:00:00.000000000,0,12960666915911099378,A,4 1996-12-22T00:00:00.000000000,18446744073709551615,2867199309159137213,B,1 1996-12-22T00:00:00.000000000,18446744073709551615,12960666915911099378,A,5 - 1996-12-22T00:25:02.000000000,9223372036854775808,2867199309159137213,B,1 - 1996-12-22T00:30:03.000000000,9223372036854775808,12960666915911099378,A,1 + 1996-12-22T00:25:02.000000000,0,2867199309159137213,B,1 + 1996-12-22T00:30:03.000000000,0,12960666915911099378,A,1 1996-12-23T00:00:00.000000000,18446744073709551615,2867199309159137213,B,2 1996-12-23T00:00:00.000000000,18446744073709551615,12960666915911099378,A,2 - 1996-12-23T00:40:04.000000000,9223372036854775808,12960666915911099378,A,1 + 1996-12-23T00:40:04.000000000,0,12960666915911099378,A,1 "###); } diff --git a/crates/sparrow-main/tests/e2e/time_tests.rs b/crates/sparrow-main/tests/e2e/time_tests.rs index f24234252..997fdb266 100644 --- a/crates/sparrow-main/tests/e2e/time_tests.rs +++ b/crates/sparrow-main/tests/e2e/time_tests.rs @@ -21,13 +21,13 @@ use crate::QueryFixture; async fn test_time_of_boolean() { insta::assert_snapshot!(QueryFixture::new("{ time_of: time_of(Booleans.a)}").run_to_csv(&boolean_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time_of - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:39:57.000000000 - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:40:57.000000000 - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:41:57.000000000 - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:42:57.000000000 - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:43:57.000000000 - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:44:57.000000000 - 1996-12-20T00:45:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:45:57.000000000 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1996-12-20T00:39:57.000000000 + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,1996-12-20T00:40:57.000000000 + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,1996-12-20T00:41:57.000000000 + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,1996-12-20T00:42:57.000000000 + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,1996-12-20T00:43:57.000000000 + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,1996-12-20T00:44:57.000000000 + 1996-12-20T00:45:57.000000000,0,2867199309159137213,B,1996-12-20T00:45:57.000000000 "###); } @@ -35,12 +35,12 @@ async fn test_time_of_boolean() { async fn test_time_of_i64() { insta::assert_snapshot!(QueryFixture::new("{ time_of: time_of(Numbers.m)}").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time_of - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:39:57.000000000 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:39:58.000000000 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:39:59.000000000 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:40:00.000000000 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:40:01.000000000 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:40:02.000000000 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1996-12-20T00:39:57.000000000 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,1996-12-20T00:39:58.000000000 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,1996-12-20T00:39:59.000000000 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,1996-12-20T00:40:00.000000000 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,1996-12-20T00:40:01.000000000 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,1996-12-20T00:40:02.000000000 "###); } @@ -48,12 +48,12 @@ async fn test_time_of_i64() { async fn test_time_of_timestamp() { insta::assert_snapshot!(QueryFixture::new("{ time_of: time_of(Times.time) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time_of - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000 "###); } @@ -61,12 +61,12 @@ async fn test_time_of_timestamp() { async fn test_time_of_string() { insta::assert_snapshot!(QueryFixture::new("{ time_of: time_of(Strings.s)}").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time_of - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:39:57.000000000 - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:40:57.000000000 - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:41:57.000000000 - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:42:57.000000000 - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:43:57.000000000 - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:44:57.000000000 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1996-12-20T00:39:57.000000000 + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,1996-12-20T00:40:57.000000000 + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,1996-12-20T00:41:57.000000000 + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,1996-12-20T00:42:57.000000000 + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,1996-12-20T00:43:57.000000000 + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,1996-12-20T00:44:57.000000000 "###); } @@ -74,12 +74,12 @@ async fn test_time_of_string() { async fn test_time_of_record() { insta::assert_snapshot!(QueryFixture::new("{ time_of: time_of(Strings)}").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time_of - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:39:57.000000000 - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:40:57.000000000 - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:41:57.000000000 - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:42:57.000000000 - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:43:57.000000000 - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:44:57.000000000 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1996-12-20T00:39:57.000000000 + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,1996-12-20T00:40:57.000000000 + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,1996-12-20T00:41:57.000000000 + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,1996-12-20T00:42:57.000000000 + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,1996-12-20T00:43:57.000000000 + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,1996-12-20T00:44:57.000000000 "###); } @@ -87,12 +87,12 @@ async fn test_time_of_record() { async fn test_time_of_record_as_i64() { insta::assert_snapshot!(QueryFixture::new("{ time_of: time_of(Strings) as i64}").run_to_csv(&strings_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time_of - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,851042397000000000 - 1996-12-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,851042457000000000 - 1996-12-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,851042517000000000 - 1996-12-20T00:42:57.000000000,9223372036854775808,2867199309159137213,B,851042577000000000 - 1996-12-20T00:43:57.000000000,9223372036854775808,2867199309159137213,B,851042637000000000 - 1996-12-20T00:44:57.000000000,9223372036854775808,2867199309159137213,B,851042697000000000 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,851042397000000000 + 1996-12-20T00:40:57.000000000,0,2867199309159137213,B,851042457000000000 + 1996-12-20T00:41:57.000000000,0,2867199309159137213,B,851042517000000000 + 1996-12-20T00:42:57.000000000,0,2867199309159137213,B,851042577000000000 + 1996-12-20T00:43:57.000000000,0,2867199309159137213,B,851042637000000000 + 1996-12-20T00:44:57.000000000,0,2867199309159137213,B,851042697000000000 "###); } @@ -100,12 +100,12 @@ async fn test_time_of_record_as_i64() { async fn test_day_of_month() { insta::assert_snapshot!(QueryFixture::new("{ day_of_month: day_of_month(Times.time) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,day_of_month - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,20 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,20 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,20 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,12 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,13 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,6 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,20 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,20 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,20 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,12 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,13 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,6 "###); } @@ -113,12 +113,12 @@ async fn test_day_of_month() { async fn test_day_of_month0() { insta::assert_snapshot!(QueryFixture::new("{ day_of_month0: day_of_month0(Times.time) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,day_of_month0 - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,19 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,19 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,19 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,11 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,12 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,5 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,19 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,19 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,19 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,11 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,12 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,5 "###); } @@ -126,12 +126,12 @@ async fn test_day_of_month0() { async fn test_day_of_year() { insta::assert_snapshot!(QueryFixture::new("{ day_of_year: day_of_year(Times.time) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,day_of_year - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,354 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,293 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,233 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,346 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,347 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,341 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,354 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,293 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,233 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,346 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,347 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,341 "###); } @@ -139,12 +139,12 @@ async fn test_day_of_year() { async fn test_day_of_year0() { insta::assert_snapshot!(QueryFixture::new("{ day_of_year0: day_of_year0(Times.time) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,day_of_year0 - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,353 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,292 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,232 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,345 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,346 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,340 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,353 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,292 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,232 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,345 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,346 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,340 "###); } @@ -152,12 +152,12 @@ async fn test_day_of_year0() { async fn test_month_of_year() { insta::assert_snapshot!(QueryFixture::new("{ month_of_year: month_of_year(Times.time) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,month_of_year - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,12 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,10 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,8 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,12 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,12 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,12 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,12 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,10 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,8 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,12 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,12 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,12 "###); } @@ -165,12 +165,12 @@ async fn test_month_of_year() { async fn test_month_of_year0() { insta::assert_snapshot!(QueryFixture::new("{ month_of_year0: month_of_year0(Times.time) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,month_of_year0 - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,11 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,9 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,7 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,11 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,11 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,11 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,11 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,9 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,7 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,11 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,11 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,11 "###); } @@ -178,12 +178,12 @@ async fn test_month_of_year0() { async fn test_year() { insta::assert_snapshot!(QueryFixture::new("{ year: year(Times.time) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,year - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004 "###); } @@ -191,12 +191,12 @@ async fn test_year() { async fn test_add_time_duration_s() { insta::assert_snapshot!(QueryFixture::new("{ add_time: Times.time | add_time(seconds(Times.n)) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,add_time - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:59.000000000 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:41:01.000000000 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:42:02.000000000 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:44:05.000000000 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:45:20.000000000 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:59.000000000 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:41:01.000000000 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:42:02.000000000 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:44:05.000000000 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:45:20.000000000 "###); } @@ -205,12 +205,12 @@ async fn test_add_time_duration_s_to_literal() { // This ensures that a string literal may be treated as a timestamp. insta::assert_snapshot!(QueryFixture::new("{ add_time: \"1994-12-20T00:39:59.000000000Z\" | add_time(seconds(Times.n)) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,add_time - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:40:01.000000000 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1994-12-20T00:40:03.000000000 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1994-12-20T00:40:04.000000000 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1994-12-20T00:40:07.000000000 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,1994-12-20T00:40:22.000000000 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:40:01.000000000 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1994-12-20T00:40:03.000000000 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1994-12-20T00:40:04.000000000 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1994-12-20T00:40:07.000000000 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,1994-12-20T00:40:22.000000000 "###); } @@ -218,12 +218,12 @@ async fn test_add_time_duration_s_to_literal() { async fn test_add_time_duration_s_literal() { insta::assert_snapshot!(QueryFixture::new("{ add_time: Times.time | add_time(seconds(10000)) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,add_time - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T03:26:37.000000000 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T03:27:37.000000000 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T03:28:37.000000000 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T03:29:37.000000000 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T03:30:37.000000000 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T03:31:37.000000000 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T03:26:37.000000000 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T03:27:37.000000000 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T03:28:37.000000000 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T03:29:37.000000000 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T03:30:37.000000000 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T03:31:37.000000000 "###); } @@ -231,12 +231,12 @@ async fn test_add_time_duration_s_literal() { async fn test_add_time_interval_months() { insta::assert_snapshot!(QueryFixture::new("{ add_time: Times.time | add_time(months(Times.n)) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,add_time - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1995-02-20T00:39:57.000000000 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1996-02-20T00:40:57.000000000 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1997-01-20T00:41:57.000000000 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1999-08-13T00:43:57.000000000 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2006-11-06T00:44:57.000000000 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1995-02-20T00:39:57.000000000 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1996-02-20T00:40:57.000000000 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1997-01-20T00:41:57.000000000 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1999-08-13T00:43:57.000000000 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2006-11-06T00:44:57.000000000 "###); } @@ -244,12 +244,12 @@ async fn test_add_time_interval_months() { async fn test_add_time_interval_months_literal() { insta::assert_snapshot!(QueryFixture::new("{ add_time: Times.time | add_time(months(27)) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,add_time - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1997-03-20T00:39:57.000000000 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1998-01-20T00:40:57.000000000 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1998-11-20T00:41:57.000000000 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,2000-03-12T00:42:57.000000000 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,2001-03-13T00:43:57.000000000 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2007-03-06T00:44:57.000000000 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1997-03-20T00:39:57.000000000 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1998-01-20T00:40:57.000000000 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1998-11-20T00:41:57.000000000 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,2000-03-12T00:42:57.000000000 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,2001-03-13T00:43:57.000000000 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2007-03-06T00:44:57.000000000 "###); } @@ -257,12 +257,12 @@ async fn test_add_time_interval_months_literal() { async fn test_add_time_interval_months_literal_negative() { insta::assert_snapshot!(QueryFixture::new("{ add_time: Times.time | add_time(months(-1)) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,add_time - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-11-20T00:39:57.000000000 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-09-20T00:40:57.000000000 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-07-20T00:41:57.000000000 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-11-12T00:42:57.000000000 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-11-13T00:43:57.000000000 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-11-06T00:44:57.000000000 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-11-20T00:39:57.000000000 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-09-20T00:40:57.000000000 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-07-20T00:41:57.000000000 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-11-12T00:42:57.000000000 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-11-13T00:43:57.000000000 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-11-06T00:44:57.000000000 "###); } @@ -270,12 +270,12 @@ async fn test_add_time_interval_months_literal_negative() { async fn test_add_time_interval_days() { insta::assert_snapshot!(QueryFixture::new("{ add_time: Times.time | add_time(days(Times.n)) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,add_time - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-22T00:39:57.000000000 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-24T00:40:57.000000000 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-25T00:41:57.000000000 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B, - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-21T00:43:57.000000000 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-29T00:44:57.000000000 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-22T00:39:57.000000000 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-24T00:40:57.000000000 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-25T00:41:57.000000000 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B, + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-21T00:43:57.000000000 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-29T00:44:57.000000000 "###); } @@ -283,12 +283,12 @@ async fn test_add_time_interval_days() { async fn test_add_time_interval_days_literal() { insta::assert_snapshot!(QueryFixture::new("{ add_time: Times.time | add_time(days(372)) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,add_time - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1995-12-27T00:39:57.000000000 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1996-10-26T00:40:57.000000000 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1997-08-27T00:41:57.000000000 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-19T00:42:57.000000000 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1999-12-20T00:43:57.000000000 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2005-12-13T00:44:57.000000000 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1995-12-27T00:39:57.000000000 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1996-10-26T00:40:57.000000000 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1997-08-27T00:41:57.000000000 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1998-12-19T00:42:57.000000000 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1999-12-20T00:43:57.000000000 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2005-12-13T00:44:57.000000000 "###); } @@ -299,12 +299,12 @@ async fn test_seconds_between() { let seconds_between = seconds_between(time, other_time) as i64 in { time, other_time, seconds_between }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,other_time,seconds_between - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000,2003-12-20T00:39:57.000000000,283996800 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000,1994-11-20T00:39:57.000000000,-28857660 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000,1998-12-20T00:39:57.000000000,73612680 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000,1992-12-20T00:39:57.000000000,-157075380 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000,, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000,1994-12-20T00:39:57.000000000,-314409900 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000,2003-12-20T00:39:57.000000000,283996800 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000,1994-11-20T00:39:57.000000000,-28857660 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000,1998-12-20T00:39:57.000000000,73612680 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000,1992-12-20T00:39:57.000000000,-157075380 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000,, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000,1994-12-20T00:39:57.000000000,-314409900 "###); } @@ -315,12 +315,12 @@ async fn test_days_between() { let days_between = days_between(time, other_time) as i32 in { time, other_time, days_between }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,other_time,days_between - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000,2003-12-20T00:39:57.000000000,3287 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000,1994-11-20T00:39:57.000000000,-334 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000,1998-12-20T00:39:57.000000000,851 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000,1992-12-20T00:39:57.000000000,-1818 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000,, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000,1994-12-20T00:39:57.000000000,-3639 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000,2003-12-20T00:39:57.000000000,3287 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000,1994-11-20T00:39:57.000000000,-334 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000,1998-12-20T00:39:57.000000000,851 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000,1992-12-20T00:39:57.000000000,-1818 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000,, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000,1994-12-20T00:39:57.000000000,-3639 "###); // Tests that interval_days can cast to other types @@ -358,12 +358,12 @@ async fn test_months_between() { let months_between = months_between(time, other_time) as i32 in { time, other_time, months_between }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,other_time,months_between - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000,2003-12-20T00:39:57.000000000,108 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000,1994-11-20T00:39:57.000000000,-11 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000,1998-12-20T00:39:57.000000000,28 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000,1992-12-20T00:39:57.000000000,-60 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000,, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000,1994-12-20T00:39:57.000000000,-120 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000,2003-12-20T00:39:57.000000000,108 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000,1994-11-20T00:39:57.000000000,-11 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000,1998-12-20T00:39:57.000000000,28 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000,1992-12-20T00:39:57.000000000,-60 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000,, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000,1994-12-20T00:39:57.000000000,-120 "###); } @@ -374,12 +374,12 @@ async fn test_seconds_between_literal() { let seconds_between = seconds_between(time, \"1994-12-20T00:41:57.000000000-08:00\") as i64 in { time, seconds_between }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,seconds_between - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000,28920 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000,-26236740 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000,-52588800 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000,-93974460 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000,-125596920 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000,-314380980 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000,28920 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000,-26236740 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000,-52588800 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000,-93974460 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000,-125596920 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000,-314380980 "###); } @@ -390,12 +390,12 @@ async fn test_days_between_literal() { let days_between = days_between(time, \"1994-12-20T00:41:57.000000000-08:00\") as i32 in { time, days_between }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,days_between - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000,0 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000,-303 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000,-608 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000,-1087 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000,-1453 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000,-3638 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000,0 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000,-303 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000,-608 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000,-1087 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000,-1453 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000,-3638 "###); } @@ -406,12 +406,12 @@ async fn test_months_between_literal() { let months_between = months_between(time, \"1994-12-20T00:41:57.000000000-08:00\") as i32 in { time, months_between }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,months_between - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000,0 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000,-10 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000,-20 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000,-36 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000,-48 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000,-120 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000,0 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000,-10 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000,-20 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000,-36 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000,-48 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000,-120 "###); } @@ -441,12 +441,12 @@ async fn test_lag_0_i64() { async fn test_lag_1_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, lag_one: lag(1, Numbers.m) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,lag_one - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17,5 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,,5 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,17 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,17 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17,5 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,,5 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,17 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,17 "###) } @@ -454,12 +454,12 @@ async fn test_lag_1_i64() { async fn test_lag_2_i64() { insta::assert_snapshot!(QueryFixture::new("{ m: Numbers.m, lag_two: Numbers.m | lag(2) }").run_to_csv(&i64_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,m,lag_two - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,5, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,24, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,17, - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12,5 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,5 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,5, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,24, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,17, + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12,5 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,5 "###) } @@ -472,11 +472,11 @@ async fn test_mean_time_between() { in { prev, curr, elapsed, mean_elapsed: mean(elapsed) }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,prev,curr,elapsed,mean_elapsed - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,,1994-12-20T00:39:57.000000000,, - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,,1995-10-20T00:40:57.000000000,, - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000,1996-08-20T00:41:57.000000000,26352060,26352060.0 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000,1997-12-12T00:42:57.000000000,41385660,33868860.0 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000,1998-12-13T00:43:57.000000000,31622460,33120060.0 - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000,2004-12-06T00:44:57.000000000,188784060,72036060.0 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,,1994-12-20T00:39:57.000000000,, + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,,1995-10-20T00:40:57.000000000,, + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000,1996-08-20T00:41:57.000000000,26352060,26352060.0 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000,1997-12-12T00:42:57.000000000,41385660,33868860.0 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000,1998-12-13T00:43:57.000000000,31622460,33120060.0 + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000,2004-12-06T00:44:57.000000000,188784060,72036060.0 "###) } diff --git a/crates/sparrow-main/tests/e2e/when_tests.rs b/crates/sparrow-main/tests/e2e/when_tests.rs index 4b8adf52f..dc1f3b573 100644 --- a/crates/sparrow-main/tests/e2e/when_tests.rs +++ b/crates/sparrow-main/tests/e2e/when_tests.rs @@ -103,9 +103,9 @@ async fn test_when_output_resets_to_null() { async fn test_boolean_when() { insta::assert_snapshot!(QueryFixture::new("{ when: WhenFixture.bool | when(WhenFixture.cond) }").run_to_csv(&when_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,when - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A, + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A, "###); } @@ -115,7 +115,7 @@ async fn test_when_cond() { // rows in the last slice of the input. insta::assert_snapshot!(QueryFixture::new("WhenFixture | when(WhenFixture.i64 == 2)").run_to_csv(&when_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,subsort,key,cond,bool,i64,string - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:40:02.000000000,0,A,true,,2,hello + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,1996-12-20T00:40:02.000000000,0,A,true,,2,hello "###) } @@ -123,9 +123,9 @@ async fn test_when_cond() { async fn test_i64_when() { insta::assert_snapshot!(QueryFixture::new("{ when: WhenFixture.i64 | when(WhenFixture.cond) }").run_to_csv(&when_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,when - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,57 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,2 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,57 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,2 "###); } @@ -133,9 +133,9 @@ async fn test_i64_when() { async fn test_timestamp_when() { insta::assert_snapshot!(QueryFixture::new("{ when: WhenFixture.time | when(WhenFixture.cond) }").run_to_csv(&when_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,when - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:39:57.000000000 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:40:00.000000000 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:40:02.000000000 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1996-12-20T00:39:57.000000000 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,1996-12-20T00:40:00.000000000 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,1996-12-20T00:40:02.000000000 "###); } @@ -143,9 +143,9 @@ async fn test_timestamp_when() { async fn test_string_when() { insta::assert_snapshot!(QueryFixture::new("{ when: WhenFixture.string | when(WhenFixture.cond) }").run_to_csv(&when_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,when - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,hello - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,hello + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,hello + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,hello "###); } @@ -153,9 +153,9 @@ async fn test_string_when() { async fn test_record_when() { insta::assert_snapshot!(QueryFixture::new("WhenFixture | when($input.cond)").run_to_csv(&when_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,subsort,key,cond,bool,i64,string - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:39:57.000000000,0,A,true,false,57,hello - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:40:00.000000000,0,A,true,,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:40:02.000000000,0,A,true,,2,hello + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1996-12-20T00:39:57.000000000,0,A,true,false,57,hello + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,1996-12-20T00:40:00.000000000,0,A,true,,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,1996-12-20T00:40:02.000000000,0,A,true,,2,hello "###); } @@ -163,9 +163,9 @@ async fn test_record_when() { async fn test_record_when_chained() { insta::assert_snapshot!(QueryFixture::new("WhenFixture | when($input.cond) | when(WhenFixture.cond)").run_to_csv(&when_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,subsort,key,cond,bool,i64,string - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:39:57.000000000,0,A,true,false,57,hello - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:40:00.000000000,0,A,true,,, - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:40:02.000000000,0,A,true,,2,hello + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1996-12-20T00:39:57.000000000,0,A,true,false,57,hello + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,1996-12-20T00:40:00.000000000,0,A,true,,, + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,1996-12-20T00:40:02.000000000,0,A,true,,2,hello "###); } diff --git a/crates/sparrow-main/tests/e2e/windowed_aggregation_tests.rs b/crates/sparrow-main/tests/e2e/windowed_aggregation_tests.rs index 269d48684..24166da13 100644 --- a/crates/sparrow-main/tests/e2e/windowed_aggregation_tests.rs +++ b/crates/sparrow-main/tests/e2e/windowed_aggregation_tests.rs @@ -67,14 +67,14 @@ async fn window_data_fixture_with_nulls() -> DataFixture { async fn test_sliding_window_with_predicate() { insta::assert_snapshot!(QueryFixture::new("{ since: count(Foo, window=since(daily())), slide: Foo | count(window=sliding(2, $input | is_valid())) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,since,slide - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1,1 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,1,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,2,2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,3,2 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,4,2 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,5,2 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,6,2 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,7,2 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1,1 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,1,1 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,2,2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,3,2 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,4,2 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,5,2 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,6,2 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,7,2 "###); } @@ -91,14 +91,14 @@ async fn test_sliding_window_with_predicate_final_results() { async fn test_count_since_window() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, cond: Foo.n < 7.0, count: count(Foo.n), count_since: count(Foo.n, window=since(Foo.n < 7.0)) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,cond,count,count_since - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,false,1,1 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,true,1,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,6.2,true,2,2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,9.25,false,3,1 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,3.0,true,4,2 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,8.0,false,5,1 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,,,5,1 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,false,6,2 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10.0,false,1,1 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,3.9,true,1,1 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,6.2,true,2,2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,9.25,false,3,1 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,3.0,true,4,2 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,8.0,false,5,1 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,,,5,1 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,10.0,false,6,2 "###); } @@ -106,14 +106,14 @@ async fn test_count_since_window() { async fn test_sum_since_window() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, cond: Foo.n < 7.0, sum: sum(Foo.n), sum_since: sum(Foo.n, window=since(Foo.n < 7.0)) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,cond,sum,sum_since - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,false,10.0,10.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,true,3.9,3.9 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,6.2,true,16.2,16.2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,9.25,false,25.45,9.25 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,3.0,true,28.45,12.25 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,8.0,false,36.45,8.0 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,,,36.45,8.0 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,false,46.45,18.0 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10.0,false,10.0,10.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,3.9,true,3.9,3.9 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,6.2,true,16.2,16.2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,9.25,false,25.45,9.25 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,3.0,true,28.45,12.25 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,8.0,false,36.45,8.0 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,,,36.45,8.0 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,10.0,false,46.45,18.0 "###); } @@ -121,14 +121,14 @@ async fn test_sum_since_window() { async fn test_min_since_window() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, cond: Foo.n < 7.0, min: min(Foo.n), min_since: min(Foo.n, window=since(Foo.n < 7.0)) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,cond,min,min_since - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,false,10.0,10.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,true,3.9,3.9 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,6.2,true,6.2,6.2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,9.25,false,6.2,9.25 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,3.0,true,3.0,3.0 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,8.0,false,3.0,8.0 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,,,3.0,8.0 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,false,3.0,8.0 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10.0,false,10.0,10.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,3.9,true,3.9,3.9 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,6.2,true,6.2,6.2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,9.25,false,6.2,9.25 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,3.0,true,3.0,3.0 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,8.0,false,3.0,8.0 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,,,3.0,8.0 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,10.0,false,3.0,8.0 "###); } @@ -136,14 +136,14 @@ async fn test_min_since_window() { async fn test_max_since_window() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, cond: Foo.n < 7.0, max: max(Foo.n), max_since: max(Foo.n, window=since(Foo.n < 7.0)) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,cond,max,max_since - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,false,10.0,10.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,true,3.9,3.9 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,6.2,true,10.0,10.0 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,9.25,false,10.0,9.25 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,3.0,true,10.0,9.25 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,8.0,false,10.0,8.0 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,,,10.0,8.0 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,false,10.0,10.0 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10.0,false,10.0,10.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,3.9,true,3.9,3.9 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,6.2,true,10.0,10.0 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,9.25,false,10.0,9.25 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,3.0,true,10.0,9.25 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,8.0,false,10.0,8.0 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,,,10.0,8.0 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,10.0,false,10.0,10.0 "###); } @@ -151,14 +151,14 @@ async fn test_max_since_window() { async fn test_mean_since_window() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, cond: Foo.n < 7.0, mean: mean(Foo.n), mean_since: mean(Foo.n, window=since(Foo.n < 7.0)) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,cond,mean,mean_since - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,false,10.0,10.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,true,3.9,3.9 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,6.2,true,8.1,8.1 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,9.25,false,8.483333333333333,9.25 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,3.0,true,7.112499999999999,6.125 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,8.0,false,7.289999999999999,8.0 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,,,7.289999999999999,8.0 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,false,7.741666666666666,9.0 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10.0,false,10.0,10.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,3.9,true,3.9,3.9 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,6.2,true,8.1,8.1 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,9.25,false,8.483333333333333,9.25 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,3.0,true,7.112499999999999,6.125 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,8.0,false,7.289999999999999,8.0 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,,,7.289999999999999,8.0 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,10.0,false,7.741666666666666,9.0 "###); } @@ -166,14 +166,14 @@ async fn test_mean_since_window() { async fn test_variance_since_window() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, cond: Foo.n < 7.0, variance: variance(Foo.n), variance_since: variance(Foo.n, window=since(Foo.n < 7.0)) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,cond,variance,variance_since - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,false,, - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,true,, - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,6.2,true,3.609999999999999,3.609999999999999 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,9.25,false,2.7005555555555554, - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,3.0,true,7.662968749999997,9.765625 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,8.0,false,6.256399999999998, - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,,,6.256399999999998, - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,false,6.233680555555555,1.0 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10.0,false,, + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,3.9,true,, + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,6.2,true,3.609999999999999,3.609999999999999 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,9.25,false,2.7005555555555554, + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,3.0,true,7.662968749999997,9.765625 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,8.0,false,6.256399999999998, + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,,,6.256399999999998, + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,10.0,false,6.233680555555555,1.0 "###); } @@ -181,14 +181,14 @@ async fn test_variance_since_window() { async fn test_last_since_window() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, cond: Foo.n < 7.0, last: last(Foo.n), last_since: last(Foo.n, window=since(Foo.n < 7.0)) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,cond,last,last_since - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,false,10.0,10.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,true,3.9,3.9 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,6.2,true,6.2,6.2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,9.25,false,9.25,9.25 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,3.0,true,3.0,3.0 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,8.0,false,8.0,8.0 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,,,8.0,8.0 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,false,10.0,10.0 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10.0,false,10.0,10.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,3.9,true,3.9,3.9 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,6.2,true,6.2,6.2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,9.25,false,9.25,9.25 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,3.0,true,3.0,3.0 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,8.0,false,8.0,8.0 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,,,8.0,8.0 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,10.0,false,10.0,10.0 "###); } @@ -196,14 +196,14 @@ async fn test_last_since_window() { async fn test_f64_first_since_window() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, cond: Foo.n < 7.0, first: first(Foo.n), first_since: first(Foo.n, window=since(Foo.n < 7.0)) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,cond,first,first_since - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,false,10.0,10.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,true,3.9,3.9 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,6.2,true,10.0,10.0 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,9.25,false,10.0,9.25 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,3.0,true,10.0,9.25 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,8.0,false,10.0,8.0 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,,,10.0,8.0 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,false,10.0,8.0 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10.0,false,10.0,10.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,3.9,true,3.9,3.9 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,6.2,true,10.0,10.0 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,9.25,false,10.0,9.25 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,3.0,true,10.0,9.25 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,8.0,false,10.0,8.0 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,,,10.0,8.0 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,10.0,false,10.0,8.0 "###); } @@ -211,14 +211,14 @@ async fn test_f64_first_since_window() { async fn test_string_first_since_window() { insta::assert_snapshot!(QueryFixture::new("{ vegetable: Foo.vegetable, cond: Foo.n < 7.0, first: first(Foo.vegetable), first_since: first(Foo.vegetable, window=since(Foo.n < 7.0)) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,vegetable,cond,first,first_since - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,arugula,false,arugula,arugula - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,beet,true,beet,beet - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,carrot,true,arugula,arugula - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,dill,false,arugula,dill - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,edamame,true,arugula,dill - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,fennel,false,arugula,fennel - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,green beans,,arugula,fennel - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,habanero,false,arugula,fennel + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,arugula,false,arugula,arugula + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,beet,true,beet,beet + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,carrot,true,arugula,arugula + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,dill,false,arugula,dill + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,edamame,true,arugula,dill + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,fennel,false,arugula,fennel + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,green beans,,arugula,fennel + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,habanero,false,arugula,fennel "###); } @@ -226,14 +226,14 @@ async fn test_string_first_since_window() { async fn test_string_last_since_window() { insta::assert_snapshot!(QueryFixture::new("{ vegetable: Foo.vegetable, cond: Foo.n < 7.0, last: last(Foo.vegetable), last_since: last(Foo.vegetable, window=since(Foo.n < 7.0)) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,vegetable,cond,last,last_since - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,arugula,false,arugula,arugula - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,beet,true,beet,beet - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,carrot,true,carrot,carrot - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,dill,false,dill,dill - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,edamame,true,edamame,edamame - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,fennel,false,fennel,fennel - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,green beans,,green beans,green beans - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,habanero,false,habanero,habanero + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,arugula,false,arugula,arugula + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,beet,true,beet,beet + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,carrot,true,carrot,carrot + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,dill,false,dill,dill + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,edamame,true,edamame,edamame + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,fennel,false,fennel,fennel + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,green beans,,green beans,green beans + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,habanero,false,habanero,habanero "###); } @@ -241,14 +241,14 @@ async fn test_string_last_since_window() { async fn test_bool_first_since_window() { insta::assert_snapshot!(QueryFixture::new("{ bool: Foo.bool, cond: Foo.n < 7.0, first: first(Foo.bool), first_since: first(Foo.bool, window=since(Foo.n < 7.0)) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,bool,cond,first,first_since - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,false,true,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,true,true,true,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,false,true,true,true - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,false,false,true,false - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,true,true,true,false - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,false,false,true,false - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,true,,true,false - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,false,false,true,false + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,false,true,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,true,true,true,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,false,true,true,true + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,false,false,true,false + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,true,true,true,false + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,false,false,true,false + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,true,,true,false + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,false,false,true,false "###); } @@ -256,14 +256,14 @@ async fn test_bool_first_since_window() { async fn test_bool_last_since_window() { insta::assert_snapshot!(QueryFixture::new("{ bool: Foo.bool, cond: Foo.n < 7.0, last: last(Foo.bool), last_since: last(Foo.bool, window=since(Foo.n < 7.0)) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,bool,cond,last,last_since - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,false,true,true - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,true,true,true,true - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,false,true,false,false - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,false,false,false,false - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,true,true,true,true - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,false,false,false,false - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,true,,true,true - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,false,false,false,false + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,false,true,true + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,true,true,true,true + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,false,true,false,false + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,false,false,false,false + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,true,true,true,true + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,false,false,false,false + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,true,,true,true + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,false,false,false,false "###); } @@ -273,14 +273,14 @@ async fn test_bool_last_since_window() { async fn test_first_since_window_emits_value_on_reset() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, cond: Foo.bool, first_since: first(Foo.n, window=since(Foo.bool)) }").run_to_csv(&window_data_fixture_with_nulls().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,cond,first_since - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,true,10.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,true,3.9 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,,false, - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,9.25,,9.25 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,,,9.25 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,false,9.25 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,1.0,true,9.25 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,true,10.0 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10.0,true,10.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,3.9,true,3.9 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,,false, + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,9.25,,9.25 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,,,9.25 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,false,9.25 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,1.0,true,9.25 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,10.0,true,10.0 "###); } @@ -308,14 +308,14 @@ async fn test_first_sliding_window_emits_value_on_reset() { async fn test_last_since_window_emits_value_on_reset() { insta::assert_snapshot!(QueryFixture::new("{ n: Foo.n, cond: Foo.bool, last_since: last(Foo.n, window=since(Foo.bool)) }").run_to_csv(&window_data_fixture_with_nulls().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,n,cond,last_since - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0,true,10.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9,true,3.9 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,,false, - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,9.25,,9.25 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,,,9.25 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,,false,9.25 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,1.0,true,1.0 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0,true,10.0 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10.0,true,10.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,3.9,true,3.9 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,,false, + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,9.25,,9.25 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,,,9.25 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,,false,9.25 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,1.0,true,1.0 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,10.0,true,10.0 "###); } @@ -341,14 +341,14 @@ async fn test_last_sliding_window_emits_value_on_reset() { async fn test_count_sliding_window_every_few_events() { insta::assert_snapshot!(QueryFixture::new("{ cond: is_valid(Foo), total_count: count(Foo), sliding_count: count(Foo, window=sliding(3, is_valid(Foo))) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,cond,total_count,sliding_count - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,1,1 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,true,1,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,true,2,2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,true,3,3 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,true,4,3 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,true,5,3 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,true,6,3 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,true,7,3 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,1,1 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,true,1,1 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,true,2,2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,true,3,3 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,true,4,3 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,true,5,3 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,true,6,3 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,true,7,3 "###); } @@ -356,14 +356,14 @@ async fn test_count_sliding_window_every_few_events() { async fn test_count_sliding_window_with_condition() { insta::assert_snapshot!(QueryFixture::new("{ cond: Foo.n > 5, sliding_count: count(Foo.n, window=sliding(2, Foo.n > 5)) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,cond,sliding_count - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,true,1 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,false,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,true,2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,true,2 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,false,2 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,true,3 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,,2 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,true,3 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,true,1 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,false,1 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,true,2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,true,2 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,false,2 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,true,3 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,,2 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,true,3 "###); } @@ -371,14 +371,14 @@ async fn test_count_sliding_window_with_condition() { async fn test_count_sliding_duration_1_equivalent_to_since() { insta::assert_snapshot!(QueryFixture::new("{ since: count(Foo.bool, window=since(Foo.n > 5)), sliding: count(Foo.bool, window=sliding(1, Foo.n > 5)) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,since,sliding - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1,1 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,1,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,1,1 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,1,1 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,1,1 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,2,2 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,1,1 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,2,2 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,1,1 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,1,1 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,1,1 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,1,1 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,1,1 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,2,2 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,1,1 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,2,2 "###); } @@ -386,14 +386,14 @@ async fn test_count_sliding_duration_1_equivalent_to_since() { async fn test_sum_sliding_every_few_events() { insta::assert_snapshot!(QueryFixture::new("{ sliding_sum: sum(Foo.n, window=sliding(2, is_valid(Foo))) }").run_to_csv(&window_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sliding_sum - 1996-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,10.0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,3.9 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,16.2 - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,15.45 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,12.25 - 1996-12-20T00:40:02.000000000,9223372036854775808,12960666915911099378,A,11.0 - 1996-12-20T00:40:03.000000000,9223372036854775808,12960666915911099378,A,8.0 - 1996-12-20T00:40:04.000000000,9223372036854775808,12960666915911099378,A,10.0 + 1996-12-20T00:39:57.000000000,0,12960666915911099378,A,10.0 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,3.9 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,16.2 + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,15.45 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,12.25 + 1996-12-20T00:40:02.000000000,0,12960666915911099378,A,11.0 + 1996-12-20T00:40:03.000000000,0,12960666915911099378,A,8.0 + 1996-12-20T00:40:04.000000000,0,12960666915911099378,A,10.0 "###); } diff --git a/crates/sparrow-main/tests/e2e/with_key_tests.rs b/crates/sparrow-main/tests/e2e/with_key_tests.rs index 47fd51710..cb0f7c289 100644 --- a/crates/sparrow-main/tests/e2e/with_key_tests.rs +++ b/crates/sparrow-main/tests/e2e/with_key_tests.rs @@ -47,12 +47,12 @@ pub(crate) async fn with_key_data_fixture() -> DataFixture { async fn test_with_key_i64_pipe() { insta::assert_snapshot!(QueryFixture::new("Table | with_key($input.foreign_key_i64)").run_to_csv(&with_key_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,subsort,key,foreign_key_i64,foreign_key_str,n - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,1996-12-20T00:39:57.000000000,0,A,0,B,0 - 1996-12-20T00:39:58.000000000,9223372036854775808,18433805721903975440,1,1996-12-20T00:39:58.000000000,0,B,1,A,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,2694864431690786590,2,1996-12-20T00:39:59.000000000,0,A,2,, - 1996-12-20T00:40:00.000000000,9223372036854775808,2694864431690786590,2,1996-12-20T00:40:00.000000000,0,A,2,C,2 - 1996-12-20T00:40:01.000000000,9223372036854775808,18433805721903975440,1,1996-12-20T00:40:01.000000000,0,A,1,A,3 - 1996-12-20T00:40:02.000000000,9223372036854775808,11832085162654999889,0,1996-12-20T00:40:02.000000000,0,A,0,B,4 + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,1996-12-20T00:39:57.000000000,0,A,0,B,0 + 1996-12-20T00:39:58.000000000,0,18433805721903975440,1,1996-12-20T00:39:58.000000000,0,B,1,A,1 + 1996-12-20T00:39:59.000000000,0,2694864431690786590,2,1996-12-20T00:39:59.000000000,0,A,2,, + 1996-12-20T00:40:00.000000000,0,2694864431690786590,2,1996-12-20T00:40:00.000000000,0,A,2,C,2 + 1996-12-20T00:40:01.000000000,0,18433805721903975440,1,1996-12-20T00:40:01.000000000,0,A,1,A,3 + 1996-12-20T00:40:02.000000000,0,11832085162654999889,0,1996-12-20T00:40:02.000000000,0,A,0,B,4 "###); } @@ -60,10 +60,10 @@ async fn test_with_key_i64_pipe() { async fn test_with_key_lookup_select() { insta::assert_snapshot!(QueryFixture::new("Table | with_key($input.foreign_key_i64) | last() | lookup(Table.foreign_key_i64) | when($input.foreign_key_i64 > 0)").run_to_csv(&with_key_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,subsort,key,foreign_key_i64,foreign_key_str,n - 1996-12-20T00:39:58.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:39:58.000000000,0,B,1,A,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:39:59.000000000,0,A,2,, - 1996-12-20T00:40:00.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:40:00.000000000,0,A,2,C,2 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:40:01.000000000,0,A,1,A,3 + 1996-12-20T00:39:58.000000000,0,2867199309159137213,B,1996-12-20T00:39:58.000000000,0,B,1,A,1 + 1996-12-20T00:39:59.000000000,0,12960666915911099378,A,1996-12-20T00:39:59.000000000,0,A,2,, + 1996-12-20T00:40:00.000000000,0,12960666915911099378,A,1996-12-20T00:40:00.000000000,0,A,2,C,2 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,1996-12-20T00:40:01.000000000,0,A,1,A,3 "###); } @@ -71,12 +71,12 @@ async fn test_with_key_lookup_select() { async fn test_with_key_i64() { insta::assert_snapshot!(QueryFixture::new("with_key(Table.foreign_key_i64, Table)").run_to_csv(&with_key_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,subsort,key,foreign_key_i64,foreign_key_str,n - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,1996-12-20T00:39:57.000000000,0,A,0,B,0 - 1996-12-20T00:39:58.000000000,9223372036854775808,18433805721903975440,1,1996-12-20T00:39:58.000000000,0,B,1,A,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,2694864431690786590,2,1996-12-20T00:39:59.000000000,0,A,2,, - 1996-12-20T00:40:00.000000000,9223372036854775808,2694864431690786590,2,1996-12-20T00:40:00.000000000,0,A,2,C,2 - 1996-12-20T00:40:01.000000000,9223372036854775808,18433805721903975440,1,1996-12-20T00:40:01.000000000,0,A,1,A,3 - 1996-12-20T00:40:02.000000000,9223372036854775808,11832085162654999889,0,1996-12-20T00:40:02.000000000,0,A,0,B,4 + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,1996-12-20T00:39:57.000000000,0,A,0,B,0 + 1996-12-20T00:39:58.000000000,0,18433805721903975440,1,1996-12-20T00:39:58.000000000,0,B,1,A,1 + 1996-12-20T00:39:59.000000000,0,2694864431690786590,2,1996-12-20T00:39:59.000000000,0,A,2,, + 1996-12-20T00:40:00.000000000,0,2694864431690786590,2,1996-12-20T00:40:00.000000000,0,A,2,C,2 + 1996-12-20T00:40:01.000000000,0,18433805721903975440,1,1996-12-20T00:40:01.000000000,0,A,1,A,3 + 1996-12-20T00:40:02.000000000,0,11832085162654999889,0,1996-12-20T00:40:02.000000000,0,A,0,B,4 "###); } @@ -89,12 +89,12 @@ async fn test_with_key_aggregate_select() { "{ sum: Table.n | when(Table.key == 'A') | sum() | with_key(Table.foreign_key_i64) }" ).run_to_csv(&with_key_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,sum - 1996-12-20T00:39:57.000000000,9223372036854775808,11832085162654999889,0,0 - 1996-12-20T00:39:58.000000000,9223372036854775808,18433805721903975440,1, - 1996-12-20T00:39:59.000000000,9223372036854775808,2694864431690786590,2,0 - 1996-12-20T00:40:00.000000000,9223372036854775808,2694864431690786590,2,2 - 1996-12-20T00:40:01.000000000,9223372036854775808,18433805721903975440,1,5 - 1996-12-20T00:40:02.000000000,9223372036854775808,11832085162654999889,0,9 + 1996-12-20T00:39:57.000000000,0,11832085162654999889,0,0 + 1996-12-20T00:39:58.000000000,0,18433805721903975440,1, + 1996-12-20T00:39:59.000000000,0,2694864431690786590,2,0 + 1996-12-20T00:40:00.000000000,0,2694864431690786590,2,2 + 1996-12-20T00:40:01.000000000,0,18433805721903975440,1,5 + 1996-12-20T00:40:02.000000000,0,11832085162654999889,0,9 "###) } @@ -106,7 +106,7 @@ async fn test_with_key_i64_parquet_output() { .run_to_parquet_hash(&with_key_data_fixture().await) .await .unwrap(), - @"ECDEFE2C9B5BC4F19875EBE6D81EE5573500E9984CC42C91CD17E6FF" + @"693FC33EDA127F2CF6E896E4424B28FAEBF07C56205F014FA2E303F9" ) } @@ -114,12 +114,12 @@ async fn test_with_key_i64_parquet_output() { async fn test_with_computed_key_i64() { insta::assert_snapshot!(QueryFixture::new("with_key(Table.foreign_key_i64 + 1, Table)").run_to_csv(&with_key_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,subsort,key,foreign_key_i64,foreign_key_str,n - 1996-12-20T00:39:57.000000000,9223372036854775808,18433805721903975440,1,1996-12-20T00:39:57.000000000,0,A,0,B,0 - 1996-12-20T00:39:58.000000000,9223372036854775808,2694864431690786590,2,1996-12-20T00:39:58.000000000,0,B,1,A,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,5496774745203840792,3,1996-12-20T00:39:59.000000000,0,A,2,, - 1996-12-20T00:40:00.000000000,9223372036854775808,5496774745203840792,3,1996-12-20T00:40:00.000000000,0,A,2,C,2 - 1996-12-20T00:40:01.000000000,9223372036854775808,2694864431690786590,2,1996-12-20T00:40:01.000000000,0,A,1,A,3 - 1996-12-20T00:40:02.000000000,9223372036854775808,18433805721903975440,1,1996-12-20T00:40:02.000000000,0,A,0,B,4 + 1996-12-20T00:39:57.000000000,0,18433805721903975440,1,1996-12-20T00:39:57.000000000,0,A,0,B,0 + 1996-12-20T00:39:58.000000000,0,2694864431690786590,2,1996-12-20T00:39:58.000000000,0,B,1,A,1 + 1996-12-20T00:39:59.000000000,0,5496774745203840792,3,1996-12-20T00:39:59.000000000,0,A,2,, + 1996-12-20T00:40:00.000000000,0,5496774745203840792,3,1996-12-20T00:40:00.000000000,0,A,2,C,2 + 1996-12-20T00:40:01.000000000,0,2694864431690786590,2,1996-12-20T00:40:01.000000000,0,A,1,A,3 + 1996-12-20T00:40:02.000000000,0,18433805721903975440,1,1996-12-20T00:40:02.000000000,0,A,0,B,4 "###); } @@ -127,12 +127,12 @@ async fn test_with_computed_key_i64() { async fn test_with_computed_key_str() { insta::assert_snapshot!(QueryFixture::new("with_key(Table.foreign_key_str, Table)").run_to_csv(&with_key_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,subsort,key,foreign_key_i64,foreign_key_str,n - 1996-12-20T00:39:57.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:39:57.000000000,0,A,0,B,0 - 1996-12-20T00:39:58.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:39:58.000000000,0,B,1,A,1 - 1996-12-20T00:39:59.000000000,9223372036854775808,5663277146615294718,,1996-12-20T00:39:59.000000000,0,A,2,, - 1996-12-20T00:40:00.000000000,9223372036854775808,2521269998124177631,C,1996-12-20T00:40:00.000000000,0,A,2,C,2 - 1996-12-20T00:40:01.000000000,9223372036854775808,12960666915911099378,A,1996-12-20T00:40:01.000000000,0,A,1,A,3 - 1996-12-20T00:40:02.000000000,9223372036854775808,2867199309159137213,B,1996-12-20T00:40:02.000000000,0,A,0,B,4 + 1996-12-20T00:39:57.000000000,0,2867199309159137213,B,1996-12-20T00:39:57.000000000,0,A,0,B,0 + 1996-12-20T00:39:58.000000000,0,12960666915911099378,A,1996-12-20T00:39:58.000000000,0,B,1,A,1 + 1996-12-20T00:39:59.000000000,0,5663277146615294718,,1996-12-20T00:39:59.000000000,0,A,2,, + 1996-12-20T00:40:00.000000000,0,2521269998124177631,C,1996-12-20T00:40:00.000000000,0,A,2,C,2 + 1996-12-20T00:40:01.000000000,0,12960666915911099378,A,1996-12-20T00:40:01.000000000,0,A,1,A,3 + 1996-12-20T00:40:02.000000000,0,2867199309159137213,B,1996-12-20T00:40:02.000000000,0,A,0,B,4 "###); } From 15bc824d5909cc3d03e5f46df1ada9710ad6db93 Mon Sep 17 00:00:00 2001 From: Jordan Frazier Date: Tue, 26 Sep 2023 22:41:28 -0700 Subject: [PATCH 6/7] fix a few more tests --- crates/sparrow-main/tests/e2e/time_tests.rs | 24 +++++++++---------- .../sparrow-runtime/src/prepare/preparer.rs | 1 - 2 files changed, 12 insertions(+), 13 deletions(-) diff --git a/crates/sparrow-main/tests/e2e/time_tests.rs b/crates/sparrow-main/tests/e2e/time_tests.rs index 997fdb266..60af97611 100644 --- a/crates/sparrow-main/tests/e2e/time_tests.rs +++ b/crates/sparrow-main/tests/e2e/time_tests.rs @@ -329,12 +329,12 @@ async fn test_days_between() { let days_between = days_between(time, other_time) as i64 in { time, other_time, days_between }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,other_time,days_between - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000,2003-12-20T00:39:57.000000000,3287 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000,1994-11-20T00:39:57.000000000,-334 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000,1998-12-20T00:39:57.000000000,851 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000,1992-12-20T00:39:57.000000000,-1818 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000,, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000,1994-12-20T00:39:57.000000000,-3639 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000,2003-12-20T00:39:57.000000000,3287 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000,1994-11-20T00:39:57.000000000,-334 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000,1998-12-20T00:39:57.000000000,851 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000,1992-12-20T00:39:57.000000000,-1818 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000,, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000,1994-12-20T00:39:57.000000000,-3639 "###); insta::assert_snapshot!(QueryFixture::new("let time = Times.time @@ -342,12 +342,12 @@ async fn test_days_between() { let days_between = days_between(time, other_time) as f32 in { time, other_time, days_between }").run_to_csv(×tamp_ns_data_fixture().await).await.unwrap(), @r###" _time,_subsort,_key_hash,_key,time,other_time,days_between - 1994-12-20T00:39:57.000000000,9223372036854775808,12960666915911099378,A,1994-12-20T00:39:57.000000000,2003-12-20T00:39:57.000000000,3287.0 - 1995-10-20T00:40:57.000000000,9223372036854775808,2867199309159137213,B,1995-10-20T00:40:57.000000000,1994-11-20T00:39:57.000000000,-334.0 - 1996-08-20T00:41:57.000000000,9223372036854775808,2867199309159137213,B,1996-08-20T00:41:57.000000000,1998-12-20T00:39:57.000000000,851.0 - 1997-12-12T00:42:57.000000000,9223372036854775808,2867199309159137213,B,1997-12-12T00:42:57.000000000,1992-12-20T00:39:57.000000000,-1818.0 - 1998-12-13T00:43:57.000000000,9223372036854775808,2867199309159137213,B,1998-12-13T00:43:57.000000000,, - 2004-12-06T00:44:57.000000000,9223372036854775808,2867199309159137213,B,2004-12-06T00:44:57.000000000,1994-12-20T00:39:57.000000000,-3639.0 + 1994-12-20T00:39:57.000000000,0,12960666915911099378,A,1994-12-20T00:39:57.000000000,2003-12-20T00:39:57.000000000,3287.0 + 1995-10-20T00:40:57.000000000,0,2867199309159137213,B,1995-10-20T00:40:57.000000000,1994-11-20T00:39:57.000000000,-334.0 + 1996-08-20T00:41:57.000000000,0,2867199309159137213,B,1996-08-20T00:41:57.000000000,1998-12-20T00:39:57.000000000,851.0 + 1997-12-12T00:42:57.000000000,0,2867199309159137213,B,1997-12-12T00:42:57.000000000,1992-12-20T00:39:57.000000000,-1818.0 + 1998-12-13T00:43:57.000000000,0,2867199309159137213,B,1998-12-13T00:43:57.000000000,, + 2004-12-06T00:44:57.000000000,0,2867199309159137213,B,2004-12-06T00:44:57.000000000,1994-12-20T00:39:57.000000000,-3639.0 "###); } diff --git a/crates/sparrow-runtime/src/prepare/preparer.rs b/crates/sparrow-runtime/src/prepare/preparer.rs index f3a2f7d79..821756183 100644 --- a/crates/sparrow-runtime/src/prepare/preparer.rs +++ b/crates/sparrow-runtime/src/prepare/preparer.rs @@ -11,7 +11,6 @@ use error_stack::{IntoReport, IntoReportCompat, ResultExt}; use futures::stream::FuturesUnordered; use futures::{StreamExt, TryStreamExt}; use sparrow_api::kaskada::v1alpha::{PreparedFile, SourceData, TableConfig}; -use sparrow_kernels::order_preserving_cast_to_u64; use uuid::Uuid; use crate::stores::{ObjectStoreRegistry, ObjectStoreUrl}; From 1b2bf063ed322381d6c3badccab440c90306f78c Mon Sep 17 00:00:00 2001 From: Jordan Frazier Date: Wed, 27 Sep 2023 09:29:36 -0700 Subject: [PATCH 7/7] move atomic subsort outside of loop in execute stream case --- crates/sparrow-runtime/src/prepare/execute_input_stream.rs | 2 +- crates/sparrow-runtime/src/prepare/prepare_input_stream.rs | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/crates/sparrow-runtime/src/prepare/execute_input_stream.rs b/crates/sparrow-runtime/src/prepare/execute_input_stream.rs index cd29452c2..45a211e36 100644 --- a/crates/sparrow-runtime/src/prepare/execute_input_stream.rs +++ b/crates/sparrow-runtime/src/prepare/execute_input_stream.rs @@ -80,6 +80,7 @@ pub async fn prepare_input<'a>( slice, )?; + let next_subsort = AtomicU64::new(prepare_hash); Ok(async_stream::try_stream! { let mut input_buffer = InputBuffer::new(); while let Some(unfiltered_batch) = reader.next().await { @@ -153,7 +154,6 @@ pub async fn prepare_input<'a>( let record_batch = slice_preparer.slice_batch(record_batch)?; // 3. Prepare the batch - let next_subsort = AtomicU64::new(prepare_hash); let record_batch = prepare_batch(&record_batch, &config, prepared_schema.clone(), &next_subsort, None).unwrap(); // 4. Update the key inverse diff --git a/crates/sparrow-runtime/src/prepare/prepare_input_stream.rs b/crates/sparrow-runtime/src/prepare/prepare_input_stream.rs index be91fce01..77e779376 100644 --- a/crates/sparrow-runtime/src/prepare/prepare_input_stream.rs +++ b/crates/sparrow-runtime/src/prepare/prepare_input_stream.rs @@ -51,6 +51,7 @@ pub async fn prepare_input<'a>( let mut metadata = PrepareMetadata::new(entity_key_column.data_type().clone()); let next_subsort = AtomicU64::new(prepare_hash); + Ok(async_stream::try_stream! { while let Some(Ok(batch)) = reader.next().await { // 1. Slicing may reduce the number of entities to operate and sort on.