diff --git a/generated_types/protos/influxdata/iox/ingester/v1/query.proto b/generated_types/protos/influxdata/iox/ingester/v1/query.proto index ff7cc66209..fc0ca483f2 100644 --- a/generated_types/protos/influxdata/iox/ingester/v1/query.proto +++ b/generated_types/protos/influxdata/iox/ingester/v1/query.proto @@ -82,8 +82,9 @@ message PartitionStatus { // Max sequence number persisted optional int64 parquet_max_sequence_number = 1; - // Max sequence number for a tombstone associated - optional int64 tombstone_max_sequence_number = 2; + // Deprecated tombstone support in ingester (#5825). + reserved "tombstone_max_sequence_number"; + reserved 2; } // Serialization of `predicate::predicate::Predicate` that contains DataFusion `Expr`s diff --git a/influxdb_iox/tests/end_to_end_cases/ingester.rs b/influxdb_iox/tests/end_to_end_cases/ingester.rs index 07ecd8fbbe..edf93bb305 100644 --- a/influxdb_iox/tests/end_to_end_cases/ingester.rs +++ b/influxdb_iox/tests/end_to_end_cases/ingester.rs @@ -52,7 +52,6 @@ async fn ingester_flight_api() { partition_id, status: Some(PartitionStatus { parquet_max_sequence_number: None, - tombstone_max_sequence_number: None }) }, ); diff --git a/ingester/src/compact.rs b/ingester/src/compact.rs index ce516ffe85..8a280cc751 100644 --- a/ingester/src/compact.rs +++ b/ingester/src/compact.rs @@ -18,7 +18,7 @@ use crate::{data::partition::PersistingBatch, query::QueryableBatch}; #[derive(Debug, Snafu)] #[allow(missing_copy_implementations, missing_docs)] -pub enum Error { +pub(crate) enum Error { #[snafu(display("Error while building logical plan for Ingester's compaction"))] LogicalPlan { source: iox_query::frontend::reorg::Error, @@ -189,8 +189,8 @@ mod tests { create_batches_with_influxtype_same_columns_different_type, create_one_record_batch_with_influxtype_duplicates, create_one_record_batch_with_influxtype_no_duplicates, - create_one_row_record_batch_with_influxtype, create_tombstone, make_meta, - make_persisting_batch, make_queryable_batch, make_queryable_batch_with_deletes, + create_one_row_record_batch_with_influxtype, make_meta, make_persisting_batch, + make_queryable_batch, }; // this test was added to guard against https://github.com/influxdata/influxdb_iox/issues/3782 @@ -223,7 +223,6 @@ mod tests { partition_id, uuid, batches, - vec![], ); // verify PK @@ -293,7 +292,6 @@ mod tests { partition_id, uuid, batches, - vec![], ); // verify PK @@ -389,7 +387,6 @@ mod tests { partition_id, uuid, batches, - vec![], ); // verify PK @@ -488,7 +485,6 @@ mod tests { partition_id, uuid, batches, - vec![], ); // verify PK @@ -588,7 +584,6 @@ mod tests { partition_id, uuid, batches, - vec![], ); // verify PK @@ -692,7 +687,6 @@ mod tests { partition_id, uuid, batches, - vec![], ); // verify PK @@ -816,54 +810,6 @@ mod tests { assert_batches_eq!(&expected, &output_batches); } - #[tokio::test] - async fn test_compact_one_batch_no_dupilcates_with_deletes() { - test_helpers::maybe_start_logging(); - - // create input data - let batches = create_one_record_batch_with_influxtype_no_duplicates().await; - let tombstones = vec![create_tombstone(1, 1, 1, 1, 0, 200000, "tag1=UT")]; - - // build queryable batch from the input batches - let compact_batch = - make_queryable_batch_with_deletes("test_table", 0, 1, batches, tombstones); - - // verify PK - let schema = compact_batch.schema(); - let pk = schema.primary_key(); - let expected_pk = vec!["tag1", "time"]; - assert_eq!(expected_pk, pk); - - let sort_key = compute_sort_key( - &schema, - compact_batch.data.iter().map(|sb| sb.data.as_ref()), - ); - assert_eq!(sort_key, SortKey::from_columns(["tag1", "time"])); - - // compact - let exc = Executor::new(1); - let stream = compact(&exc, compact_batch, sort_key).await.unwrap(); - let output_batches = datafusion::physical_plan::common::collect(stream) - .await - .unwrap(); - // verify no empty record batches - bug #3782 - assert_eq!(output_batches.len(), 2); - assert_eq!(output_batches[0].num_rows(), 1); - assert_eq!(output_batches[1].num_rows(), 1); - - // verify compacted data - // row with "tag1=UT" no longer available - let expected = vec![ - "+-----------+------+-----------------------------+", - "| field_int | tag1 | time |", - "+-----------+------+-----------------------------+", - "| 10 | VT | 1970-01-01T00:00:00.000010Z |", - "| 1000 | WA | 1970-01-01T00:00:00.000008Z |", - "+-----------+------+-----------------------------+", - ]; - assert_batches_eq!(&expected, &output_batches); - } - #[tokio::test] async fn test_compact_one_batch_with_duplicates() { // create input data @@ -1010,23 +956,12 @@ mod tests { } #[tokio::test] - async fn test_compact_many_batches_different_columns_different_order_with_duplicates_with_deletes( - ) { + async fn test_compact_many_batches_different_columns_different_order_with_duplicates() { // create many-batches input data let batches = create_batches_with_influxtype_different_columns_different_order().await; - let tombstones = vec![create_tombstone( - 1, - 1, - 1, - 100, // delete's seq_number - 0, // min time of data to get deleted - 200000, // max time of data to get deleted - "tag2=CT and field_int=1000", // delete predicate - )]; // build queryable batch from the input batches - let compact_batch = - make_queryable_batch_with_deletes("test_table", 0, 1, batches, tombstones); + let compact_batch = make_queryable_batch("test_table", 0, 1, batches); // verify PK let schema = compact_batch.schema(); @@ -1049,7 +984,6 @@ mod tests { // verify compacted data // data is sorted and all duplicates are removed - // all rows with ("tag2=CT and field_int=1000") are also removed // CORRECT RESULT let expected = vec![ "+-----------+------+------+--------------------------------+", @@ -1058,73 +992,15 @@ mod tests { "| 5 | | AL | 1970-01-01T00:00:00.000005Z |", "| 10 | | AL | 1970-01-01T00:00:00.000007Z |", "| 70 | | CT | 1970-01-01T00:00:00.000000100Z |", + "| 1000 | | CT | 1970-01-01T00:00:00.000001Z |", "| 100 | | MA | 1970-01-01T00:00:00.000000050Z |", "| 10 | AL | MA | 1970-01-01T00:00:00.000000050Z |", "| 70 | CT | CT | 1970-01-01T00:00:00.000000100Z |", "| 70 | CT | CT | 1970-01-01T00:00:00.000000500Z |", "| 30 | MT | AL | 1970-01-01T00:00:00.000000005Z |", "| 20 | MT | AL | 1970-01-01T00:00:00.000007Z |", - "+-----------+------+------+--------------------------------+", - ]; - - assert_batches_eq!(&expected, &output_batches); - } - - #[tokio::test] - async fn test_compact_many_batches_different_columns_different_order_with_duplicates_with_many_deletes( - ) { - // create many-batches input data - let batches = create_batches_with_influxtype_different_columns_different_order().await; - let tombstones = vec![ - create_tombstone( - 1, - 1, - 1, - 100, // delete's seq_number - 0, // min time of data to get deleted - 200000, // max time of data to get deleted - "tag2=CT and field_int=1000", // delete predicate - ), - create_tombstone( - 1, 1, 1, 101, // delete's seq_number - 0, // min time of data to get deleted - 200000, // max time of data to get deleted - "tag1!=MT", // delete predicate - ), - ]; - - // build queryable batch from the input batches - let compact_batch = - make_queryable_batch_with_deletes("test_table", 0, 1, batches, tombstones); - - // verify PK - let schema = compact_batch.schema(); - let pk = schema.primary_key(); - let expected_pk = vec!["tag1", "tag2", "time"]; - assert_eq!(expected_pk, pk); - - let sort_key = compute_sort_key( - &schema, - compact_batch.data.iter().map(|sb| sb.data.as_ref()), - ); - assert_eq!(sort_key, SortKey::from_columns(["tag1", "tag2", "time"])); - - // compact - let exc = Executor::new(1); - let stream = compact(&exc, compact_batch, sort_key).await.unwrap(); - let output_batches = datafusion::physical_plan::common::collect(stream) - .await - .unwrap(); - - // verify compacted data - // data is sorted and all duplicates are removed - // all rows with ("tag2=CT and field_int=1000") and ("tag1!=MT") are also removed - let expected = vec![ - "+-----------+------+------+--------------------------------+", - "| field_int | tag1 | tag2 | time |", - "+-----------+------+------+--------------------------------+", - "| 30 | MT | AL | 1970-01-01T00:00:00.000000005Z |", - "| 20 | MT | AL | 1970-01-01T00:00:00.000007Z |", + "| 1000 | MT | CT | 1970-01-01T00:00:00.000001Z |", + "| 1000 | MT | CT | 1970-01-01T00:00:00.000002Z |", "+-----------+------+------+--------------------------------+", ]; @@ -1133,31 +1009,12 @@ mod tests { // BUG #[tokio::test] - async fn test_compact_many_batches_different_columns_different_order_with_duplicates_with_many_deletes_2( - ) { + async fn test_compact_many_batches_different_columns_different_order_with_duplicates2() { // create many-batches input data let batches = create_batches_with_influxtype_different_columns_different_order().await; - let tombstones = vec![ - create_tombstone( - 1, - 1, - 1, - 100, // delete's seq_number - 0, // min time of data to get deleted - 200000, // max time of data to get deleted - "tag2=CT and field_int=1000", // delete predicate - ), - create_tombstone( - 1, 1, 1, 101, // delete's seq_number - 0, // min time of data to get deleted - 200000, // max time of data to get deleted - "tag1=MT", // delete predicate - ), - ]; // build queryable batch from the input batches - let compact_batch = - make_queryable_batch_with_deletes("test_table", 0, 1, batches, tombstones); + let compact_batch = make_queryable_batch("test_table", 0, 1, batches); // verify PK let schema = compact_batch.schema(); @@ -1180,29 +1037,22 @@ mod tests { // verify compacted data // data is sorted and all duplicates are removed - // all rows with ("tag2=CT and field_int=1000") and ("tag1=MT") are also removed - // CORRECT RESULT - // let expected = vec![ - // "+-----------+------+------+--------------------------------+", - // "| field_int | tag1 | tag2 | time |", - // "+-----------+------+------+--------------------------------+", - // "| 5 | | AL | 1970-01-01T00:00:00.000005Z |", - // "| 10 | | AL | 1970-01-01T00:00:00.000007Z |", - // "| 70 | | CT | 1970-01-01T00:00:00.000000100Z |", - // "| 100 | | MA | 1970-01-01T00:00:00.000000050Z |", - // "| 10 | AL | MA | 1970-01-01T00:00:00.000000050Z |", - // "| 70 | CT | CT | 1970-01-01T00:00:00.000000100Z |", - // "| 70 | CT | CT | 1970-01-01T00:00:00.000000500Z |", - // "+-----------+------+------+--------------------------------+", - // ]; - // current WRONMG result: "tag1 is null" is also eliminated let expected = vec![ "+-----------+------+------+--------------------------------+", "| field_int | tag1 | tag2 | time |", "+-----------+------+------+--------------------------------+", + "| 5 | | AL | 1970-01-01T00:00:00.000005Z |", + "| 10 | | AL | 1970-01-01T00:00:00.000007Z |", + "| 70 | | CT | 1970-01-01T00:00:00.000000100Z |", + "| 1000 | | CT | 1970-01-01T00:00:00.000001Z |", + "| 100 | | MA | 1970-01-01T00:00:00.000000050Z |", "| 10 | AL | MA | 1970-01-01T00:00:00.000000050Z |", "| 70 | CT | CT | 1970-01-01T00:00:00.000000100Z |", "| 70 | CT | CT | 1970-01-01T00:00:00.000000500Z |", + "| 30 | MT | AL | 1970-01-01T00:00:00.000000005Z |", + "| 20 | MT | AL | 1970-01-01T00:00:00.000007Z |", + "| 1000 | MT | CT | 1970-01-01T00:00:00.000001Z |", + "| 1000 | MT | CT | 1970-01-01T00:00:00.000002Z |", "+-----------+------+------+--------------------------------+", ]; diff --git a/ingester/src/data.rs b/ingester/src/data.rs index 4d89b8f976..66f71159bb 100644 --- a/ingester/src/data.rs +++ b/ingester/src/data.rs @@ -25,11 +25,10 @@ use crate::{ lifecycle::LifecycleHandle, }; -pub mod namespace; +pub(crate) mod namespace; pub mod partition; -mod query_dedup; -pub mod shard; -pub mod table; +pub(crate) mod shard; +pub(crate) mod table; use self::{ partition::{resolver::PartitionProvider, PartitionStatus}, @@ -52,9 +51,6 @@ pub enum Error { #[snafu(display("Table {} not found in buffer", table_name))] TableNotFound { table_name: String }, - #[snafu(display("Table must be specified in delete"))] - TableNotPresent, - #[snafu(display("Error accessing catalog: {}", source))] Catalog { source: iox_catalog::interface::Error, @@ -187,7 +183,7 @@ impl IngesterData { .get(&shard_id) .context(ShardNotFoundSnafu { shard_id })?; shard_data - .buffer_operation(dml_operation, &self.catalog, lifecycle_handle, &self.exec) + .buffer_operation(dml_operation, &self.catalog, lifecycle_handle) .await } @@ -1354,7 +1350,6 @@ mod tests { Arc::clone(&metrics), Arc::new(SystemProvider::new()), ); - let exec = Executor::new(1); let partition_provider = Arc::new(CatalogPartitionResolver::new(Arc::clone(&catalog))); @@ -1370,7 +1365,7 @@ mod tests { // to 1 already, so it shouldn't be buffered and the buffer should // remain empty. let should_pause = data - .buffer_operation(DmlOperation::Write(w1), &catalog, &manager.handle(), &exec) + .buffer_operation(DmlOperation::Write(w1), &catalog, &manager.handle()) .await .unwrap(); { @@ -1386,7 +1381,7 @@ mod tests { assert!(!should_pause); // w2 should be in the buffer - data.buffer_operation(DmlOperation::Write(w2), &catalog, &manager.handle(), &exec) + data.buffer_operation(DmlOperation::Write(w2), &catalog, &manager.handle()) .await .unwrap(); @@ -1478,19 +1473,6 @@ mod tests { .await .unwrap(); - assert_eq!( - data.shard(shard1.id) - .unwrap() - .namespace(&namespace.name.clone().into()) - .unwrap() - .table_data(&"mem".into()) - .unwrap() - .read() - .await - .tombstone_max_sequence_number(), - None, - ); - let predicate = DeletePredicate { range: TimestampRange::new(1, 2), exprs: vec![], @@ -1509,19 +1491,6 @@ mod tests { data.buffer_operation(shard1.id, DmlOperation::Delete(d1), &manager.handle()) .await .unwrap(); - - assert_eq!( - data.shard(shard1.id) - .unwrap() - .namespace(&namespace.name.into()) - .unwrap() - .table_data(&"mem".into()) - .unwrap() - .read() - .await - .tombstone_max_sequence_number(), - Some(SequenceNumber::new(2)), - ); } /// Verifies that the progress in data is the same as expected_progress @@ -1570,7 +1539,6 @@ mod tests { PartitionId::new(2), PartitionStatus { parquet_max_sequence_number: None, - tombstone_max_sequence_number: Some(SequenceNumber::new(1)), }, )), Err(ArrowError::IoError("some io error".into())), @@ -1579,7 +1547,6 @@ mod tests { PartitionId::new(1), PartitionStatus { parquet_max_sequence_number: None, - tombstone_max_sequence_number: None, }, )), ]))); @@ -1590,7 +1557,6 @@ mod tests { partition_id: PartitionId::new(2), status: PartitionStatus { parquet_max_sequence_number: None, - tombstone_max_sequence_number: Some(SequenceNumber::new(1)), }, }), Ok(FlatIngesterQueryResponse::StartSnapshot { schema: schema_1 }), @@ -1606,7 +1572,6 @@ mod tests { partition_id: PartitionId::new(1), status: PartitionStatus { parquet_max_sequence_number: None, - tombstone_max_sequence_number: None, }, }), ]; diff --git a/ingester/src/data/namespace.rs b/ingester/src/data/namespace.rs index 94013b36c8..9aa414a535 100644 --- a/ingester/src/data/namespace.rs +++ b/ingester/src/data/namespace.rs @@ -5,10 +5,10 @@ use std::{collections::HashMap, sync::Arc}; use data_types::{NamespaceId, PartitionKey, SequenceNumber, ShardId, TableId}; use dml::DmlOperation; use iox_catalog::interface::Catalog; -use iox_query::exec::Executor; use metric::U64Counter; +use observability_deps::tracing::warn; use parking_lot::RwLock; -use snafu::{OptionExt, ResultExt}; +use snafu::ResultExt; use write_summary::ShardProgress; #[cfg(test)] @@ -70,12 +70,16 @@ impl std::ops::Deref for NamespaceName { } } +impl std::fmt::Display for NamespaceName { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + self.0.fmt(f) + } +} + /// Data of a Namespace that belongs to a given Shard #[derive(Debug)] pub(crate) struct NamespaceData { namespace_id: NamespaceId, - - #[allow(dead_code)] namespace_name: NamespaceName, /// The catalog ID of the shard this namespace is being populated from. @@ -138,7 +142,7 @@ pub(crate) struct NamespaceData { impl NamespaceData { /// Initialize new tables with default partition template of daily - pub fn new( + pub(super) fn new( namespace_id: NamespaceId, namespace_name: NamespaceName, shard_id: ShardId, @@ -173,7 +177,6 @@ impl NamespaceData { dml_operation: DmlOperation, catalog: &Arc, lifecycle_handle: &dyn LifecycleHandle, - executor: &Executor, ) -> Result { let sequence_number = dml_operation .meta() @@ -225,22 +228,17 @@ impl NamespaceData { Ok(pause_writes) } DmlOperation::Delete(delete) => { - let table_name = delete - .table_name() - .context(super::TableNotPresentSnafu)? - .into(); - let table_data = match self.table_data(&table_name) { - Some(t) => t, - None => self.insert_table(&table_name, catalog).await?, - }; + // Deprecated delete support: + // https://github.com/influxdata/influxdb_iox/issues/5825 + warn!( + shard_id=%self.shard_id, + namespace_name=%self.namespace_name, + namespace_id=%self.namespace_id, + table_name=?delete.table_name(), + sequence_number=?delete.meta().sequence(), + "discarding unsupported delete op" + ); - let mut table_data = table_data.write().await; - - table_data - .buffer_delete(delete.predicate(), sequence_number, &**catalog, executor) - .await?; - - // don't pause writes since deletes don't count towards memory limits Ok(false) } } @@ -316,6 +314,7 @@ impl NamespaceData { catalog: &Arc, ) -> Result>, super::Error> { let mut repos = catalog.repositories().await; + let info = repos .tables() .get_table_persist_info(self.shard_id, self.namespace_id, table_name) @@ -338,7 +337,6 @@ impl NamespaceData { table_name.clone(), self.shard_id, self.namespace_id, - info.tombstone_max_sequence_number, Arc::clone(&self.partition_provider), )) } @@ -455,7 +453,6 @@ mod tests { let metrics = Arc::new(metric::Registry::default()); let catalog: Arc = Arc::new(iox_catalog::mem::MemCatalog::new(Arc::clone(&metrics))); - let exec = Executor::new(1); // Populate the catalog with the shard / namespace / table let (shard_id, ns_id, table_id) = @@ -502,7 +499,6 @@ mod tests { )), &catalog, &MockLifecycleHandle::default(), - &exec, ) .await .expect("buffer op should succeed"); diff --git a/ingester/src/data/partition.rs b/ingester/src/data/partition.rs index b35a2a6d31..2adfa2582c 100644 --- a/ingester/src/data/partition.rs +++ b/ingester/src/data/partition.rs @@ -3,10 +3,7 @@ use std::sync::Arc; use arrow::record_batch::RecordBatch; -use data_types::{ - NamespaceId, PartitionId, PartitionKey, SequenceNumber, ShardId, TableId, Tombstone, -}; -use iox_query::exec::Executor; +use data_types::{NamespaceId, PartitionId, PartitionKey, SequenceNumber, ShardId, TableId}; use mutable_batch::MutableBatch; use schema::{selection::Selection, sort::SortKey}; use snafu::ResultExt; @@ -17,7 +14,7 @@ use self::{ buffer::{BufferBatch, DataBuffer}, resolver::DeferredSortKey, }; -use crate::{data::query_dedup::query, query::QueryableBatch}; +use crate::query::QueryableBatch; use super::table::TableName; @@ -42,9 +39,6 @@ pub(crate) struct UnpersistedPartitionData { pub struct PartitionStatus { /// Max sequence number persisted pub parquet_max_sequence_number: Option, - - /// Max sequence number for a tombstone - pub tombstone_max_sequence_number: Option, } /// PersistingBatch contains all needed info and data for creating @@ -266,77 +260,6 @@ impl PartitionData { Ok(()) } - /// Buffers a new tombstone: - /// . All the data in the `buffer` and `snapshots` will be replaced with one - /// tombstone-applied snapshot - /// . The tombstone is only added in the `deletes_during_persisting` if the `persisting` - /// exists - pub(super) async fn buffer_tombstone(&mut self, executor: &Executor, tombstone: Tombstone) { - self.data.add_tombstone(tombstone.clone()); - - // ---------------------------------------------------------- - // First apply the tombstone on all in-memory & non-persisting data - // Make a QueryableBatch for all buffer + snapshots + the given tombstone - let max_sequence_number = tombstone.sequence_number; - let query_batch = match self.data.snapshot_to_queryable_batch( - &self.table_name, - self.id, - Some(tombstone.clone()), - ) { - Some(query_batch) if !query_batch.is_empty() => query_batch, - _ => { - // No need to proceed further - return; - } - }; - - let (min_sequence_number, _) = query_batch.min_max_sequence_numbers(); - assert!(min_sequence_number <= max_sequence_number); - - // Run query on the QueryableBatch to apply the tombstone. - let stream = match query(executor, Arc::new(query_batch)).await { - Err(e) => { - // this should never error out. if it does, we need to crash hard so - // someone can take a look. - panic!("unable to apply tombstones on snapshots: {:?}", e); - } - Ok(stream) => stream, - }; - let record_batches = match datafusion::physical_plan::common::collect(stream).await { - Err(e) => { - // this should never error out. if it does, we need to crash hard so - // someone can take a look. - panic!("unable to collect record batches: {:?}", e); - } - Ok(batches) => batches, - }; - - // Merge all result record batches into one record batch - // and make a snapshot for it - let snapshot = if !record_batches.is_empty() { - let record_batch = - arrow::compute::concat_batches(&record_batches[0].schema(), &record_batches) - .unwrap_or_else(|e| { - panic!("unable to concat record batches: {:?}", e); - }); - let snapshot = SnapshotBatch { - min_sequence_number, - max_sequence_number, - data: Arc::new(record_batch), - }; - - Some(Arc::new(snapshot)) - } else { - None - }; - - // ---------------------------------------------------------- - // Add the tombstone-applied data back in as one snapshot - if let Some(snapshot) = snapshot { - self.data.snapshots.push(snapshot); - } - } - /// Return the progress from this Partition pub(super) fn progress(&self) -> ShardProgress { self.data.progress() @@ -402,7 +325,6 @@ mod tests { use mutable_batch_lp::test_helpers::lp_to_mutable_batch; use super::*; - use crate::test_util::create_tombstone; #[test] fn snapshot_buffer_different_but_compatible_schemas() { @@ -449,7 +371,7 @@ mod tests { // Test deletes mixed with writes on a single parittion #[tokio::test] - async fn writes_and_deletes() { + async fn writes() { // Make a partition with empty DataBuffer let s_id = 1; let t_id = 1; @@ -464,7 +386,6 @@ mod tests { SortKeyState::Provided(None), None, ); - let exec = Executor::new(1); // ------------------------------------------ // Fill `buffer` @@ -487,42 +408,8 @@ mod tests { SequenceNumber::new(2) ); assert_eq!(p.data.snapshots.len(), 0); - assert_eq!(p.data.deletes_during_persisting().len(), 0); assert_eq!(p.data.persisting, None); - // ------------------------------------------ - // Delete - // --- seq_num: 3 - let ts = create_tombstone( - 1, // tombstone id - t_id, // table id - s_id, // shard id - 3, // delete's seq_number - 0, // min time of data to get deleted - 20, // max time of data to get deleted - "day=thu", // delete predicate - ); - // one row will get deleted, the other is moved to snapshot - p.buffer_tombstone(&exec, ts).await; - - // verify data - assert!(p.data.buffer.is_none()); // always empty after delete - assert_eq!(p.data.snapshots.len(), 1); // one snpashot if there is data - assert_eq!(p.data.deletes_during_persisting().len(), 0); - assert_eq!(p.data.persisting, None); - // snapshot only has one row since the other one got deleted - let data = (*p.data.snapshots[0].data).clone(); - let expected = vec![ - "+--------+-----+------+--------------------------------+", - "| city | day | temp | time |", - "+--------+-----+------+--------------------------------+", - "| Boston | fri | 50 | 1970-01-01T00:00:00.000000010Z |", - "+--------+-----+------+--------------------------------+", - ]; - assert_batches_sorted_eq!(&expected, &[data]); - assert_eq!(p.data.snapshots[0].min_sequence_number.get(), 1); - assert_eq!(p.data.snapshots[0].max_sequence_number.get(), 3); - // ------------------------------------------ // Fill `buffer` // --- seq_num: 4 @@ -542,50 +429,15 @@ mod tests { // verify data assert_eq!( p.data.buffer.as_ref().unwrap().min_sequence_number, - SequenceNumber::new(4) + SequenceNumber::new(1) ); assert_eq!( p.data.buffer.as_ref().unwrap().max_sequence_number, SequenceNumber::new(5) ); - assert_eq!(p.data.snapshots.len(), 1); // existing sanpshot - assert_eq!(p.data.deletes_during_persisting().len(), 0); + assert_eq!(p.data.snapshots.len(), 0); assert_eq!(p.data.persisting, None); - - // ------------------------------------------ - // Delete - // --- seq_num: 6 - let ts = create_tombstone( - 2, // tombstone id - t_id, // table id - s_id, // shard id - 6, // delete's seq_number - 10, // min time of data to get deleted - 50, // max time of data to get deleted - "city=Boston", // delete predicate - ); - // two rows will get deleted, one from existing snapshot, one from the buffer being moved - // to snpashot - p.buffer_tombstone(&exec, ts).await; - - // verify data - assert!(p.data.buffer.is_none()); // always empty after delete - assert_eq!(p.data.snapshots.len(), 1); // one snpashot - assert_eq!(p.data.deletes_during_persisting().len(), 0); - assert_eq!(p.data.persisting, None); - // snapshot only has two rows since the other 2 rows with city=Boston have got deleted - let data = (*p.data.snapshots[0].data).clone(); - let expected = vec![ - "+---------+-----+------+--------------------------------+", - "| city | day | temp | time |", - "+---------+-----+------+--------------------------------+", - "| Andover | tue | 56 | 1970-01-01T00:00:00.000000030Z |", - "| Medford | sun | 55 | 1970-01-01T00:00:00.000000022Z |", - "+---------+-----+------+--------------------------------+", - ]; - assert_batches_sorted_eq!(&expected, &[data]); - assert_eq!(p.data.snapshots[0].min_sequence_number.get(), 1); - assert_eq!(p.data.snapshots[0].max_sequence_number.get(), 6); + assert!(p.data.buffer.is_some()); // ------------------------------------------ // Persisting @@ -594,32 +446,12 @@ mod tests { // verify data assert!(p.data.buffer.is_none()); // always empty after issuing persit assert_eq!(p.data.snapshots.len(), 0); // always empty after issuing persit - assert_eq!(p.data.deletes_during_persisting().len(), 0); // deletes not happen yet assert_eq!(p.data.persisting, Some(Arc::clone(&p_batch))); - // ------------------------------------------ - // Delete - // --- seq_num: 7 - let ts = create_tombstone( - 3, // tombstone id - t_id, // table id - s_id, // shard id - 7, // delete's seq_number - 10, // min time of data to get deleted - 50, // max time of data to get deleted - "temp=55", // delete predicate - ); - // if a query come while persisting, the row with temp=55 will be deleted before - // data is sent back to Querier - p.buffer_tombstone(&exec, ts).await; - // verify data - assert!(p.data.buffer.is_none()); // always empty after delete - // no snpashots becasue buffer has not data yet and the - // snapshot was empty too - assert_eq!(p.data.snapshots.len(), 0); - assert_eq!(p.data.deletes_during_persisting().len(), 1); // tombstone added since data is - // persisting + assert!(p.data.buffer.is_none()); + assert_eq!(p.data.snapshots.len(), 0); // no snpashots becasue buffer has not data yet and the + // snapshot was empty too assert_eq!(p.data.persisting, Some(Arc::clone(&p_batch))); // ------------------------------------------ @@ -640,7 +472,6 @@ mod tests { SequenceNumber::new(8) ); // 1 newly added mutable batch of 3 rows of data assert_eq!(p.data.snapshots.len(), 0); // still empty - assert_eq!(p.data.deletes_during_persisting().len(), 1); assert_eq!(p.data.persisting, Some(Arc::clone(&p_batch))); // ------------------------------------------ @@ -649,7 +480,6 @@ mod tests { // verify data assert!(p.data.buffer.is_none()); // empty after snapshot assert_eq!(p.data.snapshots.len(), 1); // data moved from buffer - assert_eq!(p.data.deletes_during_persisting().len(), 1); assert_eq!(p.data.persisting, Some(Arc::clone(&p_batch))); // snapshot has three rows moved from buffer let data = (*p.data.snapshots[0].data).clone(); @@ -665,41 +495,5 @@ mod tests { assert_batches_sorted_eq!(&expected, &[data]); assert_eq!(p.data.snapshots[0].min_sequence_number.get(), 8); assert_eq!(p.data.snapshots[0].max_sequence_number.get(), 8); - - // ------------------------------------------ - // Delete - // --- seq_num: 9 - let ts = create_tombstone( - 4, // tombstone id - t_id, // table id - s_id, // shard id - 9, // delete's seq_number - 10, // min time of data to get deleted - 50, // max time of data to get deleted - "temp=60", // delete predicate - ); - // the row with temp=60 will be removed from the sanphot - p.buffer_tombstone(&exec, ts).await; - - // verify data - assert!(p.data.buffer.is_none()); // always empty after delete - assert_eq!(p.data.snapshots.len(), 1); // new snapshot of the existing with delete applied - assert_eq!(p.data.deletes_during_persisting().len(), 2); // one more tombstone added make it 2 - assert_eq!(p.data.persisting, Some(Arc::clone(&p_batch))); - // snapshot has only 2 rows because the row with tem=60 was removed - let data = (*p.data.snapshots[0].data).clone(); - let expected = vec![ - "+------------+-----+------+--------------------------------+", - "| city | day | temp | time |", - "+------------+-----+------+--------------------------------+", - "| Wilmington | sun | 55 | 1970-01-01T00:00:00.000000035Z |", - "| Boston | sun | 62 | 1970-01-01T00:00:00.000000038Z |", - "+------------+-----+------+--------------------------------+", - ]; - assert_batches_sorted_eq!(&expected, &[data]); - assert_eq!(p.data.snapshots[0].min_sequence_number.get(), 8); - assert_eq!(p.data.snapshots[0].max_sequence_number.get(), 9); - - exec.join().await; } } diff --git a/ingester/src/data/partition/buffer.rs b/ingester/src/data/partition/buffer.rs index 3195b9c74d..866e7a966c 100644 --- a/ingester/src/data/partition/buffer.rs +++ b/ingester/src/data/partition/buffer.rs @@ -2,7 +2,7 @@ use std::sync::Arc; -use data_types::{PartitionId, SequenceNumber, ShardId, TableId, Tombstone}; +use data_types::{PartitionId, SequenceNumber, ShardId, TableId}; use mutable_batch::MutableBatch; use schema::selection::Selection; use snafu::ResultExt; @@ -40,14 +40,6 @@ pub(crate) struct DataBuffer { /// Buffer of incoming writes pub(crate) buffer: Option, - /// Buffer of tombstones whose time range may overlap with this partition. - /// All tombstones were already applied to corresponding snapshots. This list - /// only keep the ones that come during persisting. The reason - /// we keep them becasue if a query comes, we need to apply these tombstones - /// on the persiting data before sending it to the Querier - /// When the `persiting` is done and removed, this list will get empty, too - deletes_during_persisting: Vec, - /// Data in `buffer` will be moved to a `snapshot` when one of these happens: /// . A background persist is called /// . A read request from Querier @@ -72,14 +64,6 @@ pub(crate) struct DataBuffer { } impl DataBuffer { - /// Add a new tombstones into the [`DataBuffer`]. - pub(super) fn add_tombstone(&mut self, tombstone: Tombstone) { - // Only keep this tombstone if some data is being persisted - if self.persisting.is_some() { - self.deletes_during_persisting.push(tombstone); - } - } - /// If a [`BufferBatch`] exists, convert it to a [`SnapshotBatch`] and add /// it to the list of snapshots. /// @@ -113,7 +97,6 @@ impl DataBuffer { &mut self, table_name: &TableName, partition_id: PartitionId, - tombstone: Option, ) -> Option { self.generate_snapshot() .expect("This mutable batch snapshot error should be impossible."); @@ -121,21 +104,11 @@ impl DataBuffer { let mut data = vec![]; std::mem::swap(&mut data, &mut self.snapshots); - let mut tombstones = vec![]; - if let Some(tombstone) = tombstone { - tombstones.push(tombstone); - } - // only produce batch if there is any data if data.is_empty() { None } else { - Some(QueryableBatch::new( - table_name.clone(), - partition_id, - data, - tombstones, - )) + Some(QueryableBatch::new(table_name.clone(), partition_id, data)) } } @@ -172,9 +145,7 @@ impl DataBuffer { panic!("Unable to snapshot while persisting. This is an unexpected state.") } - if let Some(queryable_batch) = - self.snapshot_to_queryable_batch(table_name, partition_id, None) - { + if let Some(queryable_batch) = self.snapshot_to_queryable_batch(table_name, partition_id) { let persisting_batch = Arc::new(PersistingBatch { shard_id, table_id, @@ -199,12 +170,7 @@ impl DataBuffer { }; // persisting data - let mut queryable_batch = (*persisting.data).clone(); - - // Add new tombstones if any - queryable_batch.add_tombstones(&self.deletes_during_persisting); - - Some(queryable_batch) + Some((*persisting.data).clone()) } /// Return the progress in this DataBuffer @@ -241,12 +207,6 @@ impl DataBuffer { pub(crate) fn mark_persisted(&mut self) { self.persisting = None; - self.deletes_during_persisting.clear() - } - - #[cfg(test)] - pub(super) fn deletes_during_persisting(&self) -> &[Tombstone] { - self.deletes_during_persisting.as_ref() } } diff --git a/ingester/src/data/query_dedup.rs b/ingester/src/data/query_dedup.rs deleted file mode 100644 index 199e3ae14e..0000000000 --- a/ingester/src/data/query_dedup.rs +++ /dev/null @@ -1,159 +0,0 @@ -use std::sync::Arc; - -use datafusion::{error::DataFusionError, physical_plan::SendableRecordBatchStream}; -use iox_query::{ - exec::{Executor, ExecutorType}, - QueryChunk, QueryChunkMeta, ScanPlanBuilder, -}; -use observability_deps::tracing::debug; -use snafu::{ResultExt, Snafu}; - -use crate::query::QueryableBatch; - -#[derive(Debug, Snafu)] -#[allow(missing_copy_implementations, missing_docs)] -pub enum Error { - #[snafu(display("Error creating plan for querying Ingester data to send to Querier"))] - Frontend { - source: iox_query::frontend::common::Error, - }, - - #[snafu(display("Error building logical plan for querying Ingester data to send to Querier"))] - LogicalPlan { source: DataFusionError }, - - #[snafu(display( - "Error building physical plan for querying Ingester data to send to Querier: {}", - source - ))] - PhysicalPlan { source: DataFusionError }, - - #[snafu(display( - "Error executing the query for getting Ingester data to send to Querier: {}", - source - ))] - ExecutePlan { source: DataFusionError }, -} - -/// A specialized `Error` for Ingester's Query errors -pub type Result = std::result::Result; - -/// Query a given Queryable Batch, applying selection and filters as appropriate -/// Return stream of record batches -pub(crate) async fn query( - executor: &Executor, - data: Arc, -) -> Result { - // Build logical plan for filtering data - // Note that this query will also apply the delete predicates that go with the QueryableBatch - - // TODO: Since we have different type of servers (router, - // ingester, compactor, and querier), we may want to add more - // types into the ExecutorType to have better log and resource - // managment - let ctx = executor.new_context(ExecutorType::Query); - - // Creates an execution plan for a scan and filter data of a single chunk - let schema = data.schema(); - let table_name = data.table_name().to_string(); - - debug!(%table_name, "Creating single chunk scan plan"); - - let logical_plan = ScanPlanBuilder::new(schema, ctx.child_ctx("scan_and_filter planning")) - .with_chunks([data as _]) - .build() - .context(FrontendSnafu)? - .plan_builder - .build() - .context(LogicalPlanSnafu)?; - - debug!(%table_name, plan=%logical_plan.display_indent_schema(), - "created single chunk scan plan"); - - // Build physical plan - let physical_plan = ctx - .create_physical_plan(&logical_plan) - .await - .context(PhysicalPlanSnafu {})?; - - // Execute the plan and return the filtered stream - let output_stream = ctx - .execute_stream(physical_plan) - .await - .context(ExecutePlanSnafu {})?; - - Ok(output_stream) -} - -#[cfg(test)] -mod tests { - use arrow_util::assert_batches_eq; - - use super::*; - use crate::test_util::{ - create_one_record_batch_with_influxtype_no_duplicates, create_tombstone, - make_queryable_batch, make_queryable_batch_with_deletes, - }; - - #[tokio::test] - async fn test_query() { - test_helpers::maybe_start_logging(); - - // create input data - let batches = create_one_record_batch_with_influxtype_no_duplicates().await; - - // build queryable batch from the input batches - let batch = make_queryable_batch("test_table", 0, 1, batches); - - // query without filters - let exc = Executor::new(1); - let stream = query(&exc, batch).await.unwrap(); - let output_batches = datafusion::physical_plan::common::collect(stream) - .await - .unwrap(); - - // verify data: all rows and columns should be returned - let expected = vec![ - "+-----------+------+-----------------------------+", - "| field_int | tag1 | time |", - "+-----------+------+-----------------------------+", - "| 70 | UT | 1970-01-01T00:00:00.000020Z |", - "| 10 | VT | 1970-01-01T00:00:00.000010Z |", - "| 1000 | WA | 1970-01-01T00:00:00.000008Z |", - "+-----------+------+-----------------------------+", - ]; - assert_batches_eq!(&expected, &output_batches); - - exc.join().await; - } - - #[tokio::test] - async fn test_query_with_delete() { - test_helpers::maybe_start_logging(); - - // create input data - let batches = create_one_record_batch_with_influxtype_no_duplicates().await; - let tombstones = vec![create_tombstone(1, 1, 1, 1, 0, 200000, "tag1=UT")]; - - // build queryable batch from the input batches - let batch = make_queryable_batch_with_deletes("test_table", 0, 1, batches, tombstones); - - let exc = Executor::new(1); - let stream = query(&exc, batch).await.unwrap(); - let output_batches = datafusion::physical_plan::common::collect(stream) - .await - .unwrap(); - - // verify data: - let expected = vec![ - "+-----------+------+-----------------------------+", - "| field_int | tag1 | time |", - "+-----------+------+-----------------------------+", - "| 10 | VT | 1970-01-01T00:00:00.000010Z |", - "| 1000 | WA | 1970-01-01T00:00:00.000008Z |", - "+-----------+------+-----------------------------+", - ]; - assert_batches_eq!(&expected, &output_batches); - - exc.join().await; - } -} diff --git a/ingester/src/data/shard.rs b/ingester/src/data/shard.rs index 041001126b..b01504085f 100644 --- a/ingester/src/data/shard.rs +++ b/ingester/src/data/shard.rs @@ -5,7 +5,6 @@ use std::{collections::HashMap, sync::Arc}; use data_types::{NamespaceId, ShardId, ShardIndex}; use dml::DmlOperation; use iox_catalog::interface::Catalog; -use iox_query::exec::Executor; use metric::U64Counter; use parking_lot::RwLock; use snafu::{OptionExt, ResultExt}; @@ -100,7 +99,6 @@ impl ShardData { dml_operation: DmlOperation, catalog: &Arc, lifecycle_handle: &dyn LifecycleHandle, - executor: &Executor, ) -> Result { let namespace_data = match self.namespace(&NamespaceName::from(dml_operation.namespace())) { Some(d) => d, @@ -111,7 +109,7 @@ impl ShardData { }; namespace_data - .buffer_operation(dml_operation, catalog, lifecycle_handle, executor) + .buffer_operation(dml_operation, catalog, lifecycle_handle) .await } @@ -218,7 +216,6 @@ mod tests { let metrics = Arc::new(metric::Registry::default()); let catalog: Arc = Arc::new(iox_catalog::mem::MemCatalog::new(Arc::clone(&metrics))); - let exec = Executor::new(1); // Populate the catalog with the shard / namespace / table let (shard_id, ns_id, table_id) = @@ -262,7 +259,6 @@ mod tests { )), &catalog, &MockLifecycleHandle::default(), - &exec, ) .await .expect("buffer op should succeed"); diff --git a/ingester/src/data/table.rs b/ingester/src/data/table.rs index 008f74d149..357c3edd6c 100644 --- a/ingester/src/data/table.rs +++ b/ingester/src/data/table.rs @@ -2,14 +2,8 @@ use std::{collections::HashMap, sync::Arc}; -use data_types::{ - DeletePredicate, NamespaceId, PartitionId, PartitionKey, SequenceNumber, ShardId, TableId, - Timestamp, -}; -use iox_catalog::interface::Catalog; -use iox_query::exec::Executor; +use data_types::{NamespaceId, PartitionId, PartitionKey, SequenceNumber, ShardId, TableId}; use mutable_batch::MutableBatch; -use snafu::ResultExt; use write_summary::ShardProgress; use super::partition::{ @@ -90,9 +84,6 @@ pub(crate) struct TableData { shard_id: ShardId, namespace_id: NamespaceId, - // the max sequence number for a tombstone associated with this table - tombstone_max_sequence_number: Option, - /// An abstract constructor of [`PartitionData`] instances for a given /// `(key, shard, table)` triplet. partition_provider: Arc, @@ -117,7 +108,6 @@ impl TableData { table_name: TableName, shard_id: ShardId, namespace_id: NamespaceId, - tombstone_max_sequence_number: Option, partition_provider: Arc, ) -> Self { Self { @@ -125,7 +115,6 @@ impl TableData { table_name, shard_id, namespace_id, - tombstone_max_sequence_number, partition_data: Default::default(), partition_provider, } @@ -141,12 +130,6 @@ impl TableData { .flatten() } - /// Return tombstone_max_sequence_number - #[allow(dead_code)] // Used in tests - pub(super) fn tombstone_max_sequence_number(&self) -> Option { - self.tombstone_max_sequence_number - } - // buffers the table write and returns true if the lifecycle manager indicates that // ingest should be paused. pub(super) async fn buffer_table_write( @@ -204,41 +187,6 @@ impl TableData { Ok(should_pause) } - pub(super) async fn buffer_delete( - &mut self, - predicate: &DeletePredicate, - sequence_number: SequenceNumber, - catalog: &dyn Catalog, - executor: &Executor, - ) -> Result<(), super::Error> { - let min_time = Timestamp::new(predicate.range.start()); - let max_time = Timestamp::new(predicate.range.end()); - - let mut repos = catalog.repositories().await; - let tombstone = repos - .tombstones() - .create_or_get( - self.table_id, - self.shard_id, - sequence_number, - min_time, - max_time, - &predicate.expr_sql_string(), - ) - .await - .context(super::CatalogSnafu)?; - - // remember "persisted" state - self.tombstone_max_sequence_number = Some(sequence_number); - - // modify one partition at a time - for data in self.partition_data.by_key.values_mut() { - data.buffer_tombstone(executor, tombstone.clone()).await; - } - - Ok(()) - } - /// Return the [`PartitionData`] for the specified ID. #[allow(unused)] pub(crate) fn get_partition( @@ -277,7 +225,6 @@ impl TableData { persisting: p.get_persisting_data(), partition_status: PartitionStatus { parquet_max_sequence_number: p.max_persisted_sequence_number(), - tombstone_max_sequence_number: self.tombstone_max_sequence_number, }, }) .collect() @@ -316,6 +263,7 @@ mod tests { use assert_matches::assert_matches; use data_types::{PartitionId, ShardIndex}; + use iox_catalog::interface::Catalog; use mutable_batch::writer; use mutable_batch_lp::lines_to_batches; use schema::{InfluxColumnType, InfluxFieldType}; @@ -367,7 +315,6 @@ mod tests { TABLE_NAME.into(), shard_id, ns_id, - None, partition_provider, ); @@ -427,7 +374,6 @@ mod tests { TABLE_NAME.into(), shard_id, ns_id, - None, partition_provider, ); diff --git a/ingester/src/querier_handler.rs b/ingester/src/querier_handler.rs index 7ff7494af0..88371e2c40 100644 --- a/ingester/src/querier_handler.rs +++ b/ingester/src/querier_handler.rs @@ -155,7 +155,6 @@ fn prepare_data_to_querier_for_partition( request.table.clone().into(), unpersisted_partition_data.partition_id, vec![], - vec![], ) }) .with_data(unpersisted_partition_data.non_persisted); @@ -201,10 +200,7 @@ mod tests { use super::*; use crate::{ data::FlatIngesterQueryResponse, - test_util::{ - make_ingester_data, make_ingester_data_with_tombstones, DataLocation, TEST_NAMESPACE, - TEST_TABLE, - }, + test_util::{make_ingester_data, DataLocation, TEST_NAMESPACE, TEST_TABLE}, }; #[tokio::test] @@ -362,182 +358,6 @@ mod tests { } } - #[tokio::test] - async fn test_prepare_data_to_querier_with_tombstones() { - test_helpers::maybe_start_logging(); - - // make 7 scenarios for ingester data with tombstones - let mut scenarios = vec![]; - for loc in &[ - DataLocation::BUFFER, - DataLocation::BUFFER_SNAPSHOT, - DataLocation::BUFFER_PERSISTING, - DataLocation::BUFFER_SNAPSHOT_PERSISTING, - DataLocation::SNAPSHOT, - DataLocation::SNAPSHOT_PERSISTING, - DataLocation::PERSISTING, - ] { - let scenario = Arc::new(make_ingester_data_with_tombstones(*loc).await); - scenarios.push((loc, scenario)); - } - - // read data from all scenarios without any filters - let request = Arc::new(IngesterQueryRequest::new( - TEST_NAMESPACE.to_string(), - TEST_TABLE.to_string(), - vec![], - None, - )); - let expected_not_persisting = vec![ - "+------------+-----+------+--------------------------------+", - "| city | day | temp | time |", - "+------------+-----+------+--------------------------------+", - "| Andover | mon | | 1970-01-01T00:00:00.000000046Z |", - "| Andover | tue | 56 | 1970-01-01T00:00:00.000000030Z |", - "| Medford | sun | 55 | 1970-01-01T00:00:00.000000022Z |", - "| Medford | wed | | 1970-01-01T00:00:00.000000026Z |", - "| Reading | mon | 58 | 1970-01-01T00:00:00.000000040Z |", - "| Wilmington | mon | | 1970-01-01T00:00:00.000000035Z |", - "+------------+-----+------+--------------------------------+", - ]; - // For "persisting" data locations the tombstones were NOT applied because they arrived AFTER the data - // transitioned into the "persisting" state. In this case, the ingester will apply the tombstones. - let expected_persisting = vec![ - "+------------+-----+------+--------------------------------+", - "| city | day | temp | time |", - "+------------+-----+------+--------------------------------+", - "| Andover | mon | | 1970-01-01T00:00:00.000000046Z |", - "| Andover | tue | 56 | 1970-01-01T00:00:00.000000030Z |", - "| Boston | mon | | 1970-01-01T00:00:00.000000038Z |", - "| Boston | sun | 60 | 1970-01-01T00:00:00.000000036Z |", - "| Medford | sun | 55 | 1970-01-01T00:00:00.000000022Z |", - "| Medford | wed | | 1970-01-01T00:00:00.000000026Z |", - "| Reading | mon | 58 | 1970-01-01T00:00:00.000000040Z |", - "| Wilmington | mon | | 1970-01-01T00:00:00.000000035Z |", - "+------------+-----+------+--------------------------------+", - ]; - for (loc, scenario) in &scenarios { - println!("Location: {loc:?}"); - let expected = if loc.intersects(DataLocation::PERSISTING) { - &expected_persisting - } else { - &expected_not_persisting - }; - - let stream = prepare_data_to_querier(scenario, &request).await.unwrap(); - let result = ingester_response_to_record_batches(stream).await; - assert_batches_sorted_eq!(expected, &result); - } - - // read data from all scenarios and filter out column day - let request = Arc::new(IngesterQueryRequest::new( - TEST_NAMESPACE.to_string(), - TEST_TABLE.to_string(), - vec!["city".to_string(), "temp".to_string(), "time".to_string()], - None, - )); - let expected_not_persisting = vec![ - "+------------+------+--------------------------------+", - "| city | temp | time |", - "+------------+------+--------------------------------+", - "| Andover | | 1970-01-01T00:00:00.000000046Z |", - "| Andover | 56 | 1970-01-01T00:00:00.000000030Z |", - "| Medford | | 1970-01-01T00:00:00.000000026Z |", - "| Medford | 55 | 1970-01-01T00:00:00.000000022Z |", - "| Reading | 58 | 1970-01-01T00:00:00.000000040Z |", - "| Wilmington | | 1970-01-01T00:00:00.000000035Z |", - "+------------+------+--------------------------------+", - ]; - // For "persisting" data locations the tombstones were NOT applied because they arrived AFTER the data - // transitioned into the "persisting" state. In this case, the ingester will apply the tombstones. - let expected_persisting = vec![ - "+------------+------+--------------------------------+", - "| city | temp | time |", - "+------------+------+--------------------------------+", - "| Andover | | 1970-01-01T00:00:00.000000046Z |", - "| Andover | 56 | 1970-01-01T00:00:00.000000030Z |", - "| Boston | | 1970-01-01T00:00:00.000000038Z |", - "| Boston | 60 | 1970-01-01T00:00:00.000000036Z |", - "| Medford | | 1970-01-01T00:00:00.000000026Z |", - "| Medford | 55 | 1970-01-01T00:00:00.000000022Z |", - "| Reading | 58 | 1970-01-01T00:00:00.000000040Z |", - "| Wilmington | | 1970-01-01T00:00:00.000000035Z |", - "+------------+------+--------------------------------+", - ]; - for (loc, scenario) in &scenarios { - println!("Location: {loc:?}"); - let expected = if loc.intersects(DataLocation::PERSISTING) { - &expected_persisting - } else { - &expected_not_persisting - }; - - let stream = prepare_data_to_querier(scenario, &request).await.unwrap(); - let result = ingester_response_to_record_batches(stream).await; - assert_batches_sorted_eq!(expected, &result); - } - - // read data from all scenarios, filter out column day, city Medford, time outside range [0, 42) - let expr = col("city").not_eq(lit("Medford")); - let pred = Predicate::default().with_expr(expr).with_range(0, 42); - let request = Arc::new(IngesterQueryRequest::new( - TEST_NAMESPACE.to_string(), - TEST_TABLE.to_string(), - vec!["city".to_string(), "temp".to_string(), "time".to_string()], - Some(pred), - )); - // predicates and de-dup are NOT applied!, otherwise this would look like this: - // let expected = vec![ - // "+------------+------+--------------------------------+", - // "| city | temp | time |", - // "+------------+------+--------------------------------+", - // "| Andover | 56 | 1970-01-01T00:00:00.000000030Z |", - // "| Reading | 58 | 1970-01-01T00:00:00.000000040Z |", - // "| Wilmington | | 1970-01-01T00:00:00.000000035Z |", - // "+------------+------+--------------------------------+", - // ]; - let expected_not_persisting = vec![ - "+------------+------+--------------------------------+", - "| city | temp | time |", - "+------------+------+--------------------------------+", - "| Andover | | 1970-01-01T00:00:00.000000046Z |", - "| Andover | 56 | 1970-01-01T00:00:00.000000030Z |", - "| Medford | | 1970-01-01T00:00:00.000000026Z |", - "| Medford | 55 | 1970-01-01T00:00:00.000000022Z |", - "| Reading | 58 | 1970-01-01T00:00:00.000000040Z |", - "| Wilmington | | 1970-01-01T00:00:00.000000035Z |", - "+------------+------+--------------------------------+", - ]; - // For "persisting" data locations the tombstones were NOT applied because they arrived AFTER the data - // transitioned into the "persisting" state. In this case, the ingester will apply the tombstones. - let expected_persisting = vec![ - "+------------+------+--------------------------------+", - "| city | temp | time |", - "+------------+------+--------------------------------+", - "| Andover | | 1970-01-01T00:00:00.000000046Z |", - "| Andover | 56 | 1970-01-01T00:00:00.000000030Z |", - "| Boston | | 1970-01-01T00:00:00.000000038Z |", - "| Boston | 60 | 1970-01-01T00:00:00.000000036Z |", - "| Medford | | 1970-01-01T00:00:00.000000026Z |", - "| Medford | 55 | 1970-01-01T00:00:00.000000022Z |", - "| Reading | 58 | 1970-01-01T00:00:00.000000040Z |", - "| Wilmington | | 1970-01-01T00:00:00.000000035Z |", - "+------------+------+--------------------------------+", - ]; - for (loc, scenario) in &scenarios { - println!("Location: {loc:?}"); - let expected = if loc.intersects(DataLocation::PERSISTING) { - &expected_persisting - } else { - &expected_not_persisting - }; - - let stream = prepare_data_to_querier(scenario, &request).await.unwrap(); - let result = ingester_response_to_record_batches(stream).await; - assert_batches_sorted_eq!(expected, &result); - } - } - /// Convert [`IngesterQueryResponse`] to a set of [`RecordBatch`]es. /// /// If the response contains multiple snapshots, this will merge the schemas into a single one and create diff --git a/ingester/src/query.rs b/ingester/src/query.rs index 219dcbcf6e..dc38001e4f 100644 --- a/ingester/src/query.rs +++ b/ingester/src/query.rs @@ -6,7 +6,7 @@ use arrow::record_batch::RecordBatch; use arrow_util::util::ensure_schema; use data_types::{ ChunkId, ChunkOrder, DeletePredicate, PartitionId, SequenceNumber, TableSummary, - TimestampMinMax, Tombstone, + TimestampMinMax, }; use datafusion::{ error::DataFusionError, @@ -21,10 +21,7 @@ use iox_query::{ QueryChunk, QueryChunkMeta, }; use observability_deps::tracing::trace; -use predicate::{ - delete_predicate::{tombstones_to_delete_predicates, tombstones_to_delete_predicates_iter}, - Predicate, -}; +use predicate::Predicate; use schema::{merge::merge_record_batch_schemas, selection::Selection, sort::SortKey, Schema}; use snafu::{ResultExt, Snafu}; @@ -56,9 +53,6 @@ pub(crate) struct QueryableBatch { /// data pub(crate) data: Vec>, - /// Delete predicates of the tombstones - pub(crate) delete_predicates: Vec>, - /// This is needed to return a reference for a trait function pub(crate) table_name: TableName, @@ -72,12 +66,9 @@ impl QueryableBatch { table_name: TableName, partition_id: PartitionId, data: Vec>, - deletes: Vec, ) -> Self { - let delete_predicates = tombstones_to_delete_predicates(&deletes); Self { data, - delete_predicates, table_name, partition_id, } @@ -89,12 +80,6 @@ impl QueryableBatch { self } - /// Add more tombstones - pub(crate) fn add_tombstones(&mut self, deletes: &[Tombstone]) { - let delete_predicates = tombstones_to_delete_predicates_iter(deletes); - self.delete_predicates.extend(delete_predicates); - } - /// return min and max of all the snapshots pub(crate) fn min_max_sequence_numbers(&self) -> (SequenceNumber, SequenceNumber) { let min = self @@ -113,11 +98,6 @@ impl QueryableBatch { (min, max) } - - /// return true if it has no data - pub(crate) fn is_empty(&self) -> bool { - self.data.is_empty() - } } impl QueryChunkMeta for QueryableBatch { @@ -147,16 +127,16 @@ impl QueryChunkMeta for QueryableBatch { None // Ingester data is not sorted } - fn delete_predicates(&self) -> &[Arc] { - self.delete_predicates.as_ref() - } - fn timestamp_min_max(&self) -> Option { // Note: we need to consider which option we want to go with // . Return None here and avoid taking time to compute time's min max of RecordBacthes (current choice) // . Compute time's min max here and avoid compacting non-overlapped QueryableBatches in the Ingester None } + + fn delete_predicates(&self) -> &[Arc] { + &[] + } } impl QueryChunk for QueryableBatch { @@ -265,165 +245,3 @@ impl QueryChunk for QueryableBatch { self } } - -#[cfg(test)] -mod tests { - use arrow::{ - array::{ - ArrayRef, BooleanArray, DictionaryArray, Float64Array, Int64Array, StringArray, - TimestampNanosecondArray, UInt64Array, - }, - datatypes::{DataType, Int32Type, TimeUnit}, - }; - use data_types::{DeleteExpr, Op, Scalar, TimestampRange}; - - use super::*; - use crate::test_util::create_tombstone; - - #[tokio::test] - async fn test_merge_batch_schema() { - // Merge schema of the batches - // The fields in the schema are sorted by column name - let batches = create_batches(); - let merged_schema = (*merge_record_batch_schemas(&batches)).clone(); - - // Expected Arrow schema - let arrow_schema = Arc::new(arrow::datatypes::Schema::new(vec![ - arrow::datatypes::Field::new( - "dict", - DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)), - true, - ), - arrow::datatypes::Field::new("int64", DataType::Int64, true), - arrow::datatypes::Field::new("string", DataType::Utf8, true), - arrow::datatypes::Field::new("bool", DataType::Boolean, true), - arrow::datatypes::Field::new( - "time", - DataType::Timestamp(TimeUnit::Nanosecond, None), - false, - ), - arrow::datatypes::Field::new("uint64", DataType::UInt64, false), - arrow::datatypes::Field::new("float64", DataType::Float64, true), - ])); - let expected_schema = Schema::try_from(arrow_schema) - .unwrap() - .sort_fields_by_name(); - - assert_eq!( - expected_schema, merged_schema, - "\nExpected:\n{:#?}\nActual:\n{:#?}", - expected_schema, merged_schema - ); - } - - #[tokio::test] - async fn test_tombstones_to_delete_predicates() { - // create tombstones - let tombstones = vec![ - create_tombstone(1, 1, 1, 1, 100, 200, "temp=10"), - create_tombstone(1, 1, 1, 2, 100, 350, "temp!=10 and city=Boston"), - ]; - - // This new queryable batch will convert tombstone to delete predicates - let query_batch = - QueryableBatch::new("test_table".into(), PartitionId::new(0), vec![], tombstones); - let predicates = query_batch.delete_predicates(); - let expected = vec![ - Arc::new(DeletePredicate { - range: TimestampRange::new(100, 200), - exprs: vec![DeleteExpr { - column: String::from("temp"), - op: Op::Eq, - scalar: Scalar::I64(10), - }], - }), - Arc::new(DeletePredicate { - range: TimestampRange::new(100, 350), - exprs: vec![ - DeleteExpr { - column: String::from("temp"), - op: Op::Ne, - scalar: Scalar::I64(10), - }, - DeleteExpr { - column: String::from("city"), - op: Op::Eq, - scalar: Scalar::String(String::from(r#"Boston"#)), - }, - ], - }), - ]; - - assert_eq!(expected, predicates); - } - - // ---------------------------------------------------------------------------------------------- - // Data for testing - - // Create pure RecordBatches without knowledge of Influx datatype - fn create_batches() -> Vec> { - // Batch 1: & 3 rows - let dict_array: ArrayRef = Arc::new( - vec![Some("a"), None, Some("b")] - .into_iter() - .collect::>(), - ); - let int64_array: ArrayRef = - Arc::new([Some(-1), None, Some(2)].iter().collect::()); - let string_array: ArrayRef = Arc::new( - vec![Some("foo"), Some("and"), Some("bar")] - .into_iter() - .collect::(), - ); - let bool_array: ArrayRef = Arc::new( - [Some(true), None, Some(false)] - .iter() - .collect::(), - ); - let ts_array: ArrayRef = Arc::new( - [Some(150), Some(200), Some(1526823730000000000)] - .iter() - .collect::(), - ); - let batch1 = RecordBatch::try_from_iter_with_nullable(vec![ - ("dict", dict_array, true), - ("int64", int64_array, true), - ("string", string_array, true), - ("bool", bool_array, true), - ("time", ts_array, false), // not null - ]) - .unwrap(); - - // Batch 2: & 2 rows - let dict_array: ArrayRef = Arc::new( - vec![None, Some("d")] - .into_iter() - .collect::>(), - ); - let uint64_array: ArrayRef = Arc::new([Some(1), Some(2)].iter().collect::()); // not null - let float64_array: ArrayRef = - Arc::new([Some(1.0), Some(2.0)].iter().collect::()); - let string_array: ArrayRef = Arc::new( - vec![Some("foo"), Some("bar")] - .into_iter() - .collect::(), - ); - let bool_array: ArrayRef = Arc::new([Some(true), None].iter().collect::()); - let ts_array: ArrayRef = Arc::new( - [Some(100), Some(1626823730000000000)] // not null - .iter() - .collect::(), - ); - let batch2 = RecordBatch::try_from_iter_with_nullable(vec![ - ("dict", dict_array, true), - ("uint64", uint64_array, false), // not null - ("float64", float64_array, true), - ("string", string_array, true), - ("bool", bool_array, true), - ("time", ts_array, false), // not null - ]) - .unwrap(); - - vec![Arc::new(batch1), Arc::new(batch2)] - } -} diff --git a/ingester/src/server/grpc.rs b/ingester/src/server/grpc.rs index 4f06a93a46..8cbd26afe1 100644 --- a/ingester/src/server/grpc.rs +++ b/ingester/src/server/grpc.rs @@ -410,9 +410,6 @@ impl Stream for GetStream { parquet_max_sequence_number: status .parquet_max_sequence_number .map(|x| x.get()), - tombstone_max_sequence_number: status - .tombstone_max_sequence_number - .map(|x| x.get()), }), }; prost::Message::encode(&app_metadata, &mut bytes) @@ -489,7 +486,6 @@ mod tests { partition_id: PartitionId::new(1), status: PartitionStatus { parquet_max_sequence_number: None, - tombstone_max_sequence_number: None, }, }), Ok(FlatIngesterQueryResponse::StartSnapshot { schema }), @@ -502,7 +498,6 @@ mod tests { partition_id: 1, status: Some(proto::PartitionStatus { parquet_max_sequence_number: None, - tombstone_max_sequence_number: None, }), }, }), @@ -527,7 +522,6 @@ mod tests { partition_id: PartitionId::new(1), status: PartitionStatus { parquet_max_sequence_number: None, - tombstone_max_sequence_number: None, }, }), Err(ArrowError::IoError("foo".into())), @@ -535,7 +529,6 @@ mod tests { partition_id: PartitionId::new(1), status: PartitionStatus { parquet_max_sequence_number: None, - tombstone_max_sequence_number: None, }, }), ], @@ -546,7 +539,6 @@ mod tests { partition_id: 1, status: Some(proto::PartitionStatus { parquet_max_sequence_number: None, - tombstone_max_sequence_number: None, }), }, }), diff --git a/ingester/src/stream_handler/handler.rs b/ingester/src/stream_handler/handler.rs index 1b163ea325..9a52b10505 100644 --- a/ingester/src/stream_handler/handler.rs +++ b/ingester/src/stream_handler/handler.rs @@ -945,7 +945,7 @@ mod tests { Ok(DmlOperation::Write(make_write("good_op", 2))) ]], sink_rets = [ - Err(crate::data::Error::TableNotPresent), + Err(crate::data::Error::NamespaceNotFound{namespace: "bananas".to_string() }), Ok(true), ], want_ttbr = 2, diff --git a/ingester/src/stream_handler/mod.rs b/ingester/src/stream_handler/mod.rs index 296f158e1a..5e9a351fe4 100644 --- a/ingester/src/stream_handler/mod.rs +++ b/ingester/src/stream_handler/mod.rs @@ -17,7 +17,7 @@ //! [`LifecycleManager`]: crate::lifecycle::LifecycleManager //! [`LifecycleHandle::can_resume_ingest()`]: crate::lifecycle::LifecycleHandle::can_resume_ingest() -pub mod handler; +pub(crate) mod handler; mod periodic_watermark_fetcher; mod sink; @@ -25,8 +25,8 @@ mod sink; pub mod mock_sink; #[cfg(test)] pub mod mock_watermark_fetcher; -pub mod sink_adaptor; -pub mod sink_instrumentation; +pub(crate) mod sink_adaptor; +pub(crate) mod sink_instrumentation; -pub use periodic_watermark_fetcher::*; -pub use sink::*; +pub(crate) use periodic_watermark_fetcher::*; +pub(crate) use sink::*; diff --git a/ingester/src/stream_handler/periodic_watermark_fetcher.rs b/ingester/src/stream_handler/periodic_watermark_fetcher.rs index 43c8cf52c9..37f99663cc 100644 --- a/ingester/src/stream_handler/periodic_watermark_fetcher.rs +++ b/ingester/src/stream_handler/periodic_watermark_fetcher.rs @@ -24,7 +24,7 @@ use super::sink_instrumentation::WatermarkFetcher; /// Emits an error metric named `write_buffer_watermark_fetch_errors` that /// increments once per fetch error. #[derive(Debug)] -pub struct PeriodicWatermarkFetcher { +pub(crate) struct PeriodicWatermarkFetcher { last_watermark: Arc, poll_handle: JoinHandle<()>, } diff --git a/ingester/src/stream_handler/sink.rs b/ingester/src/stream_handler/sink.rs index 5f8220a942..825b012ce9 100644 --- a/ingester/src/stream_handler/sink.rs +++ b/ingester/src/stream_handler/sink.rs @@ -5,7 +5,7 @@ use dml::DmlOperation; /// A [`DmlSink`] handles [`DmlOperation`] instances read from a shard. #[async_trait] -pub trait DmlSink: Debug + Send + Sync { +pub(crate) trait DmlSink: Debug + Send + Sync { /// Apply `op` read from a shard, returning `Ok(true)` if ingest should /// be paused. async fn apply(&self, op: DmlOperation) -> Result; diff --git a/ingester/src/stream_handler/sink_instrumentation.rs b/ingester/src/stream_handler/sink_instrumentation.rs index 24b05cbf21..998e14bb48 100644 --- a/ingester/src/stream_handler/sink_instrumentation.rs +++ b/ingester/src/stream_handler/sink_instrumentation.rs @@ -414,11 +414,13 @@ mod tests { let got = test( op, &metrics, - Err(crate::data::Error::TableNotPresent), + Err(crate::data::Error::NamespaceNotFound { + namespace: "bananas".to_string(), + }), Some(12345), ) .await; - assert_matches!(got, Err(crate::data::Error::TableNotPresent)); + assert_matches!(got, Err(crate::data::Error::NamespaceNotFound { .. })); // Validate the various write buffer metrics assert_matches!( diff --git a/ingester/src/test_util.rs b/ingester/src/test_util.rs index cde40ac9c2..05dc226f90 100644 --- a/ingester/src/test_util.rs +++ b/ingester/src/test_util.rs @@ -9,17 +9,16 @@ use arrow::record_batch::RecordBatch; use arrow_util::assert_batches_eq; use bitflags::bitflags; use data_types::{ - CompactionLevel, NamespaceId, NonEmptyString, PartitionId, PartitionKey, Sequence, - SequenceNumber, ShardId, ShardIndex, TableId, Timestamp, Tombstone, TombstoneId, + CompactionLevel, NamespaceId, PartitionId, PartitionKey, Sequence, SequenceNumber, ShardId, + ShardIndex, TableId, }; -use dml::{DmlDelete, DmlMeta, DmlOperation, DmlWrite}; +use dml::{DmlMeta, DmlOperation, DmlWrite}; use iox_catalog::{interface::Catalog, mem::MemCatalog}; use iox_query::test::{raw_data, TestChunk}; use iox_time::{SystemProvider, Time}; use mutable_batch_lp::lines_to_batches; use object_store::memory::InMemory; use parquet_file::metadata::IoxMetadata; -use predicate::delete_predicate::parse_delete_predicate; use schema::sort::SortKey; use uuid::Uuid; @@ -28,31 +27,10 @@ use crate::{ partition::{resolver::CatalogPartitionResolver, PersistingBatch, SnapshotBatch}, IngesterData, }, - lifecycle::{LifecycleConfig, LifecycleHandle, LifecycleManager}, + lifecycle::{LifecycleConfig, LifecycleManager}, query::QueryableBatch, }; -/// Create tombstone for testing -pub(crate) fn create_tombstone( - id: i64, - table_id: i64, - shard_id: i64, - seq_num: i64, - min_time: i64, - max_time: i64, - predicate: &str, -) -> Tombstone { - Tombstone { - id: TombstoneId::new(id), - table_id: TableId::new(table_id), - shard_id: ShardId::new(shard_id), - sequence_number: SequenceNumber::new(seq_num), - min_time: Timestamp::new(min_time), - max_time: Timestamp::new(max_time), - serialized_predicate: predicate.to_string(), - } -} - #[allow(clippy::too_many_arguments)] pub(crate) fn make_meta( object_store_id: Uuid, @@ -93,15 +71,8 @@ pub(crate) fn make_persisting_batch( partition_id: i64, object_store_id: Uuid, batches: Vec>, - tombstones: Vec, ) -> Arc { - let queryable_batch = make_queryable_batch_with_deletes( - table_name, - partition_id, - seq_num_start, - batches, - tombstones, - ); + let queryable_batch = make_queryable_batch(table_name, partition_id, seq_num_start, batches); Arc::new(PersistingBatch { shard_id: ShardId::new(shard_id), table_id: TableId::new(table_id), @@ -116,16 +87,6 @@ pub(crate) fn make_queryable_batch( partition_id: i64, seq_num_start: i64, batches: Vec>, -) -> Arc { - make_queryable_batch_with_deletes(table_name, partition_id, seq_num_start, batches, vec![]) -} - -pub(crate) fn make_queryable_batch_with_deletes( - table_name: &str, - partition_id: i64, - seq_num_start: i64, - batches: Vec>, - tombstones: Vec, ) -> Arc { // make snapshots for the batches let mut snapshots = vec![]; @@ -140,7 +101,6 @@ pub(crate) fn make_queryable_batch_with_deletes( table_name.into(), PartitionId::new(partition_id), snapshots, - tombstones, )) } @@ -673,47 +633,6 @@ pub(crate) async fn make_ingester_data(two_partitions: bool, loc: DataLocation) ingester } -pub(crate) async fn make_ingester_data_with_tombstones(loc: DataLocation) -> IngesterData { - // Whatever data because they won't be used in the tests - let metrics: Arc = Default::default(); - let catalog: Arc = Arc::new(MemCatalog::new(Arc::clone(&metrics))); - let object_store = Arc::new(InMemory::new()); - let exec = Arc::new(iox_query::exec::Executor::new(1)); - let lifecycle = LifecycleManager::new( - LifecycleConfig::new( - 200_000_000, - 100_000_000, - 100_000_000, - Duration::from_secs(100_000_000), - Duration::from_secs(100_000_000), - 100_000_000, - ), - Arc::clone(&metrics), - Arc::new(SystemProvider::default()), - ); - - // Make data for one shard and two tables - let shard_index = ShardIndex::new(0); - let (shard_id, _, _) = - populate_catalog(&*catalog, shard_index, TEST_NAMESPACE, TEST_TABLE).await; - - let ingester = IngesterData::new( - object_store, - Arc::clone(&catalog), - [(shard_id, shard_index)], - exec, - Arc::new(CatalogPartitionResolver::new(catalog)), - backoff::BackoffConfig::default(), - metrics, - ); - - // Make partitions per requested - make_one_partition_with_tombstones(&ingester, &lifecycle.handle(), loc, shard_index, shard_id) - .await; - - ingester -} - /// Make data for one or two partitions per requested pub(crate) fn make_partitions(two_partitions: bool, shard_index: ShardIndex) -> Vec { // In-memory data includes these rows but split between 4 groups go into @@ -783,133 +702,6 @@ pub(crate) fn make_partitions(two_partitions: bool, shard_index: ShardIndex) -> ops } -/// Make data for one partition with tombstones -async fn make_one_partition_with_tombstones( - ingester: &IngesterData, - lifecycle_handle: &dyn LifecycleHandle, - loc: DataLocation, - shard_index: ShardIndex, - shard_id: ShardId, -) { - // In-memory data includes these rows but split between 4 groups go into - // different batches of parittion 1 or partittion 2 as requeted - // let expected = vec![ - // "+------------+-----+------+--------------------------------+", - // "| city | day | temp | time |", - // "+------------+-----+------+--------------------------------+", - // "| Andover | tue | 56 | 1970-01-01T00:00:00.000000030Z |", // in group 1 - seq_num: 2 - // "| Andover | mon | | 1970-01-01T00:00:00.000000046Z |", // in group 2 - seq_num: 3 - // "| Boston | sun | 60 | 1970-01-01T00:00:00.000000036Z |", // in group 1 - seq_num: 1 --> will get deleted - // "| Boston | mon | | 1970-01-01T00:00:00.000000038Z |", // in group 3 - seq_num: 5 --> will get deleted - // "| Medford | sun | 55 | 1970-01-01T00:00:00.000000022Z |", // in group 4 - seq_num: 8 (after the tombstone's seq num) - // "| Medford | wed | | 1970-01-01T00:00:00.000000026Z |", // in group 2 - seq_num: 4 - // "| Reading | mon | 58 | 1970-01-01T00:00:00.000000040Z |", // in group 4 - seq_num: 9 - // "| Wilmington | mon | | 1970-01-01T00:00:00.000000035Z |", // in group 3 - seq_num: 6 - // "+------------+-----+------+--------------------------------+", - // ]; - - let (ops, seq_num) = - make_first_partition_data(&PartitionKey::from(TEST_PARTITION_1), shard_index); - - // Apply all ops - for op in ops { - ingester - .buffer_operation(shard_id, op, lifecycle_handle) - .await - .unwrap(); - } - - if loc.contains(DataLocation::PERSISTING) { - // Move partition 1 data to persisting - let _ignored = ingester - .shard(shard_id) - .unwrap() - .namespace(&TEST_NAMESPACE.into()) - .unwrap() - .snapshot_to_persisting(&TEST_TABLE.into(), &PartitionKey::from(TEST_PARTITION_1)) - .await; - } else if loc.contains(DataLocation::SNAPSHOT) { - // move partition 1 data to snapshot - let _ignored = ingester - .shard(shard_id) - .unwrap() - .namespace(&TEST_NAMESPACE.into()) - .unwrap() - .snapshot(&TEST_TABLE.into(), &PartitionKey::from(TEST_PARTITION_1)) - .await; - } - - // Add tombstones - // Depending on where the existing data is, they (buffer & snapshot) will be either moved to a new snapshot after - // applying the tombstone or (persisting) stay where they are and the tombstones is kept to get applied later - // ------------------------------------------ - // Delete - let mut seq_num = seq_num.get(); - seq_num += 1; - - let delete = parse_delete_predicate( - "1970-01-01T00:00:00.000000010Z", - "1970-01-01T00:00:00.000000050Z", - "city=Boston", - ) - .unwrap(); - - ingester - .buffer_operation( - shard_id, - DmlOperation::Delete(DmlDelete::new( - TEST_NAMESPACE.to_string(), - delete, - NonEmptyString::new(TEST_TABLE), - DmlMeta::sequenced( - Sequence { - shard_index, - sequence_number: SequenceNumber::new(seq_num), - }, - Time::MIN, - None, - 42, - ), - )), - lifecycle_handle, - ) - .await - .unwrap(); - - // Group 4: in buffer of p1 after the tombstone - - ingester - .buffer_operation( - shard_id, - DmlOperation::Write(make_write_op( - &PartitionKey::from(TEST_PARTITION_1), - shard_index, - TEST_NAMESPACE, - seq_num, - r#"test_table,city=Medford day="sun",temp=55 22"#, - )), - lifecycle_handle, - ) - .await - .unwrap(); - seq_num += 1; - - ingester - .buffer_operation( - shard_id, - DmlOperation::Write(make_write_op( - &PartitionKey::from(TEST_PARTITION_1), - shard_index, - TEST_NAMESPACE, - seq_num, - r#"test_table,city=Reading day="mon",temp=58 40"#, - )), - lifecycle_handle, - ) - .await - .unwrap(); -} - pub(crate) fn make_write_op( partition_key: &PartitionKey, shard_index: ShardIndex, diff --git a/iox_catalog/src/postgres.rs b/iox_catalog/src/postgres.rs index 7544e65370..d28a5f310d 100644 --- a/iox_catalog/src/postgres.rs +++ b/iox_catalog/src/postgres.rs @@ -1878,7 +1878,7 @@ LIMIT $4; sqlx::query_as::<_, PartitionParam>( r#" SELECT parquet_file.partition_id, parquet_file.shard_id, parquet_file.namespace_id, - parquet_file.table_id, + parquet_file.table_id, count(case when to_delete is null then 1 end) total_count, max(case when compaction_level= $4 then parquet_file.created_at end) FROM parquet_file diff --git a/querier/src/ingester/mod.rs b/querier/src/ingester/mod.rs index 86946d2c54..9c9f7a8910 100644 --- a/querier/src/ingester/mod.rs +++ b/querier/src/ingester/mod.rs @@ -613,9 +613,7 @@ impl IngesterStreamDecoder { partition_id, shard_id, status.parquet_max_sequence_number.map(SequenceNumber::new), - status - .tombstone_max_sequence_number - .map(SequenceNumber::new), + None, partition_sort_key, ); self.current_partition = Some(partition); @@ -1338,7 +1336,6 @@ mod tests { partition_id: 1, status: Some(PartitionStatus { parquet_max_sequence_number: None, - tombstone_max_sequence_number: None, }), }, ))], @@ -1394,7 +1391,6 @@ mod tests { partition_id: 1, status: Some(PartitionStatus { parquet_max_sequence_number: None, - tombstone_max_sequence_number: None, }), }, )), @@ -1404,7 +1400,6 @@ mod tests { partition_id: 2, status: Some(PartitionStatus { parquet_max_sequence_number: None, - tombstone_max_sequence_number: None, }), }, )), @@ -1414,7 +1409,6 @@ mod tests { partition_id: 1, status: Some(PartitionStatus { parquet_max_sequence_number: None, - tombstone_max_sequence_number: None, }), }, )), @@ -1494,7 +1488,6 @@ mod tests { partition_id: 1, status: Some(PartitionStatus { parquet_max_sequence_number: Some(11), - tombstone_max_sequence_number: Some(12), }), }, )), @@ -1524,7 +1517,6 @@ mod tests { partition_id: 2, status: Some(PartitionStatus { parquet_max_sequence_number: Some(21), - tombstone_max_sequence_number: Some(22), }), }, )), @@ -1549,7 +1541,6 @@ mod tests { partition_id: 3, status: Some(PartitionStatus { parquet_max_sequence_number: Some(31), - tombstone_max_sequence_number: Some(32), }), }, )), @@ -1579,10 +1570,7 @@ mod tests { p1.parquet_max_sequence_number, Some(SequenceNumber::new(11)) ); - assert_eq!( - p1.tombstone_max_sequence_number, - Some(SequenceNumber::new(12)) - ); + assert_eq!(p1.tombstone_max_sequence_number, None); assert_eq!(p1.chunks.len(), 2); assert_eq!(p1.chunks[0].schema().as_arrow(), schema_1_1); assert_eq!(p1.chunks[0].batches.len(), 2); @@ -1599,10 +1587,7 @@ mod tests { p2.parquet_max_sequence_number, Some(SequenceNumber::new(21)) ); - assert_eq!( - p2.tombstone_max_sequence_number, - Some(SequenceNumber::new(22)) - ); + assert_eq!(p2.tombstone_max_sequence_number, None); assert_eq!(p2.chunks.len(), 1); assert_eq!(p2.chunks[0].schema().as_arrow(), schema_2_1); assert_eq!(p2.chunks[0].batches.len(), 1); @@ -1615,10 +1600,7 @@ mod tests { p3.parquet_max_sequence_number, Some(SequenceNumber::new(31)) ); - assert_eq!( - p3.tombstone_max_sequence_number, - Some(SequenceNumber::new(32)) - ); + assert_eq!(p3.tombstone_max_sequence_number, None); assert_eq!(p3.chunks.len(), 1); assert_eq!(p3.chunks[0].schema().as_arrow(), schema_3_1); assert_eq!(p3.chunks[0].batches.len(), 1); @@ -1738,7 +1720,6 @@ mod tests { partition_id: 1, status: Some(PartitionStatus { parquet_max_sequence_number: Some(11), - tombstone_max_sequence_number: Some(12), }), }, )), @@ -1778,10 +1759,7 @@ mod tests { p1.parquet_max_sequence_number, Some(SequenceNumber::new(11)) ); - assert_eq!( - p1.tombstone_max_sequence_number, - Some(SequenceNumber::new(12)) - ); + assert_eq!(p1.tombstone_max_sequence_number, None); assert_eq!(p1.chunks.len(), 1); } diff --git a/query_tests/cases/in/delete_all.expected b/query_tests/cases/in/delete_all.expected deleted file mode 100644 index ba828eab9a..0000000000 --- a/query_tests/cases/in/delete_all.expected +++ /dev/null @@ -1,25 +0,0 @@ --- Test Setup: OneDeleteSimpleExprOneChunkDeleteAll --- SQL: SELECT * from cpu; -++ -++ --- SQL: SELECT time from cpu; -++ -++ --- SQL: SELECT count(*), count(bar), count(time) from cpu; -+-----------------+----------------+-----------------+ -| COUNT(UInt8(1)) | COUNT(cpu.bar) | COUNT(cpu.time) | -+-----------------+----------------+-----------------+ -| 0 | 0 | 0 | -+-----------------+----------------+-----------------+ --- SQL: SELECT min(bar), max(bar), min(time), max(time) from cpu; -+--------------+--------------+---------------+---------------+ -| MIN(cpu.bar) | MAX(cpu.bar) | MIN(cpu.time) | MAX(cpu.time) | -+--------------+--------------+---------------+---------------+ -| | | | | -+--------------+--------------+---------------+---------------+ --- SQL: SELECT max(bar) from cpu; -+--------------+ -| MAX(cpu.bar) | -+--------------+ -| | -+--------------+ diff --git a/query_tests/cases/in/delete_all.sql b/query_tests/cases/in/delete_all.sql deleted file mode 100644 index b79612846e..0000000000 --- a/query_tests/cases/in/delete_all.sql +++ /dev/null @@ -1,17 +0,0 @@ --- Demonstrate soft deleted rows will not be return to queries --- IOX_SETUP: OneDeleteSimpleExprOneChunkDeleteAll - --- select * -SELECT * from cpu; - --- select one specific column -SELECT time from cpu; - --- select aggregate of every column inlcuding star -SELECT count(*), count(bar), count(time) from cpu; - --- select aggregate of every column -SELECT min(bar), max(bar), min(time), max(time) from cpu; - --- select aggregate of one column -SELECT max(bar) from cpu; \ No newline at end of file diff --git a/query_tests/cases/in/delete_multi_expr_one_chunk.expected b/query_tests/cases/in/delete_multi_expr_one_chunk.expected deleted file mode 100644 index f0765f7c16..0000000000 --- a/query_tests/cases/in/delete_multi_expr_one_chunk.expected +++ /dev/null @@ -1,207 +0,0 @@ --- Test Setup: OneDeleteMultiExprsOneChunk --- SQL: SELECT * from cpu order by bar, foo, time; -+-----+-----+--------------------------------+ -| bar | foo | time | -+-----+-----+--------------------------------+ -| 1 | me | 1970-01-01T00:00:00.000000040Z | -| 2 | you | 1970-01-01T00:00:00.000000020Z | -+-----+-----+--------------------------------+ --- SQL: SELECT time, bar from cpu order by time, bar; -+--------------------------------+-----+ -| time | bar | -+--------------------------------+-----+ -| 1970-01-01T00:00:00.000000020Z | 2 | -| 1970-01-01T00:00:00.000000040Z | 1 | -+--------------------------------+-----+ --- SQL: SELECT bar from cpu order by bar; -+-----+ -| bar | -+-----+ -| 1 | -| 2 | -+-----+ --- SQL: SELECT count(time), count(*), count(bar), min(bar), max(bar), min(time), max(time) from cpu; -+-----------------+-----------------+----------------+--------------+--------------+--------------------------------+--------------------------------+ -| COUNT(cpu.time) | COUNT(UInt8(1)) | COUNT(cpu.bar) | MIN(cpu.bar) | MAX(cpu.bar) | MIN(cpu.time) | MAX(cpu.time) | -+-----------------+-----------------+----------------+--------------+--------------+--------------------------------+--------------------------------+ -| 2 | 2 | 2 | 1 | 2 | 1970-01-01T00:00:00.000000020Z | 1970-01-01T00:00:00.000000040Z | -+-----------------+-----------------+----------------+--------------+--------------+--------------------------------+--------------------------------+ --- SQL: SELECT count(time) from cpu; -+-----------------+ -| COUNT(cpu.time) | -+-----------------+ -| 2 | -+-----------------+ --- SQL: SELECT count(foo) from cpu; -+----------------+ -| COUNT(cpu.foo) | -+----------------+ -| 2 | -+----------------+ --- SQL: SELECT count(bar) from cpu; -+----------------+ -| COUNT(cpu.bar) | -+----------------+ -| 2 | -+----------------+ --- SQL: SELECT count(*) from cpu; -+-----------------+ -| COUNT(UInt8(1)) | -+-----------------+ -| 2 | -+-----------------+ --- SQL: SELECT min(bar) from cpu; -+--------------+ -| MIN(cpu.bar) | -+--------------+ -| 1 | -+--------------+ --- SQL: SELECT foo from cpu; --- Results After Sorting -+-----+ -| foo | -+-----+ -| me | -| you | -+-----+ --- SQL: SELECT min(foo) as min_foo from cpu order by min_foo; -+---------+ -| min_foo | -+---------+ -| me | -+---------+ --- SQL: SELECT max(foo) as max_foo from cpu order by max_foo; -+---------+ -| max_foo | -+---------+ -| you | -+---------+ --- SQL: SELECT min(foo) as min_foo from cpu group by time order by min_foo; -+---------+ -| min_foo | -+---------+ -| me | -| you | -+---------+ --- SQL: SELECT max(foo) as max_foo from cpu group by time order by max_foo; -+---------+ -| max_foo | -+---------+ -| me | -| you | -+---------+ --- SQL: SELECT time, max(foo) as max_foo from cpu group by time order by time, max_foo; -+--------------------------------+---------+ -| time | max_foo | -+--------------------------------+---------+ -| 1970-01-01T00:00:00.000000020Z | you | -| 1970-01-01T00:00:00.000000040Z | me | -+--------------------------------+---------+ --- SQL: SELECT min(foo) as min_foo from cpu group by bar order by min_foo; -+---------+ -| min_foo | -+---------+ -| me | -| you | -+---------+ --- SQL: SELECT bar, max(foo) as max_foo from cpu group by bar order by bar, max_foo; -+-----+---------+ -| bar | max_foo | -+-----+---------+ -| 1 | me | -| 2 | you | -+-----+---------+ --- SQL: SELECT max(foo) as max_foo from cpu group by time order by max_foo; -+---------+ -| max_foo | -+---------+ -| me | -| you | -+---------+ --- SQL: SELECT min(time) as min_time from cpu order by min_time; -+--------------------------------+ -| min_time | -+--------------------------------+ -| 1970-01-01T00:00:00.000000020Z | -+--------------------------------+ --- SQL: SELECT max(time) as max_time from cpu order by max_time; -+--------------------------------+ -| max_time | -+--------------------------------+ -| 1970-01-01T00:00:00.000000040Z | -+--------------------------------+ --- SQL: SELECT min(time) as min_time from cpu group by bar order by min_time; -+--------------------------------+ -| min_time | -+--------------------------------+ -| 1970-01-01T00:00:00.000000020Z | -| 1970-01-01T00:00:00.000000040Z | -+--------------------------------+ --- SQL: SELECT bar, min(time) as min_time from cpu group by bar order by bar, min_time; -+-----+--------------------------------+ -| bar | min_time | -+-----+--------------------------------+ -| 1 | 1970-01-01T00:00:00.000000040Z | -| 2 | 1970-01-01T00:00:00.000000020Z | -+-----+--------------------------------+ --- SQL: SELECT max(time) as max_time from cpu group by foo order by max_time; -+--------------------------------+ -| max_time | -+--------------------------------+ -| 1970-01-01T00:00:00.000000020Z | -| 1970-01-01T00:00:00.000000040Z | -+--------------------------------+ --- SQL: SELECT foo, max(time) as max_time from cpu group by foo order by foo, max_time; -+-----+--------------------------------+ -| foo | max_time | -+-----+--------------------------------+ -| me | 1970-01-01T00:00:00.000000040Z | -| you | 1970-01-01T00:00:00.000000020Z | -+-----+--------------------------------+ --- SQL: SELECT time from cpu; --- Results After Sorting -+--------------------------------+ -| time | -+--------------------------------+ -| 1970-01-01T00:00:00.000000020Z | -| 1970-01-01T00:00:00.000000040Z | -+--------------------------------+ --- SQL: SELECT max(bar) from cpu order by 1; -+--------------+ -| MAX(cpu.bar) | -+--------------+ -| 2 | -+--------------+ --- SQL: SELECT * from cpu where bar >= 1.0 order by bar, foo, time; -+-----+-----+--------------------------------+ -| bar | foo | time | -+-----+-----+--------------------------------+ -| 1 | me | 1970-01-01T00:00:00.000000040Z | -| 2 | you | 1970-01-01T00:00:00.000000020Z | -+-----+-----+--------------------------------+ --- SQL: SELECT foo from cpu where bar >= 1.0 order by foo; -+-----+ -| foo | -+-----+ -| me | -| you | -+-----+ --- SQL: SELECT time, bar from cpu where bar >= 1.0 order by bar, time; -+--------------------------------+-----+ -| time | bar | -+--------------------------------+-----+ -| 1970-01-01T00:00:00.000000040Z | 1 | -| 1970-01-01T00:00:00.000000020Z | 2 | -+--------------------------------+-----+ --- SQL: SELECT * from cpu where foo = 'you' order by bar, foo, time; -+-----+-----+--------------------------------+ -| bar | foo | time | -+-----+-----+--------------------------------+ -| 2 | you | 1970-01-01T00:00:00.000000020Z | -+-----+-----+--------------------------------+ --- SQL: SELECT min(bar) as mi, max(time) as ma from cpu where foo = 'you' order by mi, ma -+----+--------------------------------+ -| mi | ma | -+----+--------------------------------+ -| 2 | 1970-01-01T00:00:00.000000020Z | -+----+--------------------------------+ diff --git a/query_tests/cases/in/delete_multi_expr_one_chunk.sql b/query_tests/cases/in/delete_multi_expr_one_chunk.sql deleted file mode 100644 index 5295c53055..0000000000 --- a/query_tests/cases/in/delete_multi_expr_one_chunk.sql +++ /dev/null @@ -1,61 +0,0 @@ --- Demonstrate soft deleted rows will not be return to queries --- IOX_SETUP: OneDeleteMultiExprsOneChunk - --- select * -SELECT * from cpu order by bar, foo, time; - -SELECT time, bar from cpu order by time, bar; - -SELECT bar from cpu order by bar; - -SELECT count(time), count(*), count(bar), min(bar), max(bar), min(time), max(time) from cpu; - -SELECT count(time) from cpu; - -SELECT count(foo) from cpu; - -SELECT count(bar) from cpu; - -SELECT count(*) from cpu; - -SELECT min(bar) from cpu; - --- IOX_COMPARE: sorted -SELECT foo from cpu; - -SELECT min(foo) as min_foo from cpu order by min_foo; -SELECT max(foo) as max_foo from cpu order by max_foo; - -SELECT min(foo) as min_foo from cpu group by time order by min_foo; -SELECT max(foo) as max_foo from cpu group by time order by max_foo; -SELECT time, max(foo) as max_foo from cpu group by time order by time, max_foo; - -SELECT min(foo) as min_foo from cpu group by bar order by min_foo; -SELECT bar, max(foo) as max_foo from cpu group by bar order by bar, max_foo; -SELECT max(foo) as max_foo from cpu group by time order by max_foo; - -SELECT min(time) as min_time from cpu order by min_time; -SELECT max(time) as max_time from cpu order by max_time; - -SELECT min(time) as min_time from cpu group by bar order by min_time; -SELECT bar, min(time) as min_time from cpu group by bar order by bar, min_time; -SELECT max(time) as max_time from cpu group by foo order by max_time; -SELECT foo, max(time) as max_time from cpu group by foo order by foo, max_time; - --- IOX_COMPARE: sorted -SELECT time from cpu; - -SELECT max(bar) from cpu order by 1; - --------------------------------------------------------- --- With selection predicate - -SELECT * from cpu where bar >= 1.0 order by bar, foo, time; - -SELECT foo from cpu where bar >= 1.0 order by foo; - -SELECT time, bar from cpu where bar >= 1.0 order by bar, time; - -SELECT * from cpu where foo = 'you' order by bar, foo, time; - -SELECT min(bar) as mi, max(time) as ma from cpu where foo = 'you' order by mi, ma diff --git a/query_tests/cases/in/delete_simple_pred_one_chunk.expected b/query_tests/cases/in/delete_simple_pred_one_chunk.expected deleted file mode 100644 index f367cdefef..0000000000 --- a/query_tests/cases/in/delete_simple_pred_one_chunk.expected +++ /dev/null @@ -1,91 +0,0 @@ --- Test Setup: OneDeleteSimpleExprOneChunk --- SQL: SELECT * from cpu; -+-----+--------------------------------+ -| bar | time | -+-----+--------------------------------+ -| 2 | 1970-01-01T00:00:00.000000020Z | -+-----+--------------------------------+ --- SQL: SELECT time, bar from cpu; -+--------------------------------+-----+ -| time | bar | -+--------------------------------+-----+ -| 1970-01-01T00:00:00.000000020Z | 2 | -+--------------------------------+-----+ --- SQL: SELECT min(bar), max(bar) from cpu; -+--------------+--------------+ -| MIN(cpu.bar) | MAX(cpu.bar) | -+--------------+--------------+ -| 2 | 2 | -+--------------+--------------+ --- SQL: SELECT time from cpu; -+--------------------------------+ -| time | -+--------------------------------+ -| 1970-01-01T00:00:00.000000020Z | -+--------------------------------+ --- SQL: SELECT max(time) from cpu; -+--------------------------------+ -| MAX(cpu.time) | -+--------------------------------+ -| 1970-01-01T00:00:00.000000020Z | -+--------------------------------+ --- SQL: SELECT min(time) from cpu group by bar; -+--------------------------------+ -| MIN(cpu.time) | -+--------------------------------+ -| 1970-01-01T00:00:00.000000020Z | -+--------------------------------+ --- SQL: SELECT bar, min(time) from cpu group by bar; -+-----+--------------------------------+ -| bar | MIN(cpu.time) | -+-----+--------------------------------+ -| 2 | 1970-01-01T00:00:00.000000020Z | -+-----+--------------------------------+ --- SQL: SELECT count(time), max(time) from cpu; -+-----------------+--------------------------------+ -| COUNT(cpu.time) | MAX(cpu.time) | -+-----------------+--------------------------------+ -| 1 | 1970-01-01T00:00:00.000000020Z | -+-----------------+--------------------------------+ --- SQL: SELECT count(time) from cpu; -+-----------------+ -| COUNT(cpu.time) | -+-----------------+ -| 1 | -+-----------------+ --- SQL: SELECT count(time), count(*), count(bar), min(bar), max(bar), min(time), max(time) from cpu; -+-----------------+-----------------+----------------+--------------+--------------+--------------------------------+--------------------------------+ -| COUNT(cpu.time) | COUNT(UInt8(1)) | COUNT(cpu.bar) | MIN(cpu.bar) | MAX(cpu.bar) | MIN(cpu.time) | MAX(cpu.time) | -+-----------------+-----------------+----------------+--------------+--------------+--------------------------------+--------------------------------+ -| 1 | 1 | 1 | 2 | 2 | 1970-01-01T00:00:00.000000020Z | 1970-01-01T00:00:00.000000020Z | -+-----------------+-----------------+----------------+--------------+--------------+--------------------------------+--------------------------------+ --- SQL: SELECT * from cpu where bar = 2.0; -+-----+--------------------------------+ -| bar | time | -+-----+--------------------------------+ -| 2 | 1970-01-01T00:00:00.000000020Z | -+-----+--------------------------------+ --- SQL: SELECT * from cpu where bar != 2.0; -++ -++ --- SQL: SELECT count(time), count(*), count(bar), min(bar), max(bar), min(time), max(time) from cpu where bar= 2.0; -+-----------------+-----------------+----------------+--------------+--------------+--------------------------------+--------------------------------+ -| COUNT(cpu.time) | COUNT(UInt8(1)) | COUNT(cpu.bar) | MIN(cpu.bar) | MAX(cpu.bar) | MIN(cpu.time) | MAX(cpu.time) | -+-----------------+-----------------+----------------+--------------+--------------+--------------------------------+--------------------------------+ -| 1 | 1 | 1 | 2 | 2 | 1970-01-01T00:00:00.000000020Z | 1970-01-01T00:00:00.000000020Z | -+-----------------+-----------------+----------------+--------------+--------------+--------------------------------+--------------------------------+ --- SQL: SELECT count(time), count(*), count(bar), min(bar), max(bar), min(time), max(time) from cpu where bar != 2.0; -+-----------------+-----------------+----------------+--------------+--------------+---------------+---------------+ -| COUNT(cpu.time) | COUNT(UInt8(1)) | COUNT(cpu.bar) | MIN(cpu.bar) | MAX(cpu.bar) | MIN(cpu.time) | MAX(cpu.time) | -+-----------------+-----------------+----------------+--------------+--------------+---------------+---------------+ -| 0 | 0 | 0 | | | | | -+-----------------+-----------------+----------------+--------------+--------------+---------------+---------------+ --- SQL: SELECT time from cpu where bar=2; -+--------------------------------+ -| time | -+--------------------------------+ -| 1970-01-01T00:00:00.000000020Z | -+--------------------------------+ --- SQL: SELECT bar from cpu where bar!= 2; -++ -++ diff --git a/query_tests/cases/in/delete_simple_pred_one_chunk.sql b/query_tests/cases/in/delete_simple_pred_one_chunk.sql deleted file mode 100644 index 7b22641c63..0000000000 --- a/query_tests/cases/in/delete_simple_pred_one_chunk.sql +++ /dev/null @@ -1,37 +0,0 @@ --- Demonstrate soft deleted rows will not be return to queries --- IOX_SETUP: OneDeleteSimpleExprOneChunk - --- select * -SELECT * from cpu; - -SELECT time, bar from cpu; - -SELECT min(bar), max(bar) from cpu; - -SELECT time from cpu; - -SELECT max(time) from cpu; -SELECT min(time) from cpu group by bar; -SELECT bar, min(time) from cpu group by bar; - -SELECT count(time), max(time) from cpu; - -SELECT count(time) from cpu; - -SELECT count(time), count(*), count(bar), min(bar), max(bar), min(time), max(time) from cpu; - ----------------------------------------------------------------- --- Now add selection predicate -SELECT * from cpu where bar = 2.0; - -SELECT * from cpu where bar != 2.0; - -SELECT count(time), count(*), count(bar), min(bar), max(bar), min(time), max(time) from cpu where bar= 2.0; - -SELECT count(time), count(*), count(bar), min(bar), max(bar), min(time), max(time) from cpu where bar != 2.0; - -SELECT time from cpu where bar=2; - -SELECT bar from cpu where bar!= 2; - - diff --git a/query_tests/cases/in/delete_three_chunks_1.expected b/query_tests/cases/in/delete_three_chunks_1.expected deleted file mode 100644 index 47ec3d3de4..0000000000 --- a/query_tests/cases/in/delete_three_chunks_1.expected +++ /dev/null @@ -1,85 +0,0 @@ --- Test Setup: ThreeDeleteThreeChunks --- SQL: SELECT * from cpu order by foo, bar, time; -+-----+-----+--------------------------------+ -| bar | foo | time | -+-----+-----+--------------------------------+ -| 1 | me | 1970-01-01T00:00:00.000000040Z | -| 1 | me | 1970-01-01T00:00:00.000000042Z | -| 1 | me | 1970-01-01T00:00:00.000000062Z | -| 4 | me | 1970-01-01T00:00:00.000000050Z | -| 5 | me | 1970-01-01T00:00:00.000000060Z | -| 7 | me | 1970-01-01T00:00:00.000000080Z | -| 3 | you | 1970-01-01T00:00:00.000000070Z | -+-----+-----+--------------------------------+ --- SQL: SELECT time, bar from cpu order by bar, time; -+--------------------------------+-----+ -| time | bar | -+--------------------------------+-----+ -| 1970-01-01T00:00:00.000000040Z | 1 | -| 1970-01-01T00:00:00.000000042Z | 1 | -| 1970-01-01T00:00:00.000000062Z | 1 | -| 1970-01-01T00:00:00.000000070Z | 3 | -| 1970-01-01T00:00:00.000000050Z | 4 | -| 1970-01-01T00:00:00.000000060Z | 5 | -| 1970-01-01T00:00:00.000000080Z | 7 | -+--------------------------------+-----+ --- SQL: SELECT bar from cpu order by bar; -+-----+ -| bar | -+-----+ -| 1 | -| 1 | -| 1 | -| 3 | -| 4 | -| 5 | -| 7 | -+-----+ --- SQL: SELECT count(time) as t, count(*) as c, count(bar) as b, min(bar) as mi, min(time) as mt, max(time) as mat from cpu order by t, c, b, mi, mt, mat; -+---+---+---+----+--------------------------------+--------------------------------+ -| t | c | b | mi | mt | mat | -+---+---+---+----+--------------------------------+--------------------------------+ -| 7 | 7 | 7 | 1 | 1970-01-01T00:00:00.000000040Z | 1970-01-01T00:00:00.000000080Z | -+---+---+---+----+--------------------------------+--------------------------------+ --- SQL: SELECT count(time) from cpu; -+-----------------+ -| COUNT(cpu.time) | -+-----------------+ -| 7 | -+-----------------+ --- SQL: SELECT count(foo) from cpu; -+----------------+ -| COUNT(cpu.foo) | -+----------------+ -| 7 | -+----------------+ --- SQL: SELECT count(bar) from cpu; -+----------------+ -| COUNT(cpu.bar) | -+----------------+ -| 7 | -+----------------+ --- SQL: SELECT count(*) from cpu; -+-----------------+ -| COUNT(UInt8(1)) | -+-----------------+ -| 7 | -+-----------------+ --- SQL: SELECT min(bar) from cpu; -+--------------+ -| MIN(cpu.bar) | -+--------------+ -| 1 | -+--------------+ --- SQL: SELECT foo from cpu order by foo; -+-----+ -| foo | -+-----+ -| me | -| me | -| me | -| me | -| me | -| me | -| you | -+-----+ diff --git a/query_tests/cases/in/delete_three_chunks_1.sql b/query_tests/cases/in/delete_three_chunks_1.sql deleted file mode 100644 index c0105412e9..0000000000 --- a/query_tests/cases/in/delete_three_chunks_1.sql +++ /dev/null @@ -1,23 +0,0 @@ --- Demonstrate soft deleted rows will not be return to queries --- IOX_SETUP: ThreeDeleteThreeChunks - --- select * -SELECT * from cpu order by foo, bar, time; - -SELECT time, bar from cpu order by bar, time; - -SELECT bar from cpu order by bar; - -SELECT count(time) as t, count(*) as c, count(bar) as b, min(bar) as mi, min(time) as mt, max(time) as mat from cpu order by t, c, b, mi, mt, mat; - -SELECT count(time) from cpu; - -SELECT count(foo) from cpu; - -SELECT count(bar) from cpu; - -SELECT count(*) from cpu; - -SELECT min(bar) from cpu; - -SELECT foo from cpu order by foo; diff --git a/query_tests/cases/in/delete_three_chunks_2.expected b/query_tests/cases/in/delete_three_chunks_2.expected deleted file mode 100644 index 99fda88e70..0000000000 --- a/query_tests/cases/in/delete_three_chunks_2.expected +++ /dev/null @@ -1,77 +0,0 @@ --- Test Setup: ThreeDeleteThreeChunks --- SQL: SELECT min(foo) from cpu; -+--------------+ -| MIN(cpu.foo) | -+--------------+ -| me | -+--------------+ --- SQL: SELECT max(foo) from cpu; -+--------------+ -| MAX(cpu.foo) | -+--------------+ -| you | -+--------------+ --- SQL: SELECT min(time) from cpu; -+--------------------------------+ -| MIN(cpu.time) | -+--------------------------------+ -| 1970-01-01T00:00:00.000000040Z | -+--------------------------------+ --- SQL: SELECT max(time) from cpu; -+--------------------------------+ -| MAX(cpu.time) | -+--------------------------------+ -| 1970-01-01T00:00:00.000000080Z | -+--------------------------------+ --- SQL: SELECT foo, min(time) from cpu group by foo; --- Results After Sorting -+-----+--------------------------------+ -| foo | MIN(cpu.time) | -+-----+--------------------------------+ -| me | 1970-01-01T00:00:00.000000040Z | -| you | 1970-01-01T00:00:00.000000070Z | -+-----+--------------------------------+ --- SQL: SELECT bar, max(time) as max_time from cpu group by bar order by bar, max_time; -+-----+--------------------------------+ -| bar | max_time | -+-----+--------------------------------+ -| 1 | 1970-01-01T00:00:00.000000062Z | -| 3 | 1970-01-01T00:00:00.000000070Z | -| 4 | 1970-01-01T00:00:00.000000050Z | -| 5 | 1970-01-01T00:00:00.000000060Z | -| 7 | 1970-01-01T00:00:00.000000080Z | -+-----+--------------------------------+ --- SQL: SELECT max(time) as max_time from cpu group by bar order by max_time; -+--------------------------------+ -| max_time | -+--------------------------------+ -| 1970-01-01T00:00:00.000000050Z | -| 1970-01-01T00:00:00.000000060Z | -| 1970-01-01T00:00:00.000000062Z | -| 1970-01-01T00:00:00.000000070Z | -| 1970-01-01T00:00:00.000000080Z | -+--------------------------------+ --- SQL: SELECT time from cpu order by time; -+--------------------------------+ -| time | -+--------------------------------+ -| 1970-01-01T00:00:00.000000040Z | -| 1970-01-01T00:00:00.000000042Z | -| 1970-01-01T00:00:00.000000050Z | -| 1970-01-01T00:00:00.000000060Z | -| 1970-01-01T00:00:00.000000062Z | -| 1970-01-01T00:00:00.000000070Z | -| 1970-01-01T00:00:00.000000080Z | -+--------------------------------+ --- SQL: SELECT max(bar) from cpu; -+--------------+ -| MAX(cpu.bar) | -+--------------+ -| 7 | -+--------------+ --- SQL: SELECT min(time), max(time) from cpu; -+--------------------------------+--------------------------------+ -| MIN(cpu.time) | MAX(cpu.time) | -+--------------------------------+--------------------------------+ -| 1970-01-01T00:00:00.000000040Z | 1970-01-01T00:00:00.000000080Z | -+--------------------------------+--------------------------------+ diff --git a/query_tests/cases/in/delete_three_chunks_2.sql b/query_tests/cases/in/delete_three_chunks_2.sql deleted file mode 100644 index bb35711393..0000000000 --- a/query_tests/cases/in/delete_three_chunks_2.sql +++ /dev/null @@ -1,19 +0,0 @@ --- Demonstrate soft deleted rows will not be return to queries --- IOX_SETUP: ThreeDeleteThreeChunks - -SELECT min(foo) from cpu; -SELECT max(foo) from cpu; - -SELECT min(time) from cpu; -SELECT max(time) from cpu; - --- IOX_COMPARE: sorted -SELECT foo, min(time) from cpu group by foo; -SELECT bar, max(time) as max_time from cpu group by bar order by bar, max_time; -SELECT max(time) as max_time from cpu group by bar order by max_time; - -SELECT time from cpu order by time; - -SELECT max(bar) from cpu; - -SELECT min(time), max(time) from cpu; diff --git a/query_tests/cases/in/delete_three_chunks_3.expected b/query_tests/cases/in/delete_three_chunks_3.expected deleted file mode 100644 index 3e0c5fb2f6..0000000000 --- a/query_tests/cases/in/delete_three_chunks_3.expected +++ /dev/null @@ -1,76 +0,0 @@ --- Test Setup: ThreeDeleteThreeChunks --- SQL: SELECT * from cpu where bar != 1.0 order by bar, foo, time; -+-----+-----+--------------------------------+ -| bar | foo | time | -+-----+-----+--------------------------------+ -| 3 | you | 1970-01-01T00:00:00.000000070Z | -| 4 | me | 1970-01-01T00:00:00.000000050Z | -| 5 | me | 1970-01-01T00:00:00.000000060Z | -| 7 | me | 1970-01-01T00:00:00.000000080Z | -+-----+-----+--------------------------------+ --- SQL: SELECT * from cpu where foo = 'me' and bar > 2.0 order by bar, foo, time; -+-----+-----+--------------------------------+ -| bar | foo | time | -+-----+-----+--------------------------------+ -| 4 | me | 1970-01-01T00:00:00.000000050Z | -| 5 | me | 1970-01-01T00:00:00.000000060Z | -| 7 | me | 1970-01-01T00:00:00.000000080Z | -+-----+-----+--------------------------------+ --- SQL: SELECT * from cpu where bar = 1 order by bar, foo, time; -+-----+-----+--------------------------------+ -| bar | foo | time | -+-----+-----+--------------------------------+ -| 1 | me | 1970-01-01T00:00:00.000000040Z | -| 1 | me | 1970-01-01T00:00:00.000000042Z | -| 1 | me | 1970-01-01T00:00:00.000000062Z | -+-----+-----+--------------------------------+ --- SQL: SELECT * from cpu where foo = 'me' and (bar > 2 or bar = 1.0) order by bar, foo, time; -+-----+-----+--------------------------------+ -| bar | foo | time | -+-----+-----+--------------------------------+ -| 1 | me | 1970-01-01T00:00:00.000000040Z | -| 1 | me | 1970-01-01T00:00:00.000000042Z | -| 1 | me | 1970-01-01T00:00:00.000000062Z | -| 4 | me | 1970-01-01T00:00:00.000000050Z | -| 5 | me | 1970-01-01T00:00:00.000000060Z | -| 7 | me | 1970-01-01T00:00:00.000000080Z | -+-----+-----+--------------------------------+ --- SQL: SELECT * from cpu where foo = 'you' and (bar > 3.0 or bar = 1) order by bar, foo, time; -++ -++ --- SQL: SELECT min(bar) from cpu where foo = 'me' and (bar > 2 or bar = 1.0); -+--------------+ -| MIN(cpu.bar) | -+--------------+ -| 1 | -+--------------+ --- SQL: SELECT max(foo) from cpu where foo = 'me' and (bar > 2 or bar = 1.0); -+--------------+ -| MAX(cpu.foo) | -+--------------+ -| me | -+--------------+ --- SQL: SELECT min(time) from cpu where foo = 'me' and (bar > 2 or bar = 1.0); -+--------------------------------+ -| MIN(cpu.time) | -+--------------------------------+ -| 1970-01-01T00:00:00.000000040Z | -+--------------------------------+ --- SQL: SELECT count(bar) from cpu where foo = 'me' and (bar > 2 or bar = 1.0); -+----------------+ -| COUNT(cpu.bar) | -+----------------+ -| 6 | -+----------------+ --- SQL: SELECT count(time) from cpu where foo = 'me' and (bar > 2 or bar = 1.0); -+-----------------+ -| COUNT(cpu.time) | -+-----------------+ -| 6 | -+-----------------+ --- SQL: SELECT count(*) from cpu where foo = 'me' and (bar > 2 or bar = 1.0); -+-----------------+ -| COUNT(UInt8(1)) | -+-----------------+ -| 6 | -+-----------------+ diff --git a/query_tests/cases/in/delete_three_chunks_3.sql b/query_tests/cases/in/delete_three_chunks_3.sql deleted file mode 100644 index 146fcaf95e..0000000000 --- a/query_tests/cases/in/delete_three_chunks_3.sql +++ /dev/null @@ -1,27 +0,0 @@ --- Demonstrate soft deleted rows will not be return to queries --- IOX_SETUP: ThreeDeleteThreeChunks - --------------------------------------------------------- --- With selection predicate - -SELECT * from cpu where bar != 1.0 order by bar, foo, time; - -SELECT * from cpu where foo = 'me' and bar > 2.0 order by bar, foo, time; - -SELECT * from cpu where bar = 1 order by bar, foo, time; - -SELECT * from cpu where foo = 'me' and (bar > 2 or bar = 1.0) order by bar, foo, time; - -SELECT * from cpu where foo = 'you' and (bar > 3.0 or bar = 1) order by bar, foo, time; - -SELECT min(bar) from cpu where foo = 'me' and (bar > 2 or bar = 1.0); - -SELECT max(foo) from cpu where foo = 'me' and (bar > 2 or bar = 1.0); - -SELECT min(time) from cpu where foo = 'me' and (bar > 2 or bar = 1.0); - -SELECT count(bar) from cpu where foo = 'me' and (bar > 2 or bar = 1.0); - -SELECT count(time) from cpu where foo = 'me' and (bar > 2 or bar = 1.0); - -SELECT count(*) from cpu where foo = 'me' and (bar > 2 or bar = 1.0); diff --git a/query_tests/cases/in/delete_three_chunks_4.expected b/query_tests/cases/in/delete_three_chunks_4.expected deleted file mode 100644 index 2283d15375..0000000000 --- a/query_tests/cases/in/delete_three_chunks_4.expected +++ /dev/null @@ -1,49 +0,0 @@ --- Test Setup: ThreeDeleteThreeChunks --- SQL: SELECT * from cpu where bar >= 1.0 order by bar, foo, time; -+-----+-----+--------------------------------+ -| bar | foo | time | -+-----+-----+--------------------------------+ -| 1 | me | 1970-01-01T00:00:00.000000040Z | -| 1 | me | 1970-01-01T00:00:00.000000042Z | -| 1 | me | 1970-01-01T00:00:00.000000062Z | -| 3 | you | 1970-01-01T00:00:00.000000070Z | -| 4 | me | 1970-01-01T00:00:00.000000050Z | -| 5 | me | 1970-01-01T00:00:00.000000060Z | -| 7 | me | 1970-01-01T00:00:00.000000080Z | -+-----+-----+--------------------------------+ --- SQL: SELECT foo from cpu where bar >= 1.0 order by foo; -+-----+ -| foo | -+-----+ -| me | -| me | -| me | -| me | -| me | -| me | -| you | -+-----+ --- SQL: SELECT time, bar from cpu where bar >= 1.0 order by bar, time; -+--------------------------------+-----+ -| time | bar | -+--------------------------------+-----+ -| 1970-01-01T00:00:00.000000040Z | 1 | -| 1970-01-01T00:00:00.000000042Z | 1 | -| 1970-01-01T00:00:00.000000062Z | 1 | -| 1970-01-01T00:00:00.000000070Z | 3 | -| 1970-01-01T00:00:00.000000050Z | 4 | -| 1970-01-01T00:00:00.000000060Z | 5 | -| 1970-01-01T00:00:00.000000080Z | 7 | -+--------------------------------+-----+ --- SQL: SELECT * from cpu where foo = 'you' order by bar, foo, time; -+-----+-----+--------------------------------+ -| bar | foo | time | -+-----+-----+--------------------------------+ -| 3 | you | 1970-01-01T00:00:00.000000070Z | -+-----+-----+--------------------------------+ --- SQL: SELECT min(bar) as mi, max(time) as ma from cpu where foo = 'you' order by mi, ma; -+----+--------------------------------+ -| mi | ma | -+----+--------------------------------+ -| 3 | 1970-01-01T00:00:00.000000070Z | -+----+--------------------------------+ diff --git a/query_tests/cases/in/delete_three_chunks_4.sql b/query_tests/cases/in/delete_three_chunks_4.sql deleted file mode 100644 index 95442f6b07..0000000000 --- a/query_tests/cases/in/delete_three_chunks_4.sql +++ /dev/null @@ -1,13 +0,0 @@ --- Demonstrate soft deleted rows will not be return to queries --- IOX_SETUP: ThreeDeleteThreeChunks - ----------- -SELECT * from cpu where bar >= 1.0 order by bar, foo, time; - -SELECT foo from cpu where bar >= 1.0 order by foo; - -SELECT time, bar from cpu where bar >= 1.0 order by bar, time; - -SELECT * from cpu where foo = 'you' order by bar, foo, time; - -SELECT min(bar) as mi, max(time) as ma from cpu where foo = 'you' order by mi, ma; diff --git a/query_tests/cases/in/delete_two_del_multi_expr_one_chunk.expected b/query_tests/cases/in/delete_two_del_multi_expr_one_chunk.expected deleted file mode 100644 index 6871fa7358..0000000000 --- a/query_tests/cases/in/delete_two_del_multi_expr_one_chunk.expected +++ /dev/null @@ -1,34 +0,0 @@ --- Test Setup: TwoDeletesMultiExprsOneChunk --- SQL: SELECT * from cpu; -+-----+-----+--------------------------------+ -| bar | foo | time | -+-----+-----+--------------------------------+ -| 1 | me | 1970-01-01T00:00:00.000000040Z | -+-----+-----+--------------------------------+ --- SQL: SELECT foo from cpu; -+-----+ -| foo | -+-----+ -| me | -+-----+ --- SQL: SELECT * from cpu where cast(time as bigint) > 30; -+-----+-----+--------------------------------+ -| bar | foo | time | -+-----+-----+--------------------------------+ -| 1 | me | 1970-01-01T00:00:00.000000040Z | -+-----+-----+--------------------------------+ --- SQL: SELECT count(bar) from cpu where cast(time as bigint) > 30; -+----------------+ -| COUNT(cpu.bar) | -+----------------+ -| 1 | -+----------------+ --- SQL: SELECT * from cpu where cast(time as bigint) > 40; -++ -++ --- SQL: SELECT max(time) from cpu where cast(time as bigint) > 40; -+---------------+ -| MAX(cpu.time) | -+---------------+ -| | -+---------------+ diff --git a/query_tests/cases/in/delete_two_del_multi_expr_one_chunk.sql b/query_tests/cases/in/delete_two_del_multi_expr_one_chunk.sql deleted file mode 100644 index 132d6f42cf..0000000000 --- a/query_tests/cases/in/delete_two_del_multi_expr_one_chunk.sql +++ /dev/null @@ -1,15 +0,0 @@ --- Demonstrate soft deleted rows will not be return to queries --- IOX_SETUP: TwoDeletesMultiExprsOneChunk - --- select * -SELECT * from cpu; - -SELECT foo from cpu; - -SELECT * from cpu where cast(time as bigint) > 30; - -SELECT count(bar) from cpu where cast(time as bigint) > 30; - -SELECT * from cpu where cast(time as bigint) > 40; - -SELECT max(time) from cpu where cast(time as bigint) > 40; diff --git a/query_tests/src/cases.rs b/query_tests/src/cases.rs index 9946819fac..69caf0dfe5 100644 --- a/query_tests/src/cases.rs +++ b/query_tests/src/cases.rs @@ -1,8 +1,7 @@ - //! This file is auto generated by query_tests/generate. //! Do not edit manually --> will result in sadness -use std::path::Path; use crate::runner::Runner; +use std::path::Path; #[tokio::test] // Tests from "basic.sql", @@ -11,141 +10,8 @@ async fn test_cases_basic_sql() { let input_path = Path::new("cases").join("in").join("basic.sql"); let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); -} - -#[tokio::test] -// Tests from "delete_all.sql", -async fn test_cases_delete_all_sql() { - test_helpers::maybe_start_logging(); - - let input_path = Path::new("cases").join("in").join("delete_all.sql"); - let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); -} - -#[tokio::test] -// Tests from "delete_multi_expr_one_chunk.sql", -async fn test_cases_delete_multi_expr_one_chunk_sql() { - test_helpers::maybe_start_logging(); - - let input_path = Path::new("cases").join("in").join("delete_multi_expr_one_chunk.sql"); - let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); -} - -#[tokio::test] -// Tests from "delete_simple_pred_one_chunk.sql", -async fn test_cases_delete_simple_pred_one_chunk_sql() { - test_helpers::maybe_start_logging(); - - let input_path = Path::new("cases").join("in").join("delete_simple_pred_one_chunk.sql"); - let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); -} - -#[tokio::test] -// Tests from "delete_three_chunks_1.sql", -async fn test_cases_delete_three_chunks_1_sql() { - test_helpers::maybe_start_logging(); - - let input_path = Path::new("cases").join("in").join("delete_three_chunks_1.sql"); - let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); -} - -#[tokio::test] -// Tests from "delete_three_chunks_2.sql", -async fn test_cases_delete_three_chunks_2_sql() { - test_helpers::maybe_start_logging(); - - let input_path = Path::new("cases").join("in").join("delete_three_chunks_2.sql"); - let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); -} - -#[tokio::test] -// Tests from "delete_three_chunks_3.sql", -async fn test_cases_delete_three_chunks_3_sql() { - test_helpers::maybe_start_logging(); - - let input_path = Path::new("cases").join("in").join("delete_three_chunks_3.sql"); - let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); -} - -#[tokio::test] -// Tests from "delete_three_chunks_4.sql", -async fn test_cases_delete_three_chunks_4_sql() { - test_helpers::maybe_start_logging(); - - let input_path = Path::new("cases").join("in").join("delete_three_chunks_4.sql"); - let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); -} - -#[tokio::test] -// Tests from "delete_two_del_multi_expr_one_chunk.sql", -async fn test_cases_delete_two_del_multi_expr_one_chunk_sql() { - test_helpers::maybe_start_logging(); - - let input_path = Path::new("cases").join("in").join("delete_two_del_multi_expr_one_chunk.sql"); - let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); + runner.run(input_path).await.expect("test failed"); + runner.flush().expect("flush worked"); } #[tokio::test] @@ -153,15 +19,12 @@ async fn test_cases_delete_two_del_multi_expr_one_chunk_sql() { async fn test_cases_duplicates_ingester_sql() { test_helpers::maybe_start_logging(); - let input_path = Path::new("cases").join("in").join("duplicates_ingester.sql"); + let input_path = Path::new("cases") + .join("in") + .join("duplicates_ingester.sql"); let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); + runner.run(input_path).await.expect("test failed"); + runner.flush().expect("flush worked"); } #[tokio::test] @@ -171,13 +34,8 @@ async fn test_cases_duplicates_parquet_sql() { let input_path = Path::new("cases").join("in").join("duplicates_parquet.sql"); let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); + runner.run(input_path).await.expect("test failed"); + runner.flush().expect("flush worked"); } #[tokio::test] @@ -185,15 +43,12 @@ async fn test_cases_duplicates_parquet_sql() { async fn test_cases_new_sql_system_tables_sql() { test_helpers::maybe_start_logging(); - let input_path = Path::new("cases").join("in").join("new_sql_system_tables.sql"); + let input_path = Path::new("cases") + .join("in") + .join("new_sql_system_tables.sql"); let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); + runner.run(input_path).await.expect("test failed"); + runner.flush().expect("flush worked"); } #[tokio::test] @@ -203,13 +58,8 @@ async fn test_cases_pushdown_sql() { let input_path = Path::new("cases").join("in").join("pushdown.sql"); let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); + runner.run(input_path).await.expect("test failed"); + runner.flush().expect("flush worked"); } #[tokio::test] @@ -219,13 +69,8 @@ async fn test_cases_selectors_sql() { let input_path = Path::new("cases").join("in").join("selectors.sql"); let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); + runner.run(input_path).await.expect("test failed"); + runner.flush().expect("flush worked"); } #[tokio::test] @@ -235,13 +80,8 @@ async fn test_cases_several_chunks_sql() { let input_path = Path::new("cases").join("in").join("several_chunks.sql"); let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); + runner.run(input_path).await.expect("test failed"); + runner.flush().expect("flush worked"); } #[tokio::test] @@ -249,15 +89,12 @@ async fn test_cases_several_chunks_sql() { async fn test_cases_sql_information_schema_sql() { test_helpers::maybe_start_logging(); - let input_path = Path::new("cases").join("in").join("sql_information_schema.sql"); + let input_path = Path::new("cases") + .join("in") + .join("sql_information_schema.sql"); let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); + runner.run(input_path).await.expect("test failed"); + runner.flush().expect("flush worked"); } #[tokio::test] @@ -267,13 +104,8 @@ async fn test_cases_timestamps_sql() { let input_path = Path::new("cases").join("in").join("timestamps.sql"); let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); + runner.run(input_path).await.expect("test failed"); + runner.flush().expect("flush worked"); } #[tokio::test] @@ -283,13 +115,8 @@ async fn test_cases_two_chunks_sql() { let input_path = Path::new("cases").join("in").join("two_chunks.sql"); let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); + runner.run(input_path).await.expect("test failed"); + runner.flush().expect("flush worked"); } #[tokio::test] @@ -297,13 +124,10 @@ async fn test_cases_two_chunks_sql() { async fn test_cases_two_chunks_missing_columns_sql() { test_helpers::maybe_start_logging(); - let input_path = Path::new("cases").join("in").join("two_chunks_missing_columns.sql"); + let input_path = Path::new("cases") + .join("in") + .join("two_chunks_missing_columns.sql"); let mut runner = Runner::new(); - runner - .run(input_path) - .await - .expect("test failed"); - runner - .flush() - .expect("flush worked"); -} \ No newline at end of file + runner.run(input_path).await.expect("test failed"); + runner.flush().expect("flush worked"); +} diff --git a/query_tests/src/influxrpc/field_columns.rs b/query_tests/src/influxrpc/field_columns.rs index eecb583e6b..8d7339dafc 100644 --- a/query_tests/src/influxrpc/field_columns.rs +++ b/query_tests/src/influxrpc/field_columns.rs @@ -56,8 +56,6 @@ async fn test_field_columns_no_predicate() { run_field_columns_test_case(TwoMeasurementsManyFields {}, predicate, expected_fields).await; } -// NGA todo: add delete tests when the TwoMeasurementsManyFieldsWithDelete available - #[tokio::test] async fn test_field_columns_with_pred() { // get only fields from h20 (but both chunks) @@ -201,86 +199,6 @@ async fn test_field_name_plan() { run_field_columns_test_case(OneMeasurementManyFields {}, predicate, expected_fields).await; } -#[tokio::test] -async fn test_field_name_plan_with_delete() { - test_helpers::maybe_start_logging(); - - let predicate = Predicate::default().with_range(0, 2000); - let predicate = InfluxRpcPredicate::new(None, predicate); - - let expected_fields = FieldList { - fields: vec![ - Field { - name: "field1".into(), - data_type: DataType::Float64, - last_timestamp: 100, - }, - Field { - name: "field2".into(), - data_type: DataType::Utf8, - last_timestamp: 100, - }, - Field { - name: "field3".into(), - data_type: DataType::Float64, - last_timestamp: 100, - }, - ], - }; - - run_field_columns_test_case( - OneMeasurementManyFieldsWithDelete {}, - predicate, - expected_fields, - ) - .await; -} - -#[tokio::test] -async fn test_field_name_plan_with_delete_all_time() { - test_helpers::maybe_start_logging(); - - let predicate = Predicate::default(); - let predicate = InfluxRpcPredicate::new(None, predicate); - - let expected_fields = FieldList { - fields: vec![ - Field { - name: "field1".into(), - data_type: DataType::Float64, - last_timestamp: 0, // all time queries are optimized but do not return timestamps - }, - Field { - name: "field2".into(), - data_type: DataType::Utf8, - last_timestamp: 0, - }, - Field { - name: "field3".into(), - data_type: DataType::Float64, - last_timestamp: 0, - }, - Field { - name: "field4".into(), - data_type: DataType::Boolean, - last_timestamp: 0, - }, - Field { - name: "field5".into(), - data_type: DataType::Boolean, - last_timestamp: 0, - }, - ], - }; - - run_field_columns_test_case( - OneMeasurementManyFieldsWithDelete {}, - predicate, - expected_fields, - ) - .await; -} - #[tokio::test] async fn list_field_columns_all_time() { let predicate = Predicate::default().with_range(MIN_NANO_TIME, MAX_NANO_TIME); diff --git a/query_tests/src/influxrpc/read_filter.rs b/query_tests/src/influxrpc/read_filter.rs index 71cf9495c0..7f32084f46 100644 --- a/query_tests/src/influxrpc/read_filter.rs +++ b/query_tests/src/influxrpc/read_filter.rs @@ -4,15 +4,13 @@ use std::sync::Arc; #[cfg(test)] use crate::scenarios::{ DbScenario, DbSetup, EndToEndTest, TwoMeasurements, TwoMeasurementsManyFields, - TwoMeasurementsWithDelete, TwoMeasurementsWithDeleteAll, }; use crate::{ db::AbstractDb, influxrpc::util::run_series_set_plan_maybe_error, scenarios::{ MeasurementStatusCode, MeasurementsForDefect2845, MeasurementsSortableTags, - MeasurementsSortableTagsWithDelete, TwoMeasurementsMultiSeries, - TwoMeasurementsMultiSeriesWithDelete, TwoMeasurementsMultiSeriesWithDeleteAll, + TwoMeasurementsMultiSeries, }, }; use datafusion::{ @@ -220,39 +218,6 @@ async fn test_read_filter_unknown_column_in_predicate() { run_read_filter_test_case(TwoMeasurements {}, predicate, expected_results).await; } -#[tokio::test] -async fn test_read_filter_data_no_pred_with_delete() { - let expected_results = vec![ - "Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [100], values: [70.4]", - "Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [350], values: [90.0]", - "Series tags={_field=reading, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]", - "Series tags={_field=temp, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]", - ]; - - run_read_filter_test_case( - TwoMeasurementsMultiSeriesWithDelete {}, - InfluxRpcPredicate::default(), - expected_results, - ) - .await; -} - -#[tokio::test] -async fn test_read_filter_data_no_pred_with_delete_all() { - // nothing from h2o table because all rows were deleted - let expected_results = vec![ - "Series tags={_field=reading, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]", - "Series tags={_field=temp, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]", - ]; - - run_read_filter_test_case( - TwoMeasurementsMultiSeriesWithDeleteAll {}, - InfluxRpcPredicate::default(), - expected_results, - ) - .await; -} - #[tokio::test] async fn test_read_filter_data_filter() { // filter out one row in h20 @@ -281,58 +246,6 @@ async fn test_read_filter_data_filter() { run_read_filter_test_case(TwoMeasurementsMultiSeries {}, predicate, expected_results).await; } -#[tokio::test] -async fn test_read_filter_data_filter_with_delete() { - // filter out one row in h20 but the leftover row was deleted to nothing will be returned - let predicate = Predicate::default() - .with_range(200, 300) - .with_expr(col("state").eq(lit("CA"))); // state=CA - - let predicate = InfluxRpcPredicate::new(None, predicate); - - let expected_results = vec![]; - - run_read_filter_test_case( - TwoMeasurementsMultiSeriesWithDelete {}, - predicate, - expected_results.clone(), - ) - .await; - - // Same results via a != predicate. - let predicate = Predicate::default() - .with_range(200, 300) - .with_expr(col("state").not_eq(lit("MA"))); // state=CA - - let predicate = InfluxRpcPredicate::new(None, predicate); - - run_read_filter_test_case( - TwoMeasurementsMultiSeriesWithDelete {}, - predicate, - expected_results, - ) - .await; - - // Use different predicate to have data returned - let predicate = Predicate::default() - .with_range(100, 300) - .with_expr(col("state").eq(lit("MA"))) // state=MA - .with_expr(col("_measurement").eq(lit("h2o"))); - - let predicate = InfluxRpcPredicate::new(None, predicate); - - let expected_results = vec![ - "Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [100], values: [70.4]", - ]; - - run_read_filter_test_case( - TwoMeasurementsMultiSeriesWithDelete {}, - predicate, - expected_results, - ) - .await; -} - #[tokio::test] async fn test_read_filter_data_filter_fields() { // filter out one row in h20 @@ -350,8 +263,6 @@ async fn test_read_filter_data_filter_fields() { run_read_filter_test_case(TwoMeasurementsManyFields {}, predicate, expected_results).await; } -// NGA todo: add delete tests here after we have delete scenarios for 2 chunks for 1 table - #[tokio::test] async fn test_read_filter_data_filter_measurement_pred() { // use an expr on table name to pick just the last row from o2 @@ -378,16 +289,6 @@ async fn test_read_filter_data_pred_refers_to_non_existent_column() { run_read_filter_test_case(TwoMeasurements {}, predicate, expected_results).await; } -#[tokio::test] -async fn test_read_filter_data_pred_refers_to_non_existent_column_with_delete() { - let predicate = Predicate::default().with_expr(col("tag_not_in_h20").eq(lit("foo"))); - let predicate = InfluxRpcPredicate::new(None, predicate); - - let expected_results = vec![] as Vec<&str>; - - run_read_filter_test_case(TwoMeasurementsWithDelete {}, predicate, expected_results).await; -} - #[tokio::test] async fn test_read_filter_data_pred_no_columns() { // predicate with no columns, @@ -402,59 +303,6 @@ async fn test_read_filter_data_pred_no_columns() { run_read_filter_test_case(TwoMeasurements {}, predicate, expected_results).await; } -#[tokio::test] -async fn test_read_filter_data_pred_no_columns_with_delete() { - // predicate with no columns, - let predicate = Predicate::default().with_expr(lit("foo").eq(lit("foo"))); - let predicate = InfluxRpcPredicate::new(None, predicate); - - let expected_results = vec![ - "Series tags={_field=user, _measurement=cpu, region=west}\n FloatPoints timestamps: [100], values: [23.2]", - "Series tags={_field=bytes, _measurement=disk, region=east}\n IntegerPoints timestamps: [200], values: [99]", - ]; - - run_read_filter_test_case(TwoMeasurementsWithDelete {}, predicate, expected_results).await; -} - -#[tokio::test] -async fn test_read_filter_data_pred_no_columns_with_delete_all() { - // predicate with no columns, - let predicate = Predicate::default().with_expr(lit("foo").eq(lit("foo"))); - let predicate = InfluxRpcPredicate::new(None, predicate); - - // Only table disk has no deleted data - let expected_results = vec![ - "Series tags={_field=bytes, _measurement=disk, region=east}\n IntegerPoints timestamps: [200], values: [99]", - ]; - - run_read_filter_test_case(TwoMeasurementsWithDeleteAll {}, predicate, expected_results).await; -} - -#[tokio::test] -async fn test_read_filter_data_pred_refers_to_good_and_non_existent_columns() { - // predicate with both a column that does and does not appear - let predicate = Predicate::default() - .with_expr(col("state").eq(lit("MA"))) - .with_expr(col("tag_not_in_h20").eq(lit("foo"))); - let predicate = InfluxRpcPredicate::new(None, predicate); - - let expected_results = vec![] as Vec<&str>; - - run_read_filter_test_case( - TwoMeasurements {}, - predicate.clone(), - expected_results.clone(), - ) - .await; - run_read_filter_test_case( - TwoMeasurementsWithDelete {}, - predicate.clone(), - expected_results.clone(), - ) - .await; - run_read_filter_test_case(TwoMeasurementsWithDeleteAll {}, predicate, expected_results).await; -} - #[tokio::test] async fn test_read_filter_data_pred_using_regex_match() { let predicate = Predicate::default() @@ -487,50 +335,6 @@ async fn test_read_filter_data_pred_using_regex_match_on_field() { run_read_filter_test_case(TwoMeasurementsManyFields {}, predicate, expected_results).await; } -#[tokio::test] -async fn test_read_filter_data_pred_using_regex_match_with_delete() { - let predicate = Predicate::default() - .with_range(200, 300) - // will match CA state - .with_regex_match_expr("state", "C.*"); - let predicate = InfluxRpcPredicate::new(None, predicate); - - // the selected row was soft deleted - let expected_results = vec![]; - run_read_filter_test_case( - TwoMeasurementsMultiSeriesWithDelete {}, - predicate, - expected_results, - ) - .await; - - // Different predicate to have data returned - let predicate = Predicate::default() - .with_range(200, 400) - // will match CA state - .with_regex_match_expr("state", "C.*"); - let predicate = InfluxRpcPredicate::new(None, predicate); - - let expected_results = vec![ - "Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [350], values: [90.0]", - ]; - run_read_filter_test_case( - TwoMeasurementsMultiSeriesWithDelete {}, - predicate.clone(), - expected_results, - ) - .await; - - // Try same predicate but on delete_all data - let expected_results = vec![]; - run_read_filter_test_case( - TwoMeasurementsMultiSeriesWithDeleteAll {}, - predicate, - expected_results, - ) - .await; -} - #[tokio::test] async fn test_read_filter_data_pred_using_regex_not_match() { let predicate = Predicate::default() @@ -600,45 +404,6 @@ async fn test_read_filter_data_pred_unsupported_in_scan() { run_read_filter_test_case(TwoMeasurementsMultiSeries {}, predicate, expected_results).await; } -#[tokio::test] -async fn test_read_filter_data_pred_unsupported_in_scan_with_delete() { - test_helpers::maybe_start_logging(); - - // These predicates can't be pushed down into chunks, but they can - // be evaluated by the general purpose DataFusion plan - - // (STATE = 'CA') OR (READING > 0) - let predicate = - Predicate::default().with_expr(col("state").eq(lit("CA")).or(col("reading").gt(lit(0)))); - let predicate = InfluxRpcPredicate::new(None, predicate); - - // Note these results include data from both o2 and h2o - let expected_results = vec![ - "Series tags={_field=temp, _measurement=h2o, city=LA, state=CA}\n FloatPoints timestamps: [350], values: [90.0]", - "Series tags={_field=reading, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]", - "Series tags={_field=temp, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]", - ]; - - run_read_filter_test_case( - TwoMeasurementsMultiSeriesWithDelete {}, - predicate.clone(), - expected_results, - ) - .await; - - // With delete all from h2o, no rows from h2p should be returned - let expected_results = vec![ - "Series tags={_field=reading, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]", - "Series tags={_field=temp, _measurement=o2, city=Boston, state=MA}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]", - ]; - run_read_filter_test_case( - TwoMeasurementsMultiSeriesWithDeleteAll {}, - predicate, - expected_results, - ) - .await; -} - #[tokio::test] async fn test_read_filter_data_plan_order() { test_helpers::maybe_start_logging(); @@ -659,25 +424,6 @@ async fn test_read_filter_data_plan_order() { .await; } -#[tokio::test] -async fn test_read_filter_data_plan_order_with_delete() { - test_helpers::maybe_start_logging(); - let expected_results = vec![ - "Series tags={_field=other, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [250], values: [5.0]", - "Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA}\n FloatPoints timestamps: [250], values: [70.5]", - "Series tags={_field=temp, _measurement=h2o, city=Boston, state=MA, zz_tag=A}\n FloatPoints timestamps: [1000], values: [70.4]", - "Series tags={_field=temp, _measurement=h2o, city=Kingston, state=MA, zz_tag=A}\n FloatPoints timestamps: [800], values: [70.1]", - "Series tags={_field=temp, _measurement=h2o, city=Kingston, state=MA, zz_tag=B}\n FloatPoints timestamps: [100], values: [70.2]", - ]; - - run_read_filter_test_case( - MeasurementsSortableTagsWithDelete {}, - InfluxRpcPredicate::default(), - expected_results, - ) - .await; -} - #[tokio::test] async fn test_read_filter_filter_on_value() { test_helpers::maybe_start_logging(); diff --git a/query_tests/src/influxrpc/read_group.rs b/query_tests/src/influxrpc/read_group.rs index 25a0be0732..8867710b65 100644 --- a/query_tests/src/influxrpc/read_group.rs +++ b/query_tests/src/influxrpc/read_group.rs @@ -5,7 +5,6 @@ use crate::{ AnotherMeasurementForAggs, DbScenario, DbSetup, MeasurementForDefect2691, MeasurementForGroupByField, MeasurementForGroupKeys, MeasurementForMax, MeasurementForMin, MeasurementForSelectors, OneMeasurementForAggs, OneMeasurementNoTags2, - OneMeasurementNoTagsWithDelete, OneMeasurementNoTagsWithDeleteAllWithAndWithoutChunk, TwoMeasurementForAggs, TwoMeasurementsManyFields, TwoMeasurementsManyFieldsOneChunk, }, }; @@ -93,75 +92,6 @@ async fn test_read_group_data_no_tag_columns() { .await; } -#[tokio::test] -async fn test_read_group_data_no_tag_columns_count_with_delete() { - let agg = Aggregate::Count; - let group_columns = vec![]; - let expected_results = vec![ - "Group tag_keys: _field, _measurement partition_key_vals: ", - "Series tags={_field=foo, _measurement=m0}\n IntegerPoints timestamps: [2], values: [1]", - ]; - run_read_group_test_case( - OneMeasurementNoTagsWithDelete {}, - InfluxRpcPredicate::default(), - agg, - group_columns.clone(), - expected_results, - ) - .await; -} - -#[tokio::test] -async fn test_read_group_data_no_tag_columns_min_with_delete() { - let agg = Aggregate::Min; - let group_columns = vec![]; - let expected_results = vec![ - "Group tag_keys: _field, _measurement partition_key_vals: ", - "Series tags={_field=foo, _measurement=m0}\n FloatPoints timestamps: [2], values: [2.0]", - ]; - - run_read_group_test_case( - OneMeasurementNoTagsWithDelete {}, - InfluxRpcPredicate::default(), - agg, - group_columns.clone(), - expected_results, - ) - .await; -} - -#[tokio::test] -async fn test_read_group_data_no_tag_columns_count_with_delete_all() { - let agg = Aggregate::Count; - let group_columns = vec![]; - let expected_results = vec![]; - - run_read_group_test_case( - OneMeasurementNoTagsWithDeleteAllWithAndWithoutChunk {}, - InfluxRpcPredicate::default(), - agg, - group_columns.clone(), - expected_results, - ) - .await; -} - -#[tokio::test] -async fn test_read_group_data_no_tag_columns_min_with_delete_all() { - let agg = Aggregate::Min; - let group_columns = vec![]; - let expected_results = vec![]; - - run_read_group_test_case( - OneMeasurementNoTagsWithDeleteAllWithAndWithoutChunk {}, - InfluxRpcPredicate::default(), - agg, - group_columns, - expected_results, - ) - .await; -} - #[tokio::test] async fn test_read_group_data_pred() { let predicate = Predicate::default() diff --git a/query_tests/src/influxrpc/read_window_aggregate.rs b/query_tests/src/influxrpc/read_window_aggregate.rs index d1547dd6d2..1c3d1b44a0 100644 --- a/query_tests/src/influxrpc/read_window_aggregate.rs +++ b/query_tests/src/influxrpc/read_window_aggregate.rs @@ -170,47 +170,6 @@ async fn test_grouped_series_set_plan_group_aggregate_min_defect_2697() { .await; } -#[tokio::test] -async fn test_grouped_series_set_plan_group_aggregate_min_defect_2697_with_delete() { - let predicate = Predicate::default() - // time >= '2021-01-01T00:00:01.000000001Z' AND time <= '2021-01-01T00:00:01.000000031Z' - .with_range(1609459201000000001, 1609459201000000031); - let predicate = InfluxRpcPredicate::new(None, predicate); - - let agg = Aggregate::Min; - let every = WindowDuration::from_nanoseconds(10); - let offset = WindowDuration::from_nanoseconds(0); - - // one row deleted - let expected_results = vec![ - "Series tags={_field=bar, _measurement=mm, section=1a}\n FloatPoints timestamps: [1609459201000000011], values: [5.0]", - "Series tags={_field=foo, _measurement=mm, section=1a}\n FloatPoints timestamps: [1609459201000000001, 1609459201000000024], values: [1.0, 11.24]", - "Series tags={_field=bar, _measurement=mm, section=2b}\n FloatPoints timestamps: [1609459201000000009, 1609459201000000015], values: [4.0, 6.0]", - "Series tags={_field=foo, _measurement=mm, section=2b}\n FloatPoints timestamps: [1609459201000000002], values: [2.0]", - ]; - run_read_window_aggregate_test_case( - MeasurementForDefect2697WithDelete {}, - predicate.clone(), - agg, - every, - offset, - expected_results, - ) - .await; - - // all rows deleted - let expected_results = vec![]; - run_read_window_aggregate_test_case( - MeasurementForDefect2697WithDeleteAll {}, - predicate, - agg, - every, - offset, - expected_results, - ) - .await; -} - // See https://github.com/influxdata/influxdb_iox/issues/2697 #[tokio::test] async fn test_grouped_series_set_plan_group_aggregate_sum_defect_2697() { @@ -276,50 +235,6 @@ async fn test_grouped_series_set_plan_group_aggregate_filter_on_field() { .await; } -#[tokio::test] -async fn test_grouped_series_set_plan_group_aggregate_sum_defect_2697_with_delete() { - let predicate = Predicate::default() - // time >= '2021-01-01T00:00:01.000000001Z' AND time <= '2021-01-01T00:00:01.000000031Z' - .with_range(1609459201000000001, 1609459201000000031); - let predicate = InfluxRpcPredicate::new(None, predicate); - - let agg = Aggregate::Sum; - let every = WindowDuration::from_nanoseconds(10); - let offset = WindowDuration::from_nanoseconds(0); - - // one row deleted - - // The windowed aggregate is using a non-selector aggregate (SUM, COUNT, MEAD). - // For each distinct series the window defines the `time` column - let expected_results = vec![ - "Series tags={_field=bar, _measurement=mm, section=1a}\n FloatPoints timestamps: [1609459201000000020], values: [5.0]", - "Series tags={_field=foo, _measurement=mm, section=1a}\n FloatPoints timestamps: [1609459201000000010, 1609459201000000030], values: [4.0, 11.24]", - "Series tags={_field=bar, _measurement=mm, section=2b}\n FloatPoints timestamps: [1609459201000000010, 1609459201000000020], values: [4.0, 6.0]", - "Series tags={_field=foo, _measurement=mm, section=2b}\n FloatPoints timestamps: [1609459201000000010], values: [2.0]", - ]; - run_read_window_aggregate_test_case( - MeasurementForDefect2697WithDelete {}, - predicate.clone(), - agg, - every, - offset, - expected_results, - ) - .await; - - // all rows deleted - let expected_results = vec![]; - run_read_window_aggregate_test_case( - MeasurementForDefect2697WithDeleteAll {}, - predicate, - agg, - every, - offset, - expected_results, - ) - .await; -} - #[tokio::test] async fn test_read_window_aggregate_overflow() { let predicate = Predicate::default().with_range(1609459201000000001, 1609459201000000024); diff --git a/query_tests/src/influxrpc/table_names.rs b/query_tests/src/influxrpc/table_names.rs index e18710d099..c7f23c3cd1 100644 --- a/query_tests/src/influxrpc/table_names.rs +++ b/query_tests/src/influxrpc/table_names.rs @@ -100,106 +100,31 @@ async fn list_table_names_no_non_null_general_data_passes() { run_table_names_test_case(TwoMeasurementsManyFields {}, predicate, vec![]).await; } -#[tokio::test] -async fn list_table_names_no_data_pred_with_delete() { - run_table_names_test_case( - TwoMeasurementsWithDelete {}, - InfluxRpcPredicate::default(), - vec!["cpu", "disk"], - ) - .await; -} - -#[tokio::test] -async fn list_table_names_no_data_pred_with_delete_all() { - run_table_names_test_case( - TwoMeasurementsWithDeleteAll {}, - InfluxRpcPredicate::default(), - vec!["disk"], - ) - .await; -} - #[tokio::test] async fn list_table_names_data_pred_0_201() { run_table_names_test_case(TwoMeasurements {}, tsp(0, 201), vec!["cpu", "disk"]).await; } -#[tokio::test] -async fn list_table_names_data_pred_0_201_with_delete() { - run_table_names_test_case( - TwoMeasurementsWithDelete {}, - tsp(0, 201), - vec!["cpu", "disk"], - ) - .await; -} - -#[tokio::test] -async fn list_table_names_data_pred_0_201_with_delete_all() { - run_table_names_test_case(TwoMeasurementsWithDeleteAll {}, tsp(0, 201), vec!["disk"]).await; -} - #[tokio::test] async fn list_table_names_data_pred_0_200() { run_table_names_test_case(TwoMeasurements {}, tsp(0, 200), vec!["cpu"]).await; } -#[tokio::test] -async fn list_table_names_data_pred_0_200_with_delete() { - run_table_names_test_case(TwoMeasurementsWithDelete {}, tsp(0, 200), vec!["cpu"]).await; -} - -#[tokio::test] -async fn list_table_names_data_pred_0_200_with_delete_all() { - run_table_names_test_case(TwoMeasurementsWithDeleteAll {}, tsp(0, 200), vec![]).await; -} - #[tokio::test] async fn list_table_names_data_pred_50_101() { run_table_names_test_case(TwoMeasurements {}, tsp(50, 101), vec!["cpu"]).await; } -#[tokio::test] -async fn list_table_names_data_pred_50_101_with_delete() { - run_table_names_test_case(TwoMeasurementsWithDelete {}, tsp(50, 101), vec!["cpu"]).await; -} - -#[tokio::test] -async fn list_table_names_data_pred_50_101_with_delete_all() { - run_table_names_test_case(TwoMeasurementsWithDeleteAll {}, tsp(50, 101), vec![]).await; -} - #[tokio::test] async fn list_table_names_data_pred_101_160() { run_table_names_test_case(TwoMeasurements {}, tsp(101, 160), vec!["cpu"]).await; } -#[tokio::test] -async fn list_table_names_data_pred_101_160_with_delete() { - run_table_names_test_case(TwoMeasurementsWithDelete {}, tsp(101, 160), vec![]).await; -} - -#[tokio::test] -async fn list_table_names_data_pred_101_160_with_delete_all() { - run_table_names_test_case(TwoMeasurementsWithDeleteAll {}, tsp(101, 160), vec![]).await; -} - #[tokio::test] async fn list_table_names_data_pred_250_300() { run_table_names_test_case(TwoMeasurements {}, tsp(250, 300), vec![]).await; } -#[tokio::test] -async fn list_table_names_data_pred_250_300_with_delete() { - run_table_names_test_case(TwoMeasurementsWithDelete {}, tsp(250, 300), vec![]).await; -} - -#[tokio::test] -async fn list_table_names_data_pred_250_300_with_delete_all() { - run_table_names_test_case(TwoMeasurementsWithDeleteAll {}, tsp(250, 300), vec![]).await; -} - #[tokio::test] async fn list_table_names_max_time_included() { run_table_names_test_case( diff --git a/query_tests/src/influxrpc/tag_keys.rs b/query_tests/src/influxrpc/tag_keys.rs index da21ca52d7..a15672fde0 100644 --- a/query_tests/src/influxrpc/tag_keys.rs +++ b/query_tests/src/influxrpc/tag_keys.rs @@ -169,24 +169,6 @@ async fn list_tag_name_end_to_end() { run_tag_keys_test_case(EndToEndTest {}, predicate, expected_tag_keys).await; } -#[tokio::test] -async fn list_tag_name_end_to_end_with_delete_and_pred() { - let predicate = Predicate::default() - .with_range(0, 10000) - .with_expr(col("host").eq(lit("server01"))); - let predicate = InfluxRpcPredicate::new(None, predicate); - let expected_tag_keys = vec!["host", "region"]; - run_tag_keys_test_case(EndToEndTestWithDelete {}, predicate, expected_tag_keys).await; -} - -#[tokio::test] -async fn list_tag_name_end_to_end_with_delete() { - let predicate = Predicate::default().with_expr(col("_measurement").eq(lit("swap"))); - let predicate = InfluxRpcPredicate::new(None, predicate); - let expected_tag_keys = vec!["host", "name"]; - run_tag_keys_test_case(EndToEndTestWithDelete {}, predicate, expected_tag_keys).await; -} - #[tokio::test] async fn list_tag_name_max_time() { test_helpers::maybe_start_logging(); diff --git a/query_tests/src/influxrpc/tag_values.rs b/query_tests/src/influxrpc/tag_values.rs index 7a99ab59e7..0e9e2c532a 100644 --- a/query_tests/src/influxrpc/tag_values.rs +++ b/query_tests/src/influxrpc/tag_values.rs @@ -80,32 +80,6 @@ async fn list_tag_values_no_predicate_state_col() { .await; } -#[tokio::test] -async fn list_tag_values_no_predicate_state_col_with_delete() { - let tag_name = "state"; - let expected_tag_keys = vec!["CA", "MA"]; - run_tag_values_test_case( - OneMeasurementManyNullTagsWithDelete {}, - tag_name, - InfluxRpcPredicate::default(), - expected_tag_keys, - ) - .await; -} - -#[tokio::test] -async fn list_tag_values_no_predicate_state_col_with_delete_all() { - let tag_name = "state"; - let expected_tag_keys = vec![]; - run_tag_values_test_case( - OneMeasurementManyNullTagsWithDeleteAll {}, - tag_name, - InfluxRpcPredicate::default(), - expected_tag_keys, - ) - .await; -} - #[tokio::test] async fn list_tag_values_no_predicate_city_col() { let tag_name = "city"; diff --git a/query_tests/src/scenarios/library.rs b/query_tests/src/scenarios/library.rs index 5e52a1ba9c..f3fb22c756 100644 --- a/query_tests/src/scenarios/library.rs +++ b/query_tests/src/scenarios/library.rs @@ -6,7 +6,6 @@ use super::{ }; use crate::scenarios::util::{make_n_chunks_scenario, ChunkData}; use async_trait::async_trait; -use data_types::{DeleteExpr, DeletePredicate, Op, Scalar, TimestampRange}; use iox_query::frontend::sql::SqlQueryPlanner; #[derive(Debug)] @@ -83,82 +82,6 @@ impl DbSetup for OneMeasurementManyNullTags { } } -#[derive(Debug)] -pub struct OneMeasurementManyNullTagsWithDelete {} -#[async_trait] -impl DbSetup for OneMeasurementManyNullTagsWithDelete { - async fn make(&self) -> Vec { - let partition_key = "1970-01-01T00"; - - let lp_lines = vec![ - "h2o,state=CA,city=LA,county=LA temp=70.4 100", - "h2o,state=MA,city=Boston,county=Suffolk temp=72.4 250", - "h2o,state=MA,city=Boston temp=50.4 200", - "h2o,state=CA temp=79.0 300", - "h2o,state=NY temp=60.8 400", - "h2o,state=NY,city=NYC temp=61.0 500", - "h2o,state=NY,city=NYC,borough=Brooklyn temp=61.0 600", - ]; - - // pred: delete from h2o where 400 <= time <= 602 and state=NY - // 3 rows of h2o & NY state will be deleted - let delete_table_name = "h2o"; - let pred = DeletePredicate { - range: TimestampRange::new(400, 602), - exprs: vec![DeleteExpr::new( - "state".to_string(), - Op::Eq, - Scalar::String(("NY").to_string()), - )], - }; - - all_scenarios_for_one_chunk( - vec![&pred], - vec![], - lp_lines, - delete_table_name, - partition_key, - ) - .await - } -} - -#[derive(Debug)] -pub struct OneMeasurementManyNullTagsWithDeleteAll {} -#[async_trait] -impl DbSetup for OneMeasurementManyNullTagsWithDeleteAll { - async fn make(&self) -> Vec { - let partition_key = "1970-01-01T00"; - - let lp_lines = vec![ - "h2o,state=CA,city=LA,county=LA temp=70.4 100", - "h2o,state=MA,city=Boston,county=Suffolk temp=72.4 250", - "h2o,state=MA,city=Boston temp=50.4 200", - "h2o,state=CA temp=79.0 300", - "h2o,state=NY temp=60.8 400", - "h2o,state=NY,city=NYC temp=61.0 500", - "h2o,state=NY,city=NYC,borough=Brooklyn temp=61.0 600", - ]; - - // pred: delete from h2o where 100 <= time <= 602 - // all rows of h2o will be deleted - let delete_table_name = "h2o"; - let pred = DeletePredicate { - range: TimestampRange::new(100, 602), - exprs: vec![], - }; - - all_scenarios_for_one_chunk( - vec![&pred], - vec![], - lp_lines, - delete_table_name, - partition_key, - ) - .await - } -} - /// Two measurements data in different chunk scenarios #[derive(Debug)] pub struct TwoMeasurements {} @@ -177,85 +100,6 @@ impl DbSetup for TwoMeasurements { } } -/// Two measurements data in different chunk scenarios -/// with one delete applied at different stages of the chunk -#[derive(Debug)] -pub struct TwoMeasurementsWithDelete {} -#[async_trait] -impl DbSetup for TwoMeasurementsWithDelete { - async fn make(&self) -> Vec { - let partition_key = "1970-01-01T00"; - - let lp_lines = vec![ - "cpu,region=west user=23.2 100", - "cpu,region=west user=21.0 150", - "disk,region=east bytes=99i 200", - ]; - - // pred: delete from cpu where 120 <= time <= 160 and region="west" - // delete 1 row from cpu with timestamp 150 - let table_name = "cpu"; - let pred = DeletePredicate { - range: TimestampRange::new(120, 160), - exprs: vec![DeleteExpr::new( - "region".to_string(), - Op::Eq, - Scalar::String("west".to_string()), - )], - }; - - // return all possible combination scenarios of a chunk stage and when the delete - // predicates are applied - all_scenarios_for_one_chunk(vec![&pred], vec![], lp_lines, table_name, partition_key).await - } -} - -/// Two measurements data in different chunk scenarios -/// with 2 deletes that remove all data from one table -#[derive(Debug)] -pub struct TwoMeasurementsWithDeleteAll {} -#[async_trait] -impl DbSetup for TwoMeasurementsWithDeleteAll { - async fn make(&self) -> Vec { - let partition_key = "1970-01-01T00"; - - let lp_lines = vec![ - "cpu,region=west user=23.2 100", - "cpu,region=west user=21.0 150", - "disk,region=east bytes=99i 200", - ]; - - // pred: delete from cpu where 120 <= time <= 160 and region="west" - // which will delete second row of the cpu - let table_name = "cpu"; - let pred1 = DeletePredicate { - range: TimestampRange::new(120, 160), - exprs: vec![DeleteExpr::new( - "region".to_string(), - Op::Eq, - Scalar::String("west".to_string()), - )], - }; - - // delete the first row of the cpu - let pred2 = DeletePredicate { - range: TimestampRange::new(0, 110), - exprs: vec![], - }; - - // return all possible combination scenarios of a chunk stage and when the delete - // predicates are applied - all_scenarios_for_one_chunk( - vec![&pred1], - vec![&pred2], - lp_lines, - table_name, - partition_key, - ) - .await - } -} - #[derive(Debug)] pub struct TwoMeasurementsUnsignedType {} #[async_trait] @@ -710,44 +554,6 @@ impl DbSetup for OneMeasurementManyFields { all_scenarios_for_one_chunk(vec![], vec![], lp_lines, "h2o", partition_key).await } } - -#[derive(Debug)] -pub struct OneMeasurementManyFieldsWithDelete {} -#[async_trait] -impl DbSetup for OneMeasurementManyFieldsWithDelete { - async fn make(&self) -> Vec { - let partition_key = "1970-01-01T00"; - - // Order this so field3 comes before field2 - // (and thus the columns need to get reordered) - let lp_lines = vec![ - "h2o,tag1=foo,tag2=bar field1=70.6,field3=2 100", - "h2o,tag1=foo,tag2=bar field1=70.4,field2=\"ss\" 100", - "h2o,tag1=foo,tag2=bar field1=70.5,field2=\"ss\" 100", - "h2o,tag1=foo,tag2=bar field1=70.6,field4=true 1000", - "h2o,tag1=foo,tag2=bar field1=70.3,field5=false 3000", - ]; - - // pred: delete from h2o where 1000 <= time <= 1100 - // 1 rows of h2o with timestamp 1000 will be deleted which means - // field4 no longer available - let delete_table_name = "h2o"; - let pred = DeletePredicate { - range: TimestampRange::new(1000, 1100), - exprs: vec![], - }; - - all_scenarios_for_one_chunk( - vec![&pred], - vec![], - lp_lines, - delete_table_name, - partition_key, - ) - .await - } -} - /// This data (from end to end test) #[derive(Debug)] pub struct EndToEndTest {} @@ -772,48 +578,6 @@ impl DbSetup for EndToEndTest { } } -#[derive(Debug)] -pub struct EndToEndTestWithDelete {} -#[async_trait] -impl DbSetup for EndToEndTestWithDelete { - async fn make(&self) -> Vec { - let lp_lines = vec![ - "cpu_load_short,host=server01,region=us-west value=0.64 0000", - "cpu_load_short,host=server01 value=27.99 1000", - "cpu_load_short,host=server02,region=us-west value=3.89 2000", - "cpu_load_short,host=server01,region=us-east value=1234567.891011 3000", - "cpu_load_short,host=server01,region=us-west value=0.000003 4000", - "system,host=server03 uptime=1303385 5000", - "swap,host=server01,name=disk0 in=3,out=4 6000", - "status active=t 7000", - "attributes color=\"blue\" 8000", - ]; - - let partition_key = "1970-01-01T00"; - - // pred: delete from swap where 6000 <= time <= 6000 and name=disk0 - // 1 rows of swap with name=disk0 will be deleted - let delete_table_name = "swap"; - let pred = DeletePredicate { - range: TimestampRange::new(6000, 6000), - exprs: vec![DeleteExpr::new( - "name".to_string(), - Op::Eq, - Scalar::String(("disk0").to_string()), - )], - }; - - all_scenarios_for_one_chunk( - vec![&pred], - vec![], - lp_lines, - delete_table_name, - partition_key, - ) - .await - } -} - #[derive(Debug)] pub struct TwoMeasurementsMultiSeries {} #[async_trait] @@ -838,84 +602,6 @@ impl DbSetup for TwoMeasurementsMultiSeries { } } -#[derive(Debug)] -pub struct TwoMeasurementsMultiSeriesWithDelete {} -#[async_trait] -impl DbSetup for TwoMeasurementsMultiSeriesWithDelete { - async fn make(&self) -> Vec { - let partition_key = "1970-01-01T00"; - - let mut lp_lines = vec![ - "h2o,state=MA,city=Boston temp=70.4 100", // to row 2 - "h2o,state=MA,city=Boston temp=72.4 250", // to row 1 - "h2o,state=CA,city=LA temp=90.0 200", // to row 0 - "h2o,state=CA,city=LA temp=90.0 350", // to row 3 - "o2,state=MA,city=Boston temp=50.4,reading=50 100", // to row 5 - "o2,state=MA,city=Boston temp=53.4,reading=51 250", // to row 4 - ]; - - // Swap around data is not inserted in series order - lp_lines.swap(0, 2); - lp_lines.swap(4, 5); - - // pred: delete from h2o where 120 <= time <= 250 - // 2 rows of h2o with timestamp 200 and 350 will be deleted - let delete_table_name = "h2o"; - let pred = DeletePredicate { - range: TimestampRange::new(120, 250), - exprs: vec![], - }; - - all_scenarios_for_one_chunk( - vec![&pred], - vec![], - lp_lines, - delete_table_name, - partition_key, - ) - .await - } -} - -#[derive(Debug)] -pub struct TwoMeasurementsMultiSeriesWithDeleteAll {} -#[async_trait] -impl DbSetup for TwoMeasurementsMultiSeriesWithDeleteAll { - async fn make(&self) -> Vec { - let partition_key = "1970-01-01T00"; - - let mut lp_lines = vec![ - "h2o,state=MA,city=Boston temp=70.4 100", // to row 2 - "h2o,state=MA,city=Boston temp=72.4 250", // to row 1 - "h2o,state=CA,city=LA temp=90.0 200", // to row 0 - "h2o,state=CA,city=LA temp=90.0 350", // to row 3 - "o2,state=MA,city=Boston temp=50.4,reading=50 100", // to row 5 - "o2,state=MA,city=Boston temp=53.4,reading=51 250", // to row 4 - ]; - - // Swap around data is not inserted in series order - lp_lines.swap(0, 2); - lp_lines.swap(4, 5); - - // Delete all data form h2o - // pred: delete from h20 where 100 <= time <= 360 - let delete_table_name = "h2o"; - let pred = DeletePredicate { - range: TimestampRange::new(100, 360), - exprs: vec![], - }; - - all_scenarios_for_one_chunk( - vec![&pred], - vec![], - lp_lines, - delete_table_name, - partition_key, - ) - .await - } -} - pub struct MeasurementStatusCode {} #[async_trait] impl DbSetup for MeasurementStatusCode { @@ -950,44 +636,6 @@ impl DbSetup for MeasurementsSortableTags { } } -#[derive(Debug)] -pub struct MeasurementsSortableTagsWithDelete {} -#[async_trait] -impl DbSetup for MeasurementsSortableTagsWithDelete { - async fn make(&self) -> Vec { - let partition_key = "1970-01-01T00"; - - let lp_lines = vec![ - "h2o,zz_tag=A,state=MA,city=Kingston temp=70.1 800", - "h2o,state=MA,city=Kingston,zz_tag=B temp=70.2 100", - "h2o,state=CA,city=Boston temp=70.3 250", // soft deleted - "h2o,state=MA,city=Boston,zz_tag=A temp=70.4 1000", - "h2o,state=MA,city=Boston temp=70.5,other=5.0 250", - ]; - - // pred: delete from h2o where 120 <= time <= 350 and state=CA - // 1 rows of h2o with timestamp 250 will be deleted - let delete_table_name = "h2o"; - let pred = DeletePredicate { - range: TimestampRange::new(120, 350), - exprs: vec![DeleteExpr::new( - "state".to_string(), - Op::Eq, - Scalar::String(("CA").to_string()), - )], - }; - - all_scenarios_for_one_chunk( - vec![&pred], - vec![], - lp_lines, - delete_table_name, - partition_key, - ) - .await - } -} - // See issue: https://github.com/influxdata/influxdb_iox/issues/2845 #[derive(Debug)] pub struct MeasurementsForDefect2845 {} @@ -1019,65 +667,6 @@ impl DbSetup for OneMeasurementNoTags2 { } } -pub struct OneMeasurementNoTagsWithDelete {} -#[async_trait] -impl DbSetup for OneMeasurementNoTagsWithDelete { - async fn make(&self) -> Vec { - let partition_key = "1970-01-01T00"; - let lp_lines = vec!["m0 foo=1.0 1", "m0 foo=2.0 2"]; - - // pred: delete from m0 where 1 <= time <= 1 and foo=1.0 - // 1 row of m0 with timestamp 1 - let delete_table_name = "m0"; - let pred = DeletePredicate { - range: TimestampRange::new(1, 1), - exprs: vec![DeleteExpr::new( - "foo".to_string(), - Op::Eq, - Scalar::F64((1.0).into()), - )], - }; - - all_scenarios_for_one_chunk( - vec![&pred], - vec![], - lp_lines, - delete_table_name, - partition_key, - ) - .await - } -} - -/// This will create many scenarios: some have a chunk with soft deleted data, some have no chunks -/// because there is no point to create compacted chunks with all deleted data. -pub struct OneMeasurementNoTagsWithDeleteAllWithAndWithoutChunk {} -#[async_trait] -impl DbSetup for OneMeasurementNoTagsWithDeleteAllWithAndWithoutChunk { - async fn make(&self) -> Vec { - let partition_key = "1970-01-01T00"; - let lp_lines = vec!["m0 foo=1.0 1", "m0 foo=2.0 2"]; - - // pred: delete from m0 where 1 <= time <= 2 - let delete_table_name = "m0"; - let pred = DeletePredicate { - range: TimestampRange::new(1, 2), - exprs: vec![], - }; - - // Apply predicate before the chunk is moved if any. There will be scenarios without chunks - // as a consequence of not-compacting-deleted-data - all_scenarios_for_one_chunk( - vec![&pred], - vec![], - lp_lines, - delete_table_name, - partition_key, - ) - .await - } -} - pub struct OneMeasurementForAggs {} #[async_trait] impl DbSetup for OneMeasurementForAggs { @@ -1310,65 +899,6 @@ impl DbSetup for MeasurementForDefect2697 { } } -pub struct MeasurementForDefect2697WithDelete {} -#[async_trait] -impl DbSetup for MeasurementForDefect2697WithDelete { - async fn make(&self) -> Vec { - let partition_key = "2021-01-01T00"; - - let lp = vec![ - "mm,section=1a bar=5.0 1609459201000000011", - "mm,section=1a bar=0.28 1609459201000000031", - "mm,section=2b bar=4.0 1609459201000000009", - "mm,section=2b bar=6.0 1609459201000000015", - "mm,section=2b bar=1.2 1609459201000000022", - "mm,section=1a foo=1.0 1609459201000000001", - "mm,section=1a foo=3.0 1609459201000000005", - "mm,section=1a foo=11.24 1609459201000000024", - "mm,section=2b foo=2.0 1609459201000000002", - ]; - - // pred: delete from mm where 1609459201000000022 <= time <= 1609459201000000022 - // 1 row of m0 with timestamp 1609459201000000022 (section=2b bar=1.2) - let delete_table_name = "mm"; - let pred = DeletePredicate { - range: TimestampRange::new(1609459201000000022, 1609459201000000022), - exprs: vec![], - }; - - all_scenarios_for_one_chunk(vec![&pred], vec![], lp, delete_table_name, partition_key).await - } -} - -pub struct MeasurementForDefect2697WithDeleteAll {} -#[async_trait] -impl DbSetup for MeasurementForDefect2697WithDeleteAll { - async fn make(&self) -> Vec { - let partition_key = "2021-01-01T00"; - - let lp = vec![ - "mm,section=1a bar=5.0 1609459201000000011", - "mm,section=1a bar=0.28 1609459201000000031", - "mm,section=2b bar=4.0 1609459201000000009", - "mm,section=2b bar=6.0 1609459201000000015", - "mm,section=2b bar=1.2 1609459201000000022", - "mm,section=1a foo=1.0 1609459201000000001", - "mm,section=1a foo=3.0 1609459201000000005", - "mm,section=1a foo=11.24 1609459201000000024", - "mm,section=2b foo=2.0 1609459201000000002", - ]; - - // pred: delete from mm where 1 <= time <= 1609459201000000031 - let delete_table_name = "mm"; - let pred = DeletePredicate { - range: TimestampRange::new(1, 1609459201000000031), - exprs: vec![], - }; - - all_scenarios_for_one_chunk(vec![&pred], vec![], lp, delete_table_name, partition_key).await - } -} - // Test data to validate fix for: // https://github.com/influxdata/influxdb_iox/issues/2890 pub struct MeasurementForDefect2890 {} diff --git a/query_tests/src/scenarios/util.rs b/query_tests/src/scenarios/util.rs index 6b2249dc20..293280c066 100644 --- a/query_tests/src/scenarios/util.rs +++ b/query_tests/src/scenarios/util.rs @@ -1048,9 +1048,6 @@ impl QueryDataAdapter { parquet_max_sequence_number: status .parquet_max_sequence_number .map(|x| x.get()), - tombstone_max_sequence_number: status - .tombstone_max_sequence_number - .map(|x| x.get()), }), }, ),