Merge pull request #5831 from influxdata/dom/remove-tombstones
refactor(ingester): remove tombstone support and delete tests from `query_tests`pull/24376/head
commit
7c9a26849b
|
@ -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
|
||||
|
|
|
@ -52,7 +52,6 @@ async fn ingester_flight_api() {
|
|||
partition_id,
|
||||
status: Some(PartitionStatus {
|
||||
parquet_max_sequence_number: None,
|
||||
tombstone_max_sequence_number: None
|
||||
})
|
||||
},
|
||||
);
|
||||
|
|
|
@ -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 |",
|
||||
"+-----------+------+------+--------------------------------+",
|
||||
];
|
||||
|
||||
|
|
|
@ -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,
|
||||
},
|
||||
}),
|
||||
];
|
||||
|
|
|
@ -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<dyn Catalog>,
|
||||
lifecycle_handle: &dyn LifecycleHandle,
|
||||
executor: &Executor,
|
||||
) -> Result<bool, super::Error> {
|
||||
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<dyn Catalog>,
|
||||
) -> Result<Arc<tokio::sync::RwLock<TableData>>, 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<dyn Catalog> =
|
||||
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");
|
||||
|
|
|
@ -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<SequenceNumber>,
|
||||
|
||||
/// Max sequence number for a tombstone
|
||||
pub tombstone_max_sequence_number: Option<SequenceNumber>,
|
||||
}
|
||||
|
||||
/// 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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<BufferBatch>,
|
||||
|
||||
/// 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<Tombstone>,
|
||||
|
||||
/// 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<Tombstone>,
|
||||
) -> Option<QueryableBatch> {
|
||||
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()
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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<T, E = Error> = std::result::Result<T, E>;
|
||||
|
||||
/// 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<QueryableBatch>,
|
||||
) -> Result<SendableRecordBatchStream> {
|
||||
// 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;
|
||||
}
|
||||
}
|
|
@ -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<dyn Catalog>,
|
||||
lifecycle_handle: &dyn LifecycleHandle,
|
||||
executor: &Executor,
|
||||
) -> Result<bool, super::Error> {
|
||||
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<dyn Catalog> =
|
||||
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");
|
||||
|
|
|
@ -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<SequenceNumber>,
|
||||
|
||||
/// An abstract constructor of [`PartitionData`] instances for a given
|
||||
/// `(key, shard, table)` triplet.
|
||||
partition_provider: Arc<dyn PartitionProvider>,
|
||||
|
@ -117,7 +108,6 @@ impl TableData {
|
|||
table_name: TableName,
|
||||
shard_id: ShardId,
|
||||
namespace_id: NamespaceId,
|
||||
tombstone_max_sequence_number: Option<SequenceNumber>,
|
||||
partition_provider: Arc<dyn PartitionProvider>,
|
||||
) -> 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<SequenceNumber> {
|
||||
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,
|
||||
);
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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<Arc<SnapshotBatch>>,
|
||||
|
||||
/// Delete predicates of the tombstones
|
||||
pub(crate) delete_predicates: Vec<Arc<DeletePredicate>>,
|
||||
|
||||
/// 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<Arc<SnapshotBatch>>,
|
||||
deletes: Vec<Tombstone>,
|
||||
) -> 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<DeletePredicate>] {
|
||||
self.delete_predicates.as_ref()
|
||||
}
|
||||
|
||||
fn timestamp_min_max(&self) -> Option<TimestampMinMax> {
|
||||
// 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<DeletePredicate>] {
|
||||
&[]
|
||||
}
|
||||
}
|
||||
|
||||
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<Arc<RecordBatch>> {
|
||||
// Batch 1: <dict, i64, str, bool, time> & 3 rows
|
||||
let dict_array: ArrayRef = Arc::new(
|
||||
vec![Some("a"), None, Some("b")]
|
||||
.into_iter()
|
||||
.collect::<DictionaryArray<Int32Type>>(),
|
||||
);
|
||||
let int64_array: ArrayRef =
|
||||
Arc::new([Some(-1), None, Some(2)].iter().collect::<Int64Array>());
|
||||
let string_array: ArrayRef = Arc::new(
|
||||
vec![Some("foo"), Some("and"), Some("bar")]
|
||||
.into_iter()
|
||||
.collect::<StringArray>(),
|
||||
);
|
||||
let bool_array: ArrayRef = Arc::new(
|
||||
[Some(true), None, Some(false)]
|
||||
.iter()
|
||||
.collect::<BooleanArray>(),
|
||||
);
|
||||
let ts_array: ArrayRef = Arc::new(
|
||||
[Some(150), Some(200), Some(1526823730000000000)]
|
||||
.iter()
|
||||
.collect::<TimestampNanosecondArray>(),
|
||||
);
|
||||
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: <dict, u64, f64, str, bool, time> & 2 rows
|
||||
let dict_array: ArrayRef = Arc::new(
|
||||
vec![None, Some("d")]
|
||||
.into_iter()
|
||||
.collect::<DictionaryArray<Int32Type>>(),
|
||||
);
|
||||
let uint64_array: ArrayRef = Arc::new([Some(1), Some(2)].iter().collect::<UInt64Array>()); // not null
|
||||
let float64_array: ArrayRef =
|
||||
Arc::new([Some(1.0), Some(2.0)].iter().collect::<Float64Array>());
|
||||
let string_array: ArrayRef = Arc::new(
|
||||
vec![Some("foo"), Some("bar")]
|
||||
.into_iter()
|
||||
.collect::<StringArray>(),
|
||||
);
|
||||
let bool_array: ArrayRef = Arc::new([Some(true), None].iter().collect::<BooleanArray>());
|
||||
let ts_array: ArrayRef = Arc::new(
|
||||
[Some(100), Some(1626823730000000000)] // not null
|
||||
.iter()
|
||||
.collect::<TimestampNanosecondArray>(),
|
||||
);
|
||||
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)]
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
}),
|
||||
},
|
||||
}),
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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::*;
|
||||
|
|
|
@ -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<AtomicI64>,
|
||||
poll_handle: JoinHandle<()>,
|
||||
}
|
||||
|
|
|
@ -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<bool, crate::data::Error>;
|
||||
|
|
|
@ -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!(
|
||||
|
|
|
@ -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<Arc<RecordBatch>>,
|
||||
tombstones: Vec<Tombstone>,
|
||||
) -> Arc<PersistingBatch> {
|
||||
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<RecordBatch>>,
|
||||
) -> Arc<QueryableBatch> {
|
||||
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<Arc<RecordBatch>>,
|
||||
tombstones: Vec<Tombstone>,
|
||||
) -> Arc<QueryableBatch> {
|
||||
// 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<metric::Registry> = Default::default();
|
||||
let catalog: Arc<dyn Catalog> = 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<DmlOperation> {
|
||||
// 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,
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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) |
|
||||
+--------------+
|
||||
| |
|
||||
+--------------+
|
|
@ -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;
|
|
@ -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 |
|
||||
+----+--------------------------------+
|
|
@ -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
|
|
@ -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;
|
||||
++
|
||||
++
|
|
@ -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;
|
||||
|
||||
|
|
@ -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 |
|
||||
+-----+
|
|
@ -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;
|
|
@ -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 |
|
||||
+--------------------------------+--------------------------------+
|
|
@ -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;
|
|
@ -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 |
|
||||
+-----------------+
|
|
@ -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);
|
|
@ -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 |
|
||||
+----+--------------------------------+
|
|
@ -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;
|
|
@ -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) |
|
||||
+---------------+
|
||||
| |
|
||||
+---------------+
|
|
@ -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;
|
|
@ -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");
|
||||
}
|
||||
runner.run(input_path).await.expect("test failed");
|
||||
runner.flush().expect("flush worked");
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -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<DbScenario> {
|
||||
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<DbScenario> {
|
||||
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<DbScenario> {
|
||||
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<DbScenario> {
|
||||
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<DbScenario> {
|
||||
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<DbScenario> {
|
||||
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<DbScenario> {
|
||||
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<DbScenario> {
|
||||
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<DbScenario> {
|
||||
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<DbScenario> {
|
||||
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<DbScenario> {
|
||||
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<DbScenario> {
|
||||
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<DbScenario> {
|
||||
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 {}
|
||||
|
|
|
@ -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()),
|
||||
}),
|
||||
},
|
||||
),
|
||||
|
|
Loading…
Reference in New Issue