refactor: remove table_names from Predicate (#3545)

* refactor: remove table_names from Predicate

* chore: fix benchmarks

* chore: review feedback

Co-authored-by: Edd Robinson <me@edd.io>

* chore: review feedback

* chore: replace Default::default with InfluxRpcPredicate::default()

Co-authored-by: Edd Robinson <me@edd.io>
Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>
pull/24376/head
Raphael Taylor-Davies 2022-01-27 14:44:49 +00:00 committed by GitHub
parent 16d584b2ff
commit 21c1824a7a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
37 changed files with 1346 additions and 1299 deletions

1
Cargo.lock generated
View File

@ -3229,6 +3229,7 @@ dependencies = [
"data_types", "data_types",
"datafusion 0.1.0", "datafusion 0.1.0",
"datafusion_util", "datafusion_util",
"itertools",
"observability_deps", "observability_deps",
"ordered-float 2.10.0", "ordered-float 2.10.0",
"regex", "regex",

View File

@ -86,6 +86,21 @@ impl ChunkStorage {
Self::ObjectStoreOnly => "ObjectStoreOnly", Self::ObjectStoreOnly => "ObjectStoreOnly",
} }
} }
/// Returns true if has data in the mutable buffer
pub fn has_mutable_buffer(&self) -> bool {
matches!(self, Self::OpenMutableBuffer | Self::ClosedMutableBuffer)
}
/// Returns true if has data in the read buffer
pub fn has_read_buffer(&self) -> bool {
matches!(self, Self::ReadBuffer | Self::ReadBufferAndObjectStore)
}
/// Returns true if has data in object storage
pub fn has_object_store(&self) -> bool {
matches!(self, Self::ReadBufferAndObjectStore | Self::ObjectStoreOnly)
}
} }
/// Any lifecycle action currently in progress for this chunk /// Any lifecycle action currently in progress for this chunk

View File

@ -1,11 +1,7 @@
//! This module contains the interface to the Catalog / Chunks used by //! This module contains the interface to the Catalog / Chunks used by
//! the query engine //! the query engine
use super::{ use super::{catalog::Catalog, chunk::DbChunk, query_log::QueryLog};
catalog::{Catalog, TableNameFilter},
chunk::DbChunk,
query_log::QueryLog,
};
use crate::system_tables; use crate::system_tables;
use async_trait::async_trait; use async_trait::async_trait;
use data_types::{chunk_metadata::ChunkSummary, partition_metadata::PartitionAddr}; use data_types::{chunk_metadata::ChunkSummary, partition_metadata::PartitionAddr};
@ -18,7 +14,7 @@ use job_registry::JobRegistry;
use metric::{Attributes, DurationCounter, Metric, U64Counter}; use metric::{Attributes, DurationCounter, Metric, U64Counter};
use observability_deps::tracing::debug; use observability_deps::tracing::debug;
use parking_lot::{MappedMutexGuard, Mutex, MutexGuard}; use parking_lot::{MappedMutexGuard, Mutex, MutexGuard};
use predicate::predicate::{Predicate, PredicateBuilder}; use predicate::predicate::Predicate;
use query::{ use query::{
provider::{ChunkPruner, ProviderBuilder}, provider::{ChunkPruner, ProviderBuilder},
pruning::{prune_chunks, PruningObserver}, pruning::{prune_chunks, PruningObserver},
@ -212,19 +208,18 @@ impl ChunkAccess {
} }
} }
/// Returns all chunks that may have data that passes the /// Returns all chunks from `table_name` that may have data that passes the
/// specified predicates. The chunks are pruned as aggressively as /// specified predicates. The chunks are pruned as aggressively as
/// possible based on metadata. /// possible based on metadata.
fn candidate_chunks(&self, predicate: &Predicate) -> Vec<Arc<DbChunk>> { fn candidate_chunks(&self, table_name: &str, predicate: &Predicate) -> Vec<Arc<DbChunk>> {
let partition_key = predicate.partition_key.as_deref();
let table_names: TableNameFilter<'_> = predicate.table_names.as_ref().into();
let start = Instant::now(); let start = Instant::now();
// Apply initial partition key / table name pruning // Apply initial partition key / table name pruning
let chunks = self let chunks = self.catalog.filtered_chunks(
.catalog Some(table_name),
.filtered_chunks(table_names, partition_key, DbChunk::snapshot); predicate.partition_key.as_deref(),
DbChunk::snapshot,
);
self.access_metrics.catalog_snapshot_count.inc(1); self.access_metrics.catalog_snapshot_count.inc(1);
self.access_metrics self.access_metrics
@ -275,17 +270,12 @@ impl PruningObserver for ChunkAccess {
impl QueryDatabase for QueryCatalogAccess { impl QueryDatabase for QueryCatalogAccess {
type Chunk = DbChunk; type Chunk = DbChunk;
/// Return a covering set of chunks for a particular partition
fn chunks(&self, predicate: &Predicate) -> Vec<Arc<Self::Chunk>> {
self.chunk_access.candidate_chunks(predicate)
}
fn partition_addrs(&self) -> Vec<PartitionAddr> { fn partition_addrs(&self) -> Vec<PartitionAddr> {
self.catalog.partition_addrs() self.catalog.partition_addrs()
} }
fn chunk_summaries(&self) -> Vec<ChunkSummary> { fn table_names(&self) -> Vec<String> {
self.catalog.chunk_summaries() self.catalog.table_names()
} }
fn table_schema(&self, table_name: &str) -> Option<Arc<Schema>> { fn table_schema(&self, table_name: &str) -> Option<Arc<Schema>> {
@ -295,6 +285,15 @@ impl QueryDatabase for QueryCatalogAccess {
.map(|table| Arc::clone(&table.schema().read())) .map(|table| Arc::clone(&table.schema().read()))
} }
/// Return a covering set of chunks for a particular table and predicate
fn chunks(&self, table_name: &str, predicate: &Predicate) -> Vec<Arc<Self::Chunk>> {
self.chunk_access.candidate_chunks(table_name, predicate)
}
fn chunk_summaries(&self) -> Vec<ChunkSummary> {
self.catalog.chunk_summaries()
}
fn record_query( fn record_query(
&self, &self,
query_type: impl Into<String>, query_type: impl Into<String>,
@ -369,9 +368,10 @@ impl SchemaProvider for DbSchemaProvider {
builder = builder =
builder.add_pruner(Arc::clone(&self.chunk_access) as Arc<dyn ChunkPruner<DbChunk>>); builder.add_pruner(Arc::clone(&self.chunk_access) as Arc<dyn ChunkPruner<DbChunk>>);
let predicate = PredicateBuilder::new().table(table_name).build(); for chunk in self
.chunk_access
for chunk in self.chunk_access.candidate_chunks(&predicate) { .candidate_chunks(table_name, &Default::default())
{
builder = builder.add_chunk(chunk); builder = builder.add_chunk(chunk);
} }

View File

@ -1,5 +1,4 @@
//! This module contains the implementation of the InfluxDB IOx Metadata catalog //! This module contains the implementation of the InfluxDB IOx Metadata catalog
use std::collections::BTreeSet;
use std::sync::Arc; use std::sync::Arc;
use data_types::chunk_metadata::ChunkId; use data_types::chunk_metadata::ChunkId;
@ -49,34 +48,6 @@ pub enum Error {
pub type Result<T, E = Error> = std::result::Result<T, E>; pub type Result<T, E = Error> = std::result::Result<T, E>;
/// Specify which tables are to be matched when filtering
/// catalog chunks
#[derive(Debug, Clone, Copy)]
pub enum TableNameFilter<'a> {
/// Include all tables
AllTables,
/// Only include tables that appear in the named set
NamedTables(&'a BTreeSet<String>),
}
impl<'a> From<Option<&'a BTreeSet<String>>> for TableNameFilter<'a> {
/// Creates a [`TableNameFilter`] from an [`Option`].
///
/// If the Option is `None`, all table names will be included in
/// the results.
///
/// If the Option is `Some(set)`, only table names which apear in
/// `set` will be included in the results.
///
/// Note `Some(empty set)` will not match anything
fn from(v: Option<&'a BTreeSet<String>>) -> Self {
match v {
Some(names) => Self::NamedTables(names),
None => Self::AllTables,
}
}
}
/// InfluxDB IOx Metadata Catalog /// InfluxDB IOx Metadata Catalog
/// ///
/// The Catalog stores information such as which chunks exist, what /// The Catalog stores information such as which chunks exist, what
@ -264,16 +235,12 @@ impl Catalog {
} }
pub fn chunk_summaries(&self) -> Vec<ChunkSummary> { pub fn chunk_summaries(&self) -> Vec<ChunkSummary> {
let partition_key = None; self.filtered_chunks(None, None, CatalogChunk::summary)
let table_names = TableNameFilter::AllTables;
self.filtered_chunks(table_names, partition_key, CatalogChunk::summary)
} }
pub fn detailed_chunk_summaries(&self) -> Vec<(Arc<TableSummary>, DetailedChunkSummary)> { pub fn detailed_chunk_summaries(&self) -> Vec<(Arc<TableSummary>, DetailedChunkSummary)> {
let partition_key = None;
let table_names = TableNameFilter::AllTables;
// TODO: Having two summaries with overlapping information seems unfortunate // TODO: Having two summaries with overlapping information seems unfortunate
self.filtered_chunks(table_names, partition_key, |chunk| { self.filtered_chunks(None, None, |chunk| {
(chunk.table_summary(), chunk.detailed_summary()) (chunk.table_summary(), chunk.detailed_summary())
}) })
} }
@ -294,13 +261,11 @@ impl Catalog {
/// Calls `map` with every chunk and returns a collection of the results /// Calls `map` with every chunk and returns a collection of the results
/// ///
/// If `partition_key` is Some(partition_key) only returns chunks /// If `table_name` is `Some` restricts to chunks in that table.
/// from the specified partition. /// If `partition_key` is `Some` restricts to chunks in that partition.
///
/// `table_names` specifies which tables to include
pub fn filtered_chunks<F, C>( pub fn filtered_chunks<F, C>(
&self, &self,
table_names: TableNameFilter<'_>, table_name: Option<&str>,
partition_key: Option<&str>, partition_key: Option<&str>,
map: F, map: F,
) -> Vec<C> ) -> Vec<C>
@ -308,13 +273,9 @@ impl Catalog {
F: Fn(&CatalogChunk) -> C + Copy, F: Fn(&CatalogChunk) -> C + Copy,
{ {
let tables = self.tables.read(); let tables = self.tables.read();
let tables = match table_names { let tables = match table_name {
TableNameFilter::AllTables => itertools::Either::Left(tables.values()), Some(name) => itertools::Either::Right(tables.get(name).into_iter()),
TableNameFilter::NamedTables(named_tables) => itertools::Either::Right( None => itertools::Either::Left(tables.values()),
named_tables
.iter()
.flat_map(|table_name| tables.get(table_name.as_str()).into_iter()),
),
}; };
let partitions = tables.flat_map(|table| match partition_key { let partitions = tables.flat_map(|table| match partition_key {
@ -542,7 +503,6 @@ mod tests {
#[test] #[test]
fn filtered_chunks() { fn filtered_chunks() {
use TableNameFilter::*;
let catalog = Catalog::test(); let catalog = Catalog::test();
let p1 = catalog.get_or_create_partition("table1", "p1"); let p1 = catalog.get_or_create_partition("table1", "p1");
@ -552,13 +512,13 @@ mod tests {
create_open_chunk(&p2); create_open_chunk(&p2);
create_open_chunk(&p3); create_open_chunk(&p3);
let a = catalog.filtered_chunks(AllTables, None, |_| ()); let a = catalog.filtered_chunks(None, None, |_| ());
let b = catalog.filtered_chunks(NamedTables(&make_set("table1")), None, |_| ()); let b = catalog.filtered_chunks(Some("table1"), None, |_| ());
let c = catalog.filtered_chunks(NamedTables(&make_set("table2")), None, |_| ()); let c = catalog.filtered_chunks(Some("table2"), None, |_| ());
let d = catalog.filtered_chunks(NamedTables(&make_set("table2")), Some("p2"), |_| ()); let d = catalog.filtered_chunks(Some("table2"), Some("p2"), |_| ());
assert_eq!(a.len(), 3); assert_eq!(a.len(), 3);
assert_eq!(b.len(), 1); assert_eq!(b.len(), 1);
@ -566,10 +526,6 @@ mod tests {
assert_eq!(d.len(), 1); assert_eq!(d.len(), 1);
} }
fn make_set(s: impl Into<String>) -> BTreeSet<String> {
std::iter::once(s.into()).collect()
}
fn as_sorted<T>(mut v: Vec<T>) -> Vec<T> fn as_sorted<T>(mut v: Vec<T>) -> Vec<T>
where where
T: Ord, T: Ord,

View File

@ -296,10 +296,6 @@ impl QueryChunk for DbChunk {
} }
fn apply_predicate_to_metadata(&self, predicate: &Predicate) -> Result<PredicateMatch> { fn apply_predicate_to_metadata(&self, predicate: &Predicate) -> Result<PredicateMatch> {
if !predicate.should_include_table(self.table_name().as_ref()) {
return Ok(PredicateMatch::Zero);
}
let pred_result = match &self.state { let pred_result = match &self.state {
State::MutableBuffer { chunk, .. } => { State::MutableBuffer { chunk, .. } => {
if predicate.has_exprs() || chunk.has_timerange(&predicate.range) { if predicate.has_exprs() || chunk.has_timerange(&predicate.range) {

View File

@ -9,7 +9,7 @@ use crate::{
chunk::{CatalogChunk, ChunkStage}, chunk::{CatalogChunk, ChunkStage},
partition::Partition, partition::Partition,
table::TableSchemaUpsertHandle, table::TableSchemaUpsertHandle,
Catalog, TableNameFilter, Catalog,
}, },
lifecycle::{LockableCatalogChunk, LockableCatalogPartition}, lifecycle::{LockableCatalogChunk, LockableCatalogPartition},
write::{DeleteFilter, DeleteFilterNone, WriteFilter, WriteFilterNone}, write::{DeleteFilter, DeleteFilterNone, WriteFilter, WriteFilterNone},
@ -832,22 +832,17 @@ impl Db {
LockableChunk::load_read_buffer(chunk.write()).context(LifecycleSnafu) LockableChunk::load_read_buffer(chunk.write()).context(LifecycleSnafu)
} }
/// Return chunk summary information for all chunks in the specified /// Return chunk summary information for all chunks
/// partition across all storage systems ///
pub fn partition_chunk_summaries(&self, partition_key: &str) -> Vec<ChunkSummary> { /// If `table_name` is `Some` restricts to chunks in that table.
self.partition_tables_chunk_summaries(TableNameFilter::AllTables, partition_key) /// If `partition_key` is `Some` restricts to chunks in that partition.
} pub fn filtered_chunk_summaries(
/// Return chunk summary information for all chunks in the specified
/// tables and partition across all storage systems
pub fn partition_tables_chunk_summaries(
&self, &self,
table_name_filter: TableNameFilter<'_>, table_name: Option<&str>,
partition_key: &str, partition_key: Option<&str>,
) -> Vec<ChunkSummary> { ) -> Vec<ChunkSummary> {
let partition_key = Some(partition_key);
self.catalog self.catalog
.filtered_chunks(table_name_filter, partition_key, CatalogChunk::summary) .filtered_chunks(table_name, partition_key, CatalogChunk::summary)
} }
/// Return Summary information for all columns in all chunks in the /// Return Summary information for all columns in all chunks in the
@ -1208,22 +1203,26 @@ impl Db {
impl QueryDatabase for Db { impl QueryDatabase for Db {
type Chunk = DbChunk; type Chunk = DbChunk;
fn chunks(&self, predicate: &Predicate) -> Vec<Arc<Self::Chunk>> {
self.catalog_access.chunks(predicate)
}
fn partition_addrs(&self) -> Vec<PartitionAddr> { fn partition_addrs(&self) -> Vec<PartitionAddr> {
self.catalog_access.partition_addrs() self.catalog_access.partition_addrs()
} }
fn chunk_summaries(&self) -> Vec<ChunkSummary> { fn table_names(&self) -> Vec<String> {
self.catalog_access.chunk_summaries() self.catalog_access.table_names()
} }
fn table_schema(&self, table_name: &str) -> Option<Arc<Schema>> { fn table_schema(&self, table_name: &str) -> Option<Arc<Schema>> {
self.catalog_access.table_schema(table_name) self.catalog_access.table_schema(table_name)
} }
fn chunks(&self, table_name: &str, predicate: &Predicate) -> Vec<Arc<Self::Chunk>> {
self.catalog_access.chunks(table_name, predicate)
}
fn chunk_summaries(&self) -> Vec<ChunkSummary> {
self.catalog_access.chunk_summaries()
}
fn record_query( fn record_query(
&self, &self,
query_type: impl Into<String>, query_type: impl Into<String>,
@ -1308,10 +1307,8 @@ pub(crate) fn checkpoint_data_from_catalog(catalog: &Catalog) -> CheckpointData
pub mod test_helpers { pub mod test_helpers {
use super::*; use super::*;
use arrow::record_batch::RecordBatch; use arrow::record_batch::RecordBatch;
use data_types::chunk_metadata::ChunkStorage;
use mutable_batch_lp::lines_to_batches; use mutable_batch_lp::lines_to_batches;
use query::frontend::sql::SqlQueryPlanner; use query::frontend::sql::SqlQueryPlanner;
use std::collections::BTreeSet;
/// Try to write lineprotocol data and return all tables that where written. /// Try to write lineprotocol data and return all tables that where written.
pub fn try_write_lp(db: &Db, lp: &str) -> Result<Vec<String>, DmlError> { pub fn try_write_lp(db: &Db, lp: &str) -> Result<Vec<String>, DmlError> {
@ -1358,82 +1355,46 @@ pub mod test_helpers {
ctx.collect(physical_plan).await.unwrap() ctx.collect(physical_plan).await.unwrap()
} }
pub fn mutable_chunk_ids(db: &Db, partition_key: &str) -> Vec<ChunkId> { /// Returns the [`ChunkId`] of every chunk containing data in the mutable buffer
mutable_tables_chunk_ids(db, TableNameFilter::AllTables, partition_key) pub fn chunk_ids_mub(
}
pub fn mutable_tables_chunk_ids(
db: &Db, db: &Db,
tables: TableNameFilter<'_>, table_name: Option<&str>,
partition_key: &str, partition_key: Option<&str>,
) -> Vec<ChunkId> { ) -> Vec<ChunkId> {
let mut chunk_ids: Vec<ChunkId> = db let mut chunk_ids: Vec<ChunkId> = db
.partition_tables_chunk_summaries(tables, partition_key) .filtered_chunk_summaries(table_name, partition_key)
.into_iter() .into_iter()
.filter_map(|chunk| match chunk.storage { .filter_map(|chunk| chunk.storage.has_mutable_buffer().then(|| chunk.id))
ChunkStorage::OpenMutableBuffer | ChunkStorage::ClosedMutableBuffer => {
Some(chunk.id)
}
_ => None,
})
.collect(); .collect();
chunk_ids.sort_unstable(); chunk_ids.sort_unstable();
chunk_ids chunk_ids
} }
pub fn read_buffer_chunk_ids(db: &Db, partition_key: &str) -> Vec<ChunkId> { /// Returns the [`ChunkId`] of every chunk containing data in the read buffer
read_buffer_tables_chunk_ids(db, TableNameFilter::AllTables, partition_key) pub fn chunk_ids_rub(
}
pub fn read_buffer_table_chunk_ids(
db: &Db, db: &Db,
table_name: &str, table_name: Option<&str>,
partition_key: &str, partition_key: Option<&str>,
) -> Vec<ChunkId> {
let mut table_names = BTreeSet::new();
table_names.insert(table_name.to_string());
read_buffer_tables_chunk_ids(
db,
TableNameFilter::NamedTables(&table_names),
partition_key,
)
}
pub fn read_buffer_tables_chunk_ids(
db: &Db,
tables: TableNameFilter<'_>,
partition_key: &str,
) -> Vec<ChunkId> { ) -> Vec<ChunkId> {
let mut chunk_ids: Vec<ChunkId> = db let mut chunk_ids: Vec<ChunkId> = db
.partition_tables_chunk_summaries(tables, partition_key) .filtered_chunk_summaries(table_name, partition_key)
.into_iter() .into_iter()
.filter_map(|chunk| match chunk.storage { .filter_map(|chunk| chunk.storage.has_read_buffer().then(|| chunk.id))
ChunkStorage::ReadBuffer => Some(chunk.id),
ChunkStorage::ReadBufferAndObjectStore => Some(chunk.id),
_ => None,
})
.collect(); .collect();
chunk_ids.sort_unstable(); chunk_ids.sort_unstable();
chunk_ids chunk_ids
} }
pub fn parquet_file_chunk_ids(db: &Db, partition_key: &str) -> Vec<ChunkId> { /// Returns the [`ChunkId`] of every chunk containing data in object storage
parquet_file_tables_chunk_ids(db, TableNameFilter::AllTables, partition_key) pub fn chunk_ids_parquet(
}
pub fn parquet_file_tables_chunk_ids(
db: &Db, db: &Db,
tables: TableNameFilter<'_>, table_name: Option<&str>,
partition_key: &str, partition_key: Option<&str>,
) -> Vec<ChunkId> { ) -> Vec<ChunkId> {
let mut chunk_ids: Vec<ChunkId> = db let mut chunk_ids: Vec<ChunkId> = db
.partition_tables_chunk_summaries(tables, partition_key) .filtered_chunk_summaries(table_name, partition_key)
.into_iter() .into_iter()
.filter_map(|chunk| match chunk.storage { .filter_map(|chunk| chunk.storage.has_object_store().then(|| chunk.id))
ChunkStorage::ReadBufferAndObjectStore => Some(chunk.id),
ChunkStorage::ObjectStoreOnly => Some(chunk.id),
_ => None,
})
.collect(); .collect();
chunk_ids.sort_unstable(); chunk_ids.sort_unstable();
chunk_ids chunk_ids
@ -1443,12 +1404,10 @@ pub mod test_helpers {
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use super::*; use super::*;
use crate::test_helpers::{chunk_ids_mub, chunk_ids_parquet, chunk_ids_rub};
use crate::{ use crate::{
catalog::chunk::ChunkStage, catalog::chunk::ChunkStage,
test_helpers::{ test_helpers::{run_query, try_write_lp, write_lp},
mutable_chunk_ids, parquet_file_chunk_ids, read_buffer_chunk_ids, run_query,
try_write_lp, write_lp,
},
utils::{make_db, make_db_time, TestDb}, utils::{make_db, make_db_time, TestDb},
}; };
use ::test_helpers::{assert_contains, assert_error}; use ::test_helpers::{assert_contains, assert_error};
@ -1894,8 +1853,8 @@ mod tests {
assert_ne!(mb_chunk.id(), rb_chunk.id()); assert_ne!(mb_chunk.id(), rb_chunk.id());
// we should have chunks in both the read buffer only // we should have chunks in both the read buffer only
assert!(mutable_chunk_ids(&db, partition_key).is_empty()); assert!(chunk_ids_mub(&db, None, Some(partition_key)).is_empty());
assert_eq!(read_buffer_chunk_ids(&db, partition_key).len(), 1); assert_eq!(chunk_ids_rub(&db, None, Some(partition_key)).len(), 1);
// data should be readable // data should be readable
let expected = vec![ let expected = vec![
@ -1922,7 +1881,7 @@ mod tests {
.await .await
.unwrap(); .unwrap();
assert_eq!( assert_eq!(
read_buffer_chunk_ids(&db, partition_key), chunk_ids_rub(&db, None, Some(partition_key)),
vec![] as Vec<ChunkId> vec![] as Vec<ChunkId>
); );
@ -1978,7 +1937,7 @@ mod tests {
// no other read buffer data should be present // no other read buffer data should be present
assert_eq!( assert_eq!(
read_buffer_chunk_ids(&db, partition_key), chunk_ids_rub(&db, None, Some(partition_key)),
vec![compacted_rb_chunk.id()] vec![compacted_rb_chunk.id()]
); );
assert_ne!(old_rb_chunk.id(), compacted_rb_chunk.id()); assert_ne!(old_rb_chunk.id(), compacted_rb_chunk.id());
@ -2155,9 +2114,9 @@ mod tests {
assert_ne!(mb_chunk.id(), pq_chunk.id()); assert_ne!(mb_chunk.id(), pq_chunk.id());
// we should have chunks in both the read buffer only // we should have chunks in both the read buffer only
assert!(mutable_chunk_ids(&db, partition_key).is_empty()); assert!(chunk_ids_mub(&db, None, Some(partition_key)).is_empty());
assert_eq!(read_buffer_chunk_ids(&db, partition_key).len(), 1); assert_eq!(chunk_ids_rub(&db, None, Some(partition_key)).len(), 1);
assert_eq!(parquet_file_chunk_ids(&db, partition_key).len(), 1); assert_eq!(chunk_ids_parquet(&db, None, Some(partition_key)).len(), 1);
// Verify data written to the parquet file in object store // Verify data written to the parquet file in object store
// //
@ -2251,10 +2210,13 @@ mod tests {
let pq_chunk_id = pq_chunk.id(); let pq_chunk_id = pq_chunk.id();
// we should have chunks in both the read buffer only // we should have chunks in both the read buffer only
assert!(mutable_chunk_ids(&db, partition_key).is_empty()); assert!(chunk_ids_mub(&db, None, Some(partition_key)).is_empty());
assert_eq!(read_buffer_chunk_ids(&db, partition_key), vec![pq_chunk_id]);
assert_eq!( assert_eq!(
parquet_file_chunk_ids(&db, partition_key), chunk_ids_rub(&db, None, Some(partition_key)),
vec![pq_chunk_id]
);
assert_eq!(
chunk_ids_parquet(&db, None, Some(partition_key)),
vec![pq_chunk_id] vec![pq_chunk_id]
); );
@ -2275,10 +2237,10 @@ mod tests {
assert_eq!(pq_chunk_id, pq_chunk.id()); assert_eq!(pq_chunk_id, pq_chunk.id());
// we should only have chunk in os // we should only have chunk in os
assert!(mutable_chunk_ids(&db, partition_key).is_empty()); assert!(chunk_ids_mub(&db, None, Some(partition_key)).is_empty());
assert!(read_buffer_chunk_ids(&db, partition_key).is_empty()); assert!(chunk_ids_rub(&db, None, Some(partition_key)).is_empty());
assert_eq!( assert_eq!(
parquet_file_chunk_ids(&db, partition_key), chunk_ids_parquet(&db, None, Some(partition_key)),
vec![pq_chunk_id] vec![pq_chunk_id]
); );
@ -2466,9 +2428,9 @@ mod tests {
write_lp(&db, "cpu bar=1 10"); write_lp(&db, "cpu bar=1 10");
write_lp(&db, "cpu bar=1 20"); write_lp(&db, "cpu bar=1 20");
assert_eq!(mutable_chunk_ids(&db, partition_key).len(), 1); assert_eq!(chunk_ids_mub(&db, None, Some(partition_key)).len(), 1);
assert_eq!( assert_eq!(
read_buffer_chunk_ids(&db, partition_key), chunk_ids_rub(&db, None, Some(partition_key)),
vec![] as Vec<ChunkId> vec![] as Vec<ChunkId>
); );
@ -2485,8 +2447,8 @@ mod tests {
write_lp(&db, "cpu bar=1 40"); write_lp(&db, "cpu bar=1 40");
assert_eq!(mutable_chunk_ids(&db, partition_key).len(), 2); assert_eq!(chunk_ids_mub(&db, None, Some(partition_key)).len(), 2);
assert_eq!(read_buffer_chunk_ids(&db, partition_key).len(), 1); assert_eq!(chunk_ids_rub(&db, None, Some(partition_key)).len(), 1);
} }
#[tokio::test] #[tokio::test]
@ -2502,7 +2464,7 @@ mod tests {
print!("Partitions: {:?}", db.partition_addrs()); print!("Partitions: {:?}", db.partition_addrs());
let chunk_summaries = db.partition_chunk_summaries("1970-01-05T15"); let chunk_summaries = db.filtered_chunk_summaries(None, Some("1970-01-05T15"));
let expected = vec![ChunkSummary { let expected = vec![ChunkSummary {
partition_key: Arc::from("1970-01-05T15"), partition_key: Arc::from("1970-01-05T15"),
@ -2955,9 +2917,9 @@ mod tests {
.unwrap(); .unwrap();
// we should have chunks in both the read buffer only // we should have chunks in both the read buffer only
assert!(mutable_chunk_ids(&db, partition_key).is_empty()); assert!(chunk_ids_mub(&db, None, Some(partition_key)).is_empty());
assert_eq!(read_buffer_chunk_ids(&db, partition_key).len(), 1); assert_eq!(chunk_ids_rub(&db, None, Some(partition_key)).len(), 1);
assert_eq!(parquet_file_chunk_ids(&db, partition_key).len(), 1); assert_eq!(chunk_ids_parquet(&db, None, Some(partition_key)).len(), 1);
} }
#[tokio::test] #[tokio::test]
@ -3189,7 +3151,7 @@ mod tests {
// ==================== check: DB state ==================== // ==================== check: DB state ====================
// Re-created DB should have an "object store only"-chunk // Re-created DB should have an "object store only"-chunk
assert_eq!(chunks.len(), db.chunks(&Default::default()).len()); assert_eq!(chunks.len(), chunk_ids_parquet(&db, None, None).len());
for (table_name, partition_key, chunk_id) in &chunks { for (table_name, partition_key, chunk_id) in &chunks {
let (chunk, _order) = db.chunk(table_name, partition_key, *chunk_id).unwrap(); let (chunk, _order) = db.chunk(table_name, partition_key, *chunk_id).unwrap();
let chunk = chunk.read(); let chunk = chunk.read();
@ -3526,7 +3488,7 @@ mod tests {
write_lp(db.as_ref(), "cpu bar=1 10"); write_lp(db.as_ref(), "cpu bar=1 10");
let partition_key = "1970-01-01T00"; let partition_key = "1970-01-01T00";
let chunks = db.partition_chunk_summaries(partition_key); let chunks = db.filtered_chunk_summaries(None, Some(partition_key));
assert_eq!(chunks.len(), 1); assert_eq!(chunks.len(), 1);
let chunk_id = chunks[0].id; let chunk_id = chunks[0].id;
@ -3561,7 +3523,10 @@ mod tests {
let partition_key = "1970-01-01T00"; let partition_key = "1970-01-01T00";
// two chunks created // two chunks created
assert_eq!(db.partition_chunk_summaries(partition_key).len(), 2); assert_eq!(
db.filtered_chunk_summaries(None, Some(partition_key)).len(),
2
);
// We don't support dropping unpersisted chunks from a persisted DB because we would forget the write buffer // We don't support dropping unpersisted chunks from a persisted DB because we would forget the write buffer
// progress (partition checkpoints are only created when new parquet files are stored). // progress (partition checkpoints are only created when new parquet files are stored).
@ -3581,7 +3546,10 @@ mod tests {
db.drop_partition("cpu", partition_key).await.unwrap(); db.drop_partition("cpu", partition_key).await.unwrap();
// no chunks left // no chunks left
assert_eq!(db.partition_chunk_summaries(partition_key), vec![]); assert_eq!(
db.filtered_chunk_summaries(None, Some(partition_key)),
vec![]
);
} }
#[tokio::test] #[tokio::test]

View File

@ -1,8 +1,6 @@
use crate::{ use crate::{load::load_or_create_preserved_catalog, DatabaseToCommit, Db};
catalog::TableNameFilter, load::load_or_create_preserved_catalog, DatabaseToCommit, Db,
};
use data_types::{ use data_types::{
chunk_metadata::{ChunkStorage, ChunkSummary}, chunk_metadata::ChunkSummary,
database_rules::{DatabaseRules, LifecycleRules, PartitionTemplate, TemplatePart}, database_rules::{DatabaseRules, LifecycleRules, PartitionTemplate, TemplatePart},
server_id::ServerId, server_id::ServerId,
DatabaseName, DatabaseName,
@ -11,12 +9,9 @@ use iox_object_store::IoxObjectStore;
use job_registry::JobRegistry; use job_registry::JobRegistry;
use object_store::ObjectStore; use object_store::ObjectStore;
use persistence_windows::checkpoint::ReplayPlan; use persistence_windows::checkpoint::ReplayPlan;
use query::exec::Executor;
use query::exec::ExecutorConfig; use query::exec::ExecutorConfig;
use query::{exec::Executor, QueryDatabase}; use std::{borrow::Cow, convert::TryFrom, num::NonZeroU32, sync::Arc, time::Duration};
use std::{
borrow::Cow, collections::BTreeSet, convert::TryFrom, num::NonZeroU32, sync::Arc,
time::Duration,
};
use time::{Time, TimeProvider}; use time::{Time, TimeProvider};
use uuid::Uuid; use uuid::Uuid;
@ -190,100 +185,53 @@ pub async fn make_db_time() -> (Arc<Db>, Arc<time::MockProvider>) {
(db, provider) (db, provider)
} }
fn chunk_summary_iter(db: &Db) -> impl Iterator<Item = ChunkSummary> + '_ { /// Counts the number of chunks passing a predicate
db.partition_addrs() fn count_table_chunks(
db: &Db,
table_name: Option<&str>,
partition_key: Option<&str>,
predicate: impl Fn(&ChunkSummary) -> bool,
) -> usize {
db.filtered_chunk_summaries(table_name, partition_key)
.into_iter() .into_iter()
.flat_map(move |addr| db.partition_chunk_summaries(addr.partition_key.as_ref())) .filter(predicate)
.count()
} }
/// Returns the number of mutable buffer chunks in the specified database /// Returns the number of mutable buffer chunks in the specified database
pub fn count_mutable_buffer_chunks(db: &Db) -> usize { pub fn count_mutable_buffer_chunks(db: &Db) -> usize {
chunk_summary_iter(db) count_table_chunks(db, None, None, |s| s.storage.has_mutable_buffer())
.filter(|s| {
s.storage == ChunkStorage::OpenMutableBuffer
|| s.storage == ChunkStorage::ClosedMutableBuffer
})
.count()
} }
/// return number of MUB chunks of a given table of a partition /// return number of MUB chunks of a given table of a partition
pub fn count_mub_table_chunks(db: &Db, table_name: &str, partition_key: &str) -> usize { pub fn count_mub_table_chunks(db: &Db, table_name: &str, partition_key: &str) -> usize {
let mut table_names = BTreeSet::new(); count_table_chunks(db, Some(table_name), Some(partition_key), |s| {
table_names.insert(table_name.to_string()); s.storage.has_mutable_buffer()
count_mub_tables_chunks( })
db,
TableNameFilter::NamedTables(&table_names),
partition_key,
)
}
pub fn count_mub_tables_chunks(db: &Db, tables: TableNameFilter<'_>, partition_key: &str) -> usize {
db.partition_tables_chunk_summaries(tables, partition_key)
.into_iter()
.filter_map(|chunk| match chunk.storage {
ChunkStorage::OpenMutableBuffer | ChunkStorage::ClosedMutableBuffer => Some(1),
_ => None,
})
.count()
} }
/// Returns the number of read buffer chunks in the specified database /// Returns the number of read buffer chunks in the specified database
pub fn count_read_buffer_chunks(db: &Db) -> usize { pub fn count_read_buffer_chunks(db: &Db) -> usize {
chunk_summary_iter(db) count_table_chunks(db, None, None, |s| s.storage.has_read_buffer())
.filter(|s| {
s.storage == ChunkStorage::ReadBuffer
|| s.storage == ChunkStorage::ReadBufferAndObjectStore
})
.count()
} }
/// return number of RUB chunks of a given table of a partition /// return number of RUB chunks of a given table of a partition
pub fn count_rub_table_chunks(db: &Db, table_name: &str, partition_key: &str) -> usize { pub fn count_rub_table_chunks(db: &Db, table_name: &str, partition_key: &str) -> usize {
let mut table_names = BTreeSet::new(); count_table_chunks(db, Some(table_name), Some(partition_key), |s| {
table_names.insert(table_name.to_string()); s.storage.has_read_buffer()
count_rub_tables_chunks( })
db,
TableNameFilter::NamedTables(&table_names),
partition_key,
)
}
pub fn count_rub_tables_chunks(db: &Db, tables: TableNameFilter<'_>, partition_key: &str) -> usize {
db.partition_tables_chunk_summaries(tables, partition_key)
.into_iter()
.filter_map(|chunk| match chunk.storage {
ChunkStorage::ReadBuffer | ChunkStorage::ReadBufferAndObjectStore => Some(1),
_ => None,
})
.count()
} }
/// Returns the number of object store chunks in the specified database /// Returns the number of object store chunks in the specified database
pub fn count_object_store_chunks(db: &Db) -> usize { pub fn count_object_store_chunks(db: &Db) -> usize {
chunk_summary_iter(db) count_table_chunks(db, None, None, |s| s.storage.has_object_store())
.filter(|s| {
s.storage == ChunkStorage::ReadBufferAndObjectStore
|| s.storage == ChunkStorage::ObjectStoreOnly
})
.count()
} }
/// return number of OS chunks of a given table of a partition /// return number of OS chunks of a given table of a partition
pub fn count_os_table_chunks(db: &Db, table_name: &str, partition_key: &str) -> usize { pub fn count_os_table_chunks(db: &Db, table_name: &str, partition_key: &str) -> usize {
let mut table_names = BTreeSet::new(); count_table_chunks(db, Some(table_name), Some(partition_key), |s| {
table_names.insert(table_name.to_string()); s.storage.has_object_store()
count_os_tables_chunks( })
db,
TableNameFilter::NamedTables(&table_names),
partition_key,
)
}
pub fn count_os_tables_chunks(db: &Db, tables: TableNameFilter<'_>, partition_key: &str) -> usize {
db.partition_tables_chunk_summaries(tables, partition_key)
.into_iter()
.filter_map(|chunk| match chunk.storage {
ChunkStorage::ObjectStoreOnly | ChunkStorage::ReadBufferAndObjectStore => Some(1),
_ => None,
})
.count()
} }
static PANIC_DATABASE: once_cell::race::OnceBox<parking_lot::Mutex<hashbrown::HashSet<String>>> = static PANIC_DATABASE: once_cell::race::OnceBox<parking_lot::Mutex<hashbrown::HashSet<String>>> =

View File

@ -2,7 +2,6 @@
use std::sync::Arc; use std::sync::Arc;
use datafusion::physical_plan::ExecutionPlan; use datafusion::physical_plan::ExecutionPlan;
use predicate::predicate::Predicate;
use query::{ use query::{
exec::IOxExecutionContext, exec::IOxExecutionContext,
frontend::{influxrpc::InfluxRpcPlanner, sql::SqlQueryPlanner}, frontend::{influxrpc::InfluxRpcPlanner, sql::SqlQueryPlanner},
@ -12,6 +11,7 @@ use query::{
}; };
pub use datafusion::error::{DataFusionError as Error, Result}; pub use datafusion::error::{DataFusionError as Error, Result};
use predicate::rpc_predicate::InfluxRpcPredicate;
/// Query planner that plans queries on a separate threadpool. /// Query planner that plans queries on a separate threadpool.
/// ///
@ -49,7 +49,7 @@ impl Planner {
pub async fn table_names<D>( pub async fn table_names<D>(
&self, &self,
database: Arc<D>, database: Arc<D>,
predicate: Predicate, predicate: InfluxRpcPredicate,
) -> Result<StringSetPlan> ) -> Result<StringSetPlan>
where where
D: QueryDatabase + 'static, D: QueryDatabase + 'static,
@ -67,7 +67,11 @@ impl Planner {
/// Creates a plan as described on /// Creates a plan as described on
/// [`InfluxRpcPlanner::tag_keys`], on a separate threadpool /// [`InfluxRpcPlanner::tag_keys`], on a separate threadpool
pub async fn tag_keys<D>(&self, database: Arc<D>, predicate: Predicate) -> Result<StringSetPlan> pub async fn tag_keys<D>(
&self,
database: Arc<D>,
predicate: InfluxRpcPredicate,
) -> Result<StringSetPlan>
where where
D: QueryDatabase + 'static, D: QueryDatabase + 'static,
{ {
@ -88,7 +92,7 @@ impl Planner {
&self, &self,
database: Arc<D>, database: Arc<D>,
tag_name: impl Into<String> + Send, tag_name: impl Into<String> + Send,
predicate: Predicate, predicate: InfluxRpcPredicate,
) -> Result<StringSetPlan> ) -> Result<StringSetPlan>
where where
D: QueryDatabase + 'static, D: QueryDatabase + 'static,
@ -110,7 +114,7 @@ impl Planner {
pub async fn field_columns<D>( pub async fn field_columns<D>(
&self, &self,
database: Arc<D>, database: Arc<D>,
predicate: Predicate, predicate: InfluxRpcPredicate,
) -> Result<FieldListPlan> ) -> Result<FieldListPlan>
where where
D: QueryDatabase + 'static, D: QueryDatabase + 'static,
@ -131,7 +135,7 @@ impl Planner {
pub async fn read_filter<D>( pub async fn read_filter<D>(
&self, &self,
database: Arc<D>, database: Arc<D>,
predicate: Predicate, predicate: InfluxRpcPredicate,
) -> Result<SeriesSetPlans> ) -> Result<SeriesSetPlans>
where where
D: QueryDatabase + 'static, D: QueryDatabase + 'static,
@ -152,7 +156,7 @@ impl Planner {
pub async fn read_group<D>( pub async fn read_group<D>(
&self, &self,
database: Arc<D>, database: Arc<D>,
predicate: Predicate, predicate: InfluxRpcPredicate,
agg: Aggregate, agg: Aggregate,
group_columns: Vec<String>, group_columns: Vec<String>,
) -> Result<SeriesSetPlans> ) -> Result<SeriesSetPlans>
@ -175,7 +179,7 @@ impl Planner {
pub async fn read_window_aggregate<D>( pub async fn read_window_aggregate<D>(
&self, &self,
database: Arc<D>, database: Arc<D>,
predicate: Predicate, predicate: InfluxRpcPredicate,
agg: Aggregate, agg: Aggregate,
every: WindowDuration, every: WindowDuration,
offset: WindowDuration, offset: WindowDuration,

View File

@ -258,7 +258,7 @@ impl management_service_server::ManagementService for ManagementService {
.map_err(default_server_error_handler)?; .map_err(default_server_error_handler)?;
let chunks: Vec<Chunk> = db let chunks: Vec<Chunk> = db
.partition_chunk_summaries(&partition_key) .filtered_chunk_summaries(None, Some(&partition_key))
.into_iter() .into_iter()
.map(|summary| summary.into()) .map(|summary| summary.into())
.collect(); .collect();

View File

@ -6,12 +6,10 @@ use std::{collections::BTreeSet, fmt, sync::Arc};
use arrow::datatypes::DataType as ArrowDataType; use arrow::datatypes::DataType as ArrowDataType;
use observability_deps::tracing::trace; use observability_deps::tracing::trace;
use query::{ use predicate::rpc_predicate::{FIELD_COLUMN_NAME, MEASUREMENT_COLUMN_NAME};
exec::{ use query::exec::{
fieldlist::FieldList, fieldlist::FieldList,
seriesset::series::{self, Either}, seriesset::series::{self, Either},
},
frontend::influxrpc::{FIELD_COLUMN_NAME, MEASUREMENT_COLUMN_NAME},
}; };
use generated_types::{ use generated_types::{

View File

@ -5,6 +5,7 @@
//! RPCPredicate --> query::Predicates //! RPCPredicate --> query::Predicates
//! //!
//! Aggregates / windows --> query::GroupByAndAggregate //! Aggregates / windows --> query::GroupByAndAggregate
use std::collections::BTreeSet;
use std::{convert::TryFrom, fmt}; use std::{convert::TryFrom, fmt};
use datafusion::{ use datafusion::{
@ -16,16 +17,18 @@ use generated_types::{
aggregate::AggregateType as RPCAggregateType, node::Comparison as RPCComparison, aggregate::AggregateType as RPCAggregateType, node::Comparison as RPCComparison,
node::Logical as RPCLogical, node::Value as RPCValue, read_group_request::Group as RPCGroup, node::Logical as RPCLogical, node::Value as RPCValue, read_group_request::Group as RPCGroup,
Aggregate as RPCAggregate, Duration as RPCDuration, Node as RPCNode, Predicate as RPCPredicate, Aggregate as RPCAggregate, Duration as RPCDuration, Node as RPCNode, Predicate as RPCPredicate,
Window as RPCWindow, TimestampRange as RPCTimestampRange, Window as RPCWindow,
}; };
use super::{TAG_KEY_FIELD, TAG_KEY_MEASUREMENT}; use super::{TAG_KEY_FIELD, TAG_KEY_MEASUREMENT};
use observability_deps::tracing::warn; use observability_deps::tracing::warn;
use predicate::{predicate::PredicateBuilder, regex::regex_match_expr}; use predicate::rpc_predicate::InfluxRpcPredicate;
use query::{ use predicate::{
frontend::influxrpc::{FIELD_COLUMN_NAME, MEASUREMENT_COLUMN_NAME}, predicate::PredicateBuilder,
group_by::{Aggregate as QueryAggregate, WindowDuration}, regex::regex_match_expr,
rpc_predicate::{FIELD_COLUMN_NAME, MEASUREMENT_COLUMN_NAME},
}; };
use query::group_by::{Aggregate as QueryAggregate, WindowDuration};
use snafu::{OptionExt, ResultExt, Snafu}; use snafu::{OptionExt, ResultExt, Snafu};
#[derive(Debug, Snafu)] #[derive(Debug, Snafu)]
@ -146,15 +149,21 @@ pub enum GroupByAndAggregate {
}, },
} }
/// A trait for adding gRPC specific nodes to the generic predicate builder #[derive(Debug, Default)]
pub trait AddRpcNode pub struct InfluxRpcPredicateBuilder {
where table_names: Option<BTreeSet<String>>,
Self: Sized, inner: PredicateBuilder,
{
fn rpc_predicate(self, predicate: Option<RPCPredicate>) -> Result<Self>;
} }
impl AddRpcNode for PredicateBuilder { impl InfluxRpcPredicateBuilder {
/// Sets the timestamp range
pub fn set_range(mut self, range: Option<RPCTimestampRange>) -> Self {
if let Some(range) = range {
self.inner = self.inner.timestamp_range(range.start, range.end)
}
self
}
/// Adds the predicates represented by the Node (predicate tree) /// Adds the predicates represented by the Node (predicate tree)
/// into predicates that can be evaluted by the storage system /// into predicates that can be evaluted by the storage system
/// ///
@ -173,7 +182,7 @@ impl AddRpcNode for PredicateBuilder {
/// ///
/// This code pulls apart the predicates, if any, into a StoragePredicate /// This code pulls apart the predicates, if any, into a StoragePredicate
/// that breaks the predicate apart /// that breaks the predicate apart
fn rpc_predicate(self, rpc_predicate: Option<RPCPredicate>) -> Result<Self> { pub fn rpc_predicate(self, rpc_predicate: Option<RPCPredicate>) -> Result<Self> {
match rpc_predicate { match rpc_predicate {
// no input predicate, is fine // no input predicate, is fine
None => Ok(self), None => Ok(self),
@ -192,6 +201,40 @@ impl AddRpcNode for PredicateBuilder {
} }
} }
} }
/// Adds an optional table name restriction to the existing list
pub fn table_option(self, table: Option<String>) -> Self {
if let Some(table) = table {
self.tables(vec![table])
} else {
self
}
}
/// Sets table name restrictions from something that can iterate
/// over items that can be converted into `Strings`
pub fn tables<I, S>(mut self, tables: I) -> Self
where
I: IntoIterator<Item = S>,
S: Into<String>,
{
// We need to distinguish predicates like `table_name In
// (foo, bar)` and `table_name = foo and table_name = bar` in order to handle
// this
assert!(
self.table_names.is_none(),
"Multiple table predicate specification not yet supported"
);
let table_names: BTreeSet<String> = tables.into_iter().map(|s| s.into()).collect();
self.table_names = Some(table_names);
self
}
pub fn build(self) -> InfluxRpcPredicate {
InfluxRpcPredicate::new(self.table_names, self.inner.build())
}
} }
/// cleans up / normalizes the input in preparation for other /// cleans up / normalizes the input in preparation for other
@ -246,7 +289,10 @@ fn normalize_node(node: RPCNode) -> Result<RPCNode> {
/// ///
/// It recognizes special predicate patterns. If no patterns are /// It recognizes special predicate patterns. If no patterns are
/// matched, it falls back to a generic DataFusion Expr /// matched, it falls back to a generic DataFusion Expr
fn convert_simple_node(builder: PredicateBuilder, node: RPCNode) -> Result<PredicateBuilder> { fn convert_simple_node(
mut builder: InfluxRpcPredicateBuilder,
node: RPCNode,
) -> Result<InfluxRpcPredicateBuilder> {
if let Ok(in_list) = InList::try_from(&node) { if let Ok(in_list) = InList::try_from(&node) {
let InList { lhs, value_list } = in_list; let InList { lhs, value_list } = in_list;
@ -256,15 +302,17 @@ fn convert_simple_node(builder: PredicateBuilder, node: RPCNode) -> Result<Predi
// add the table names as a predicate // add the table names as a predicate
return Ok(builder.tables(value_list)); return Ok(builder.tables(value_list));
} else if tag_name.is_field() { } else if tag_name.is_field() {
return Ok(builder.field_columns(value_list)); builder.inner = builder.inner.field_columns(value_list);
return Ok(builder);
} }
} }
} }
// If no special case applies, fall back to generic conversion // If no special case applies, fall back to generic conversion
let expr = convert_node_to_expr(node)?; let expr = convert_node_to_expr(node)?;
builder.inner = builder.inner.add_expr(expr);
Ok(builder.add_expr(expr)) Ok(builder)
} }
/// converts a tree of (a AND (b AND c)) into [a, b, c] /// converts a tree of (a AND (b AND c)) into [a, b, c]
@ -785,17 +833,26 @@ fn format_comparison(v: i32, f: &mut fmt::Formatter<'_>) -> fmt::Result {
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use generated_types::node::Type as RPCNodeType; use generated_types::node::Type as RPCNodeType;
use predicate::predicate::Predicate;
use std::collections::BTreeSet; use std::collections::BTreeSet;
use super::*; use super::*;
fn table_predicate(predicate: InfluxRpcPredicate) -> Predicate {
let predicates = predicate.table_predicates(|| std::iter::once("foo".to_string()));
assert_eq!(predicates.len(), 1);
predicates.into_iter().next().unwrap().1
}
#[test] #[test]
fn test_convert_predicate_none() { fn test_convert_predicate_none() {
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.rpc_predicate(None) .rpc_predicate(None)
.unwrap() .unwrap()
.build(); .build();
let predicate = table_predicate(predicate);
assert!(predicate.exprs.is_empty()); assert!(predicate.exprs.is_empty());
} }
@ -803,7 +860,7 @@ mod tests {
fn test_convert_predicate_empty() { fn test_convert_predicate_empty() {
let rpc_predicate = RPCPredicate { root: None }; let rpc_predicate = RPCPredicate { root: None };
let res = PredicateBuilder::default().rpc_predicate(Some(rpc_predicate)); let res = InfluxRpcPredicateBuilder::default().rpc_predicate(Some(rpc_predicate));
let expected_error = "Unexpected empty predicate: Node"; let expected_error = "Unexpected empty predicate: Node";
let actual_error = res.unwrap_err().to_string(); let actual_error = res.unwrap_err().to_string();
@ -823,11 +880,12 @@ mod tests {
root: Some(comparison), root: Some(comparison),
}; };
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.rpc_predicate(Some(rpc_predicate)) .rpc_predicate(Some(rpc_predicate))
.expect("successfully converting predicate") .expect("successfully converting predicate")
.build(); .build();
let predicate = table_predicate(predicate);
let converted_expr = &predicate.exprs; let converted_expr = &predicate.exprs;
assert_eq!( assert_eq!(
@ -866,18 +924,28 @@ mod tests {
// _measurement != "foo" // _measurement != "foo"
let rpc_predicate = make_tagref_not_equal_predicate(TAG_KEY_MEASUREMENT, "foo"); let rpc_predicate = make_tagref_not_equal_predicate(TAG_KEY_MEASUREMENT, "foo");
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.rpc_predicate(Some(rpc_predicate)) .rpc_predicate(Some(rpc_predicate))
.expect("successfully converting predicate") .expect("successfully converting predicate")
.build(); .build();
let expected_exprs = vec![col("_measurement").not_eq(lit("foo"))]; let tables = ["foo", "bar"];
assert_eq!( let table_predicates =
&expected_exprs, &predicate.exprs, predicate.table_predicates(|| tables.iter().map(ToString::to_string));
"expected '{:#?}' doesn't match actual '{:#?}'", assert_eq!(table_predicates.len(), 2);
expected_exprs, predicate.exprs,
); for (expected_table, (table, predicate)) in tables.iter().zip(table_predicates) {
assert_eq!(*expected_table, &table);
let expected_exprs = vec![lit(table).not_eq(lit("foo"))];
assert_eq!(
&expected_exprs, &predicate.exprs,
"expected '{:#?}' doesn't match actual '{:#?}'",
expected_exprs, predicate.exprs,
);
}
} }
#[test] #[test]
@ -885,10 +953,11 @@ mod tests {
// _field != "bar" // _field != "bar"
let rpc_predicate = make_tagref_not_equal_predicate(TAG_KEY_FIELD, "bar"); let rpc_predicate = make_tagref_not_equal_predicate(TAG_KEY_FIELD, "bar");
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.rpc_predicate(Some(rpc_predicate)) .rpc_predicate(Some(rpc_predicate))
.expect("successfully converting predicate") .expect("successfully converting predicate")
.build(); .build();
let predicate = table_predicate(predicate);
let expected_exprs = vec![col("_field").not_eq(lit("bar"))]; let expected_exprs = vec![col("_field").not_eq(lit("bar"))];
@ -911,7 +980,7 @@ mod tests {
root: Some(comparison), root: Some(comparison),
}; };
let res = PredicateBuilder::default().rpc_predicate(Some(rpc_predicate)); let res = InfluxRpcPredicateBuilder::default().rpc_predicate(Some(rpc_predicate));
let expected_error = "Error creating predicate: Unsupported number of children in binary operator Gt: 0 (must be 2)"; let expected_error = "Error creating predicate: Unsupported number of children in binary operator Gt: 0 (must be 2)";
let actual_error = res.unwrap_err().to_string(); let actual_error = res.unwrap_err().to_string();
@ -942,7 +1011,7 @@ mod tests {
root: Some(comparison), root: Some(comparison),
}; };
let res = PredicateBuilder::default().rpc_predicate(Some(rpc_predicate)); let res = InfluxRpcPredicateBuilder::default().rpc_predicate(Some(rpc_predicate));
let expected_error = "Error creating predicate: Unknown comparison node type: 42"; let expected_error = "Error creating predicate: Unknown comparison node type: 42";
let actual_error = res.unwrap_err().to_string(); let actual_error = res.unwrap_err().to_string();
@ -973,7 +1042,7 @@ mod tests {
root: Some(comparison), root: Some(comparison),
}; };
let res = PredicateBuilder::default().rpc_predicate(Some(rpc_predicate)); let res = InfluxRpcPredicateBuilder::default().rpc_predicate(Some(rpc_predicate));
let expected_error = "Error creating predicate: Unknown logical node type: 42"; let expected_error = "Error creating predicate: Unknown logical node type: 42";
let actual_error = res.unwrap_err().to_string(); let actual_error = res.unwrap_err().to_string();
@ -993,13 +1062,16 @@ mod tests {
root: Some(field_selection), root: Some(field_selection),
}; };
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.rpc_predicate(Some(rpc_predicate)) .rpc_predicate(Some(rpc_predicate))
.unwrap() .unwrap()
.build(); .build();
assert!(predicate.table_names().is_none());
let predicate = table_predicate(predicate);
assert!(predicate.exprs.is_empty()); assert!(predicate.exprs.is_empty());
assert!(predicate.table_names.is_none());
assert_eq!(predicate.field_columns, Some(to_set(&["field1"]))); assert_eq!(predicate.field_columns, Some(to_set(&["field1"])));
assert!(predicate.range.is_none()); assert!(predicate.range.is_none());
} }
@ -1019,13 +1091,16 @@ mod tests {
root: Some(wrapped), root: Some(wrapped),
}; };
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.rpc_predicate(Some(rpc_predicate)) .rpc_predicate(Some(rpc_predicate))
.unwrap() .unwrap()
.build(); .build();
assert!(predicate.table_names().is_none());
let predicate = table_predicate(predicate);
assert!(predicate.exprs.is_empty()); assert!(predicate.exprs.is_empty());
assert!(predicate.table_names.is_none());
assert_eq!(predicate.field_columns, Some(to_set(&["field1"]))); assert_eq!(predicate.field_columns, Some(to_set(&["field1"])));
assert!(predicate.range.is_none()); assert!(predicate.range.is_none());
} }
@ -1039,13 +1114,16 @@ mod tests {
root: Some(selection), root: Some(selection),
}; };
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.rpc_predicate(Some(rpc_predicate)) .rpc_predicate(Some(rpc_predicate))
.unwrap() .unwrap()
.build(); .build();
assert!(predicate.table_names().is_none());
let predicate = table_predicate(predicate);
assert!(predicate.exprs.is_empty()); assert!(predicate.exprs.is_empty());
assert!(predicate.table_names.is_none());
assert_eq!( assert_eq!(
predicate.field_columns, predicate.field_columns,
Some(to_set(&["field1", "field2", "field3"])) Some(to_set(&["field1", "field2", "field3"]))
@ -1066,11 +1144,15 @@ mod tests {
root: Some(selection), root: Some(selection),
}; };
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.rpc_predicate(Some(rpc_predicate)) .rpc_predicate(Some(rpc_predicate))
.unwrap() .unwrap()
.build(); .build();
assert!(predicate.table_names().is_none());
let predicate = table_predicate(predicate);
let converted_expr = &predicate.exprs; let converted_expr = &predicate.exprs;
assert_eq!( assert_eq!(
@ -1078,7 +1160,6 @@ mod tests {
"expected '{:#?}' doesn't match actual '{:#?}'", "expected '{:#?}' doesn't match actual '{:#?}'",
expected_expr, converted_expr expected_expr, converted_expr
); );
assert!(predicate.table_names.is_none());
assert_eq!(predicate.field_columns, Some(to_set(&["field1", "field2"]))); assert_eq!(predicate.field_columns, Some(to_set(&["field1", "field2"])));
assert!(predicate.range.is_none()); assert!(predicate.range.is_none());
@ -1092,13 +1173,15 @@ mod tests {
root: Some(measurement_selection), root: Some(measurement_selection),
}; };
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.rpc_predicate(Some(rpc_predicate)) .rpc_predicate(Some(rpc_predicate))
.unwrap() .unwrap()
.build(); .build();
assert_eq!(predicate.table_names(), Some(&to_set(&["m1"])));
let predicate = table_predicate(predicate);
assert!(predicate.exprs.is_empty()); assert!(predicate.exprs.is_empty());
assert_eq!(predicate.table_names, Some(to_set(&["m1"])));
assert!(predicate.field_columns.is_none()); assert!(predicate.field_columns.is_none());
assert!(predicate.range.is_none()); assert!(predicate.range.is_none());
} }

View File

@ -25,7 +25,6 @@ use generated_types::{
TimestampRange, TimestampRange,
}; };
use observability_deps::tracing::{error, info, trace}; use observability_deps::tracing::{error, info, trace};
use predicate::predicate::PredicateBuilder;
use query::{ use query::{
exec::{ exec::{
fieldlist::FieldList, seriesset::converter::Error as SeriesSetError, fieldlist::FieldList, seriesset::converter::Error as SeriesSetError,
@ -42,7 +41,7 @@ use crate::influxdb_ioxd::{
fieldlist_to_measurement_fields_response, series_or_groups_to_read_response, fieldlist_to_measurement_fields_response, series_or_groups_to_read_response,
tag_keys_to_byte_vecs, tag_keys_to_byte_vecs,
}, },
expr::{self, AddRpcNode, GroupByAndAggregate, Loggable, SpecialTagKeys}, expr::{self, GroupByAndAggregate, InfluxRpcPredicateBuilder, Loggable, SpecialTagKeys},
input::GrpcInputs, input::GrpcInputs,
StorageService, StorageService,
}, },
@ -774,20 +773,6 @@ where
} }
} }
trait SetRange {
/// sets the timestamp range to range, if present
fn set_range(self, range: Option<TimestampRange>) -> Self;
}
impl SetRange for PredicateBuilder {
fn set_range(self, range: Option<TimestampRange>) -> Self {
if let Some(range) = range {
self.timestamp_range(range.start, range.end)
} else {
self
}
}
}
fn get_database_name(input: &impl GrpcInputs) -> Result<DatabaseName<'static>, Status> { fn get_database_name(input: &impl GrpcInputs) -> Result<DatabaseName<'static>, Status> {
org_and_bucket_to_database(input.org_id()?.to_string(), &input.bucket_name()?) org_and_bucket_to_database(input.org_id()?.to_string(), &input.bucket_name()?)
.map_err(|e| Status::internal(e.to_string())) .map_err(|e| Status::internal(e.to_string()))
@ -813,7 +798,7 @@ where
let rpc_predicate_string = format!("{:?}", rpc_predicate); let rpc_predicate_string = format!("{:?}", rpc_predicate);
let db_name = db_name.as_str(); let db_name = db_name.as_str();
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.set_range(range) .set_range(range)
.rpc_predicate(rpc_predicate) .rpc_predicate(rpc_predicate)
.context(ConvertingPredicateSnafu { .context(ConvertingPredicateSnafu {
@ -861,7 +846,7 @@ where
let rpc_predicate_string = format!("{:?}", rpc_predicate); let rpc_predicate_string = format!("{:?}", rpc_predicate);
let db_name = db_name.as_str(); let db_name = db_name.as_str();
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.set_range(range) .set_range(range)
.table_option(measurement) .table_option(measurement)
.rpc_predicate(rpc_predicate) .rpc_predicate(rpc_predicate)
@ -907,7 +892,7 @@ where
{ {
let rpc_predicate_string = format!("{:?}", rpc_predicate); let rpc_predicate_string = format!("{:?}", rpc_predicate);
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.set_range(range) .set_range(range)
.table_option(measurement) .table_option(measurement)
.rpc_predicate(rpc_predicate) .rpc_predicate(rpc_predicate)
@ -1034,7 +1019,7 @@ where
let rpc_predicate_string = format!("{:?}", req.predicate); let rpc_predicate_string = format!("{:?}", req.predicate);
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.set_range(req.range) .set_range(req.range)
.rpc_predicate(req.predicate) .rpc_predicate(req.predicate)
.context(ConvertingPredicateSnafu { .context(ConvertingPredicateSnafu {
@ -1084,7 +1069,7 @@ where
let rpc_predicate_string = format!("{:?}", rpc_predicate); let rpc_predicate_string = format!("{:?}", rpc_predicate);
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.set_range(range) .set_range(range)
.rpc_predicate(rpc_predicate) .rpc_predicate(rpc_predicate)
.context(ConvertingPredicateSnafu { .context(ConvertingPredicateSnafu {
@ -1141,7 +1126,7 @@ where
{ {
let rpc_predicate_string = format!("{:?}", rpc_predicate); let rpc_predicate_string = format!("{:?}", rpc_predicate);
let predicate = PredicateBuilder::default() let predicate = InfluxRpcPredicateBuilder::default()
.set_range(range) .set_range(range)
.table_option(measurement) .table_option(measurement)
.rpc_predicate(rpc_predicate) .rpc_predicate(rpc_predicate)
@ -1366,7 +1351,7 @@ mod tests {
Client as StorageClient, OrgAndBucket, Client as StorageClient, OrgAndBucket,
}; };
use panic_logging::SendPanicsToTracing; use panic_logging::SendPanicsToTracing;
use predicate::predicate::PredicateMatch; use predicate::predicate::{PredicateBuilder, PredicateMatch};
use query::{ use query::{
exec::Executor, exec::Executor,
test::{TestChunk, TestDatabase, TestError}, test::{TestChunk, TestDatabase, TestError},
@ -1706,7 +1691,6 @@ mod tests {
let expected_predicate = PredicateBuilder::default() let expected_predicate = PredicateBuilder::default()
.timestamp_range(150, 200) .timestamp_range(150, 200)
.add_expr(make_state_ma_expr()) .add_expr(make_state_ma_expr())
.table("m4")
.build(); .build();
assert!( assert!(
@ -1728,7 +1712,7 @@ mod tests {
let db_info = org_and_bucket(); let db_info = org_and_bucket();
// predicate specifies m4, so this is filtered out // predicate specifies m4, so this is filtered out
let chunk = TestChunk::new("my_table").with_error("This is an error"); let chunk = TestChunk::new("m5").with_error("This is an error");
fixture fixture
.test_storage .test_storage
@ -2261,7 +2245,7 @@ mod tests {
let db_info = org_and_bucket(); let db_info = org_and_bucket();
let chunk = TestChunk::new("my_table").with_error("Sugar we are going down"); let chunk = TestChunk::new("m5").with_error("Sugar we are going down");
fixture fixture
.test_storage .test_storage
@ -2745,7 +2729,7 @@ mod tests {
let db_info = org_and_bucket(); let db_info = org_and_bucket();
let chunk = TestChunk::new("t").with_error("Sugar we are going down"); let chunk = TestChunk::new("TheMeasurement").with_error("Sugar we are going down");
fixture fixture
.test_storage .test_storage

View File

@ -9,11 +9,12 @@ chrono = { version = "0.4", default-features = false }
data_types = { path = "../data_types" } data_types = { path = "../data_types" }
datafusion = { path = "../datafusion" } datafusion = { path = "../datafusion" }
datafusion_util = { path = "../datafusion_util" } datafusion_util = { path = "../datafusion_util" }
schema = { path = "../schema" } itertools = "0.10"
observability_deps = { path = "../observability_deps" } observability_deps = { path = "../observability_deps" }
ordered-float = "2" ordered-float = "2"
regex = "1" regex = "1"
regex-syntax = "0.6.25" regex-syntax = "0.6.25"
schema = { path = "../schema" }
serde_json = "1.0.72" serde_json = "1.0.72"
snafu = "0.7" snafu = "0.7"
sqlparser = "0.13.0" sqlparser = "0.13.0"

View File

@ -71,7 +71,6 @@ pub type Result<T, E = Error> = std::result::Result<T, E>;
impl From<DeletePredicate> for crate::predicate::Predicate { impl From<DeletePredicate> for crate::predicate::Predicate {
fn from(pred: DeletePredicate) -> Self { fn from(pred: DeletePredicate) -> Self {
Self { Self {
table_names: None,
field_columns: None, field_columns: None,
partition_key: None, partition_key: None,
range: Some(pred.range), range: Some(pred.range),

View File

@ -12,3 +12,4 @@ pub mod delete_expr;
pub mod delete_predicate; pub mod delete_predicate;
pub mod predicate; pub mod predicate;
pub mod regex; pub mod regex;
pub mod rpc_predicate;

View File

@ -21,7 +21,6 @@ use schema::TIME_COLUMN_NAME;
/// This `Predicate` represents the empty predicate (aka that /// This `Predicate` represents the empty predicate (aka that
/// evaluates to true for all rows). /// evaluates to true for all rows).
pub const EMPTY_PREDICATE: Predicate = Predicate { pub const EMPTY_PREDICATE: Predicate = Predicate {
table_names: None,
field_columns: None, field_columns: None,
exprs: vec![], exprs: vec![],
range: None, range: None,
@ -53,10 +52,6 @@ pub enum PredicateMatch {
/// restrictions that only apply to certain types of columns. /// restrictions that only apply to certain types of columns.
#[derive(Clone, Debug, Default, PartialEq, PartialOrd)] #[derive(Clone, Debug, Default, PartialEq, PartialOrd)]
pub struct Predicate { pub struct Predicate {
/// Optional table restriction. If present, restricts the results
/// to only tables whose names are in `table_names`
pub table_names: Option<BTreeSet<String>>,
/// Optional field restriction. If present, restricts the results to only /// Optional field restriction. If present, restricts the results to only
/// tables which have *at least one* of the fields in field_columns. /// tables which have *at least one* of the fields in field_columns.
pub field_columns: Option<BTreeSet<String>>, pub field_columns: Option<BTreeSet<String>>,
@ -102,15 +97,6 @@ impl Predicate {
builder.build() builder.build()
} }
/// Return true if results from this table should be included in
/// results
pub fn should_include_table(&self, table_name: &str) -> bool {
match &self.table_names {
None => true, // No table name restriction on predicate
Some(table_names) => table_names.contains(table_name),
}
}
/// Return true if the field should be included in results /// Return true if the field should be included in results
pub fn should_include_field(&self, field_name: &str) -> bool { pub fn should_include_field(&self, field_name: &str) -> bool {
match &self.field_columns { match &self.field_columns {
@ -213,7 +199,7 @@ impl Predicate {
/// ///
/// This is used in certain cases to retain compatibility with the /// This is used in certain cases to retain compatibility with the
/// existing storage engine /// existing storage engine
pub fn clear_timestamp_if_max_range(mut self) -> Self { pub(crate) fn clear_timestamp_if_max_range(mut self) -> Self {
self.range = self.range.take().and_then(|range| { self.range = self.range.take().and_then(|range| {
if range.start() <= MIN_NANO_TIME && range.end() >= MAX_NANO_TIME { if range.start() <= MIN_NANO_TIME && range.end() >= MAX_NANO_TIME {
None None
@ -240,10 +226,6 @@ impl fmt::Display for Predicate {
write!(f, "Predicate")?; write!(f, "Predicate")?;
if let Some(table_names) = &self.table_names {
write!(f, " table_names: {{{}}}", iter_to_str(table_names))?;
}
if let Some(field_columns) = &self.field_columns { if let Some(field_columns) = &self.field_columns {
write!(f, " field_columns: {{{}}}", iter_to_str(field_columns))?; write!(f, " field_columns: {{{}}}", iter_to_str(field_columns))?;
} }
@ -353,41 +335,6 @@ impl PredicateBuilder {
self self
} }
/// Adds an optional table name restriction to the existing list
pub fn table_option(self, table: Option<String>) -> Self {
if let Some(table) = table {
self.tables(vec![table])
} else {
self
}
}
/// Set the table restriction to `table`
pub fn table(self, table: impl Into<String>) -> Self {
self.tables(vec![table.into()])
}
/// Sets table name restrictions from something that can iterate
/// over items that can be converted into `Strings`
pub fn tables<I, S>(mut self, tables: I) -> Self
where
I: IntoIterator<Item = S>,
S: Into<String>,
{
// We need to distinguish predicates like `table_name In
// (foo, bar)` and `table_name = foo and table_name = bar` in order to handle
// this
assert!(
self.inner.table_names.is_none(),
"Multiple table predicate specification not yet supported"
);
let table_names: BTreeSet<String> = tables.into_iter().map(|s| s.into()).collect();
self.inner.table_names = Some(table_names);
self
}
/// Sets field_column restriction /// Sets field_column restriction
pub fn field_columns(mut self, columns: Vec<impl Into<String>>) -> Self { pub fn field_columns(mut self, columns: Vec<impl Into<String>>) -> Self {
// We need to distinguish predicates like `column_name In // We need to distinguish predicates like `column_name In
@ -639,12 +586,11 @@ mod tests {
let p = PredicateBuilder::new() let p = PredicateBuilder::new()
.timestamp_range(1, 100) .timestamp_range(1, 100)
.add_expr(col("foo").eq(lit(42))) .add_expr(col("foo").eq(lit(42)))
.table("my_table")
.field_columns(vec!["f1", "f2"]) .field_columns(vec!["f1", "f2"])
.partition_key("the_key") .partition_key("the_key")
.build(); .build();
assert_eq!(p.to_string(), "Predicate table_names: {my_table} field_columns: {f1, f2} partition_key: 'the_key' range: [1 - 100] exprs: [#foo = Int32(42)]"); assert_eq!(p.to_string(), "Predicate field_columns: {f1, f2} partition_key: 'the_key' range: [1 - 100] exprs: [#foo = Int32(42)]");
} }
#[test] #[test]

View File

@ -0,0 +1,387 @@
//! Interface logic between IOx ['Predicate`] and predicates used by the
//! InfluxDB Storage gRPC API
use crate::predicate::{BinaryExpr, Predicate};
use datafusion::error::Result as DataFusionResult;
use datafusion::logical_plan::{lit, Column, Expr, ExprRewriter, Operator};
use datafusion::scalar::ScalarValue;
use datafusion_util::AsExpr;
use std::collections::BTreeSet;
/// Any column references to this name are rewritten to be
/// the actual table name by the Influx gRPC planner.
///
/// This is required to support predicates like
/// `_measurement = "foo" OR tag1 = "bar"`
///
/// The plan for each table will have the value of `_measurement`
/// filled in with a literal for the respective name of that field
pub const MEASUREMENT_COLUMN_NAME: &str = "_measurement";
/// Any equality expressions using this column name are removed and replaced
/// with projections on the specified column.
///
/// This is required to support predicates like
/// `_field` = temperature
pub const FIELD_COLUMN_NAME: &str = "_field";
/// Any column references to this name are rewritten to be a disjunctive set of
/// expressions to all field columns for the table schema.
///
/// This is required to support predicates like
/// `_value` = 1.77
///
/// The plan for each table will have expression containing `_value` rewritten
/// into multiple expressions (one for each field column).
pub const VALUE_COLUMN_NAME: &str = "_value";
/// Predicate used by the InfluxDB Storage gRPC API
#[derive(Debug, Clone, Default)]
pub struct InfluxRpcPredicate {
/// Optional table restriction. If present, restricts the results
/// to only tables whose names are in `table_names`
table_names: Option<BTreeSet<String>>,
/// The inner predicate
inner: Predicate,
}
impl InfluxRpcPredicate {
/// Create a new [`InfluxRpcPredicate`]
pub fn new(table_names: Option<BTreeSet<String>>, predicate: Predicate) -> Self {
Self {
table_names,
inner: predicate,
}
}
/// Create a new [`InfluxRpcPredicate`] with the given table
pub fn new_table(table: impl Into<String>, predicate: Predicate) -> Self {
Self::new(Some(std::iter::once(table.into()).collect()), predicate)
}
/// Removes the timestamp range from this predicate, if the range
/// is for the entire min/max valid range.
///
/// This is used in certain cases to retain compatibility with the
/// existing storage engine
pub fn clear_timestamp_if_max_range(self) -> Self {
Self {
inner: self.inner.clear_timestamp_if_max_range(),
..self
}
}
/// Convert to a list of [`Predicate`] to apply to specific tables
///
/// `all_table_names` yields a list of all table names in the databases and is used when
/// the storage predicate has no table restriction
///
/// Returns a list of [`Predicate`] and their associated table name
pub fn table_predicates<F, I>(&self, all_table_names: F) -> Vec<(String, Predicate)>
where
F: FnOnce() -> I,
I: IntoIterator<Item = String>,
{
let table_names = match &self.table_names {
Some(table_names) => itertools::Either::Left(table_names.iter().cloned()),
None => itertools::Either::Right(all_table_names().into_iter()),
};
table_names
.map(|table| {
let predicate = normalize_predicate(&table, &self.inner);
(table, predicate)
})
.collect()
}
/// Returns the table names this predicate is restricted to if any
pub fn table_names(&self) -> Option<&BTreeSet<String>> {
self.table_names.as_ref()
}
/// Returns true if ths predicate evaluates to true for all rows
pub fn is_empty(&self) -> bool {
self.table_names.is_none() && self.inner.is_empty()
}
}
/// Predicate that has been "specialized" / normalized for a
/// particular table. Specifically:
///
/// * all references to the [MEASUREMENT_COLUMN_NAME] column in any
/// `Exprs` are rewritten with the actual table name
/// * any expression on the [VALUE_COLUMN_NAME] column is rewritten to be
/// applied across all field columns.
/// * any expression on the [FIELD_COLUMN_NAME] is rewritten to be
/// applied for the particular fields.
///
/// For example if the original predicate was
/// ```text
/// _measurement = "some_table"
/// ```
///
/// When evaluated on table "cpu" then the predicate is rewritten to
/// ```text
/// "cpu" = "some_table"
/// ```
///
/// if the original predicate contained
/// ```text
/// _value > 34.2
/// ```
///
/// When evaluated on table "cpu" then the expression is rewritten as a
/// collection of disjunctive expressions against all field columns
/// ```text
/// ("field1" > 34.2 OR "field2" > 34.2 OR "fieldn" > 34.2)
/// ```
fn normalize_predicate(table_name: &str, predicate: &Predicate) -> Predicate {
let mut predicate = predicate.clone();
let mut field_projections = BTreeSet::new();
let mut field_value_exprs = vec![];
predicate.exprs = predicate
.exprs
.into_iter()
.map(|e| rewrite_measurement_references(table_name, e))
// Rewrite any references to `_value = some_value` to literal true values.
// Keeps track of these expressions, which can then be used to
// augment field projections with conditions using `CASE` statements.
.map(|e| rewrite_field_value_references(&mut field_value_exprs, e))
.map(|e| {
// Rewrite any references to `_field = a_field_name` with a literal true
// and keep track of referenced field names to add to the field
// column projection set.
rewrite_field_column_references(&mut field_projections, e)
})
.collect::<Vec<_>>();
// Store any field value (`_value`) expressions on the `Predicate`.
predicate.value_expr = field_value_exprs;
if !field_projections.is_empty() {
match &mut predicate.field_columns {
Some(field_columns) => field_columns.extend(field_projections.into_iter()),
None => predicate.field_columns = Some(field_projections),
};
}
predicate
}
/// Rewrites all references to the [MEASUREMENT_COLUMN_NAME] column
/// with the actual table name
fn rewrite_measurement_references(table_name: &str, expr: Expr) -> Expr {
let mut rewriter = MeasurementRewriter { table_name };
expr.rewrite(&mut rewriter).expect("rewrite is infallible")
}
struct MeasurementRewriter<'a> {
table_name: &'a str,
}
impl ExprRewriter for MeasurementRewriter<'_> {
fn mutate(&mut self, expr: Expr) -> DataFusionResult<Expr> {
Ok(match expr {
// rewrite col("_measurement") --> "table_name"
Expr::Column(Column { relation, name }) if name == MEASUREMENT_COLUMN_NAME => {
// should not have a qualified foo._measurement
// reference
assert!(relation.is_none());
lit(self.table_name)
}
// no rewrite needed
_ => expr,
})
}
}
/// Rewrites an expression on `_value` as a boolean true literal, pushing any
/// encountered expressions onto `value_exprs` so they can be moved onto column
/// projections.
fn rewrite_field_value_references(value_exprs: &mut Vec<BinaryExpr>, expr: Expr) -> Expr {
let mut rewriter = FieldValueRewriter { value_exprs };
expr.rewrite(&mut rewriter).expect("rewrite is infallible")
}
struct FieldValueRewriter<'a> {
value_exprs: &'a mut Vec<BinaryExpr>,
}
impl<'a> ExprRewriter for FieldValueRewriter<'a> {
fn mutate(&mut self, expr: Expr) -> DataFusionResult<Expr> {
Ok(match expr {
Expr::BinaryExpr {
ref left,
op,
ref right,
} => {
if let Expr::Column(inner) = &**left {
if inner.name == VALUE_COLUMN_NAME {
self.value_exprs.push(BinaryExpr {
left: inner.to_owned(),
op,
right: right.as_expr(),
});
return Ok(lit(true));
}
}
expr
}
_ => expr,
})
}
}
/// Rewrites a predicate on `_field` as a projection on a specific defined by
/// the literal in the expression.
///
/// For example, the expression `_field = "load4"` is removed from the
/// normalised expression, and a column "load4" added to the predicate
/// projection.
fn rewrite_field_column_references(
field_projections: &'_ mut BTreeSet<String>,
expr: Expr,
) -> Expr {
let mut rewriter = FieldColumnRewriter { field_projections };
expr.rewrite(&mut rewriter).expect("rewrite is infallible")
}
struct FieldColumnRewriter<'a> {
field_projections: &'a mut BTreeSet<String>,
}
impl<'a> ExprRewriter for FieldColumnRewriter<'a> {
fn mutate(&mut self, expr: Expr) -> DataFusionResult<Expr> {
Ok(match expr {
Expr::BinaryExpr {
ref left,
op,
ref right,
} => {
if let Expr::Column(inner) = &**left {
if inner.name != FIELD_COLUMN_NAME || op != Operator::Eq {
// TODO(edd): add support for !=
return Ok(expr);
}
if let Expr::Literal(ScalarValue::Utf8(Some(name))) = &**right {
self.field_projections.insert(name.to_owned());
return Ok(lit(true));
}
}
expr
}
_ => expr,
})
}
}
#[cfg(test)]
mod tests {
use super::*;
use datafusion::logical_plan::{binary_expr, col};
#[test]
fn test_field_value_rewriter() {
let mut rewriter = FieldValueRewriter {
value_exprs: &mut vec![],
};
let cases = vec![
(
binary_expr(col("f1"), Operator::Eq, lit(1.82)),
binary_expr(col("f1"), Operator::Eq, lit(1.82)),
vec![],
),
(col("t2"), col("t2"), vec![]),
(
binary_expr(col(VALUE_COLUMN_NAME), Operator::Eq, lit(1.82)),
// _value = 1.82 -> true
lit(true),
vec![BinaryExpr {
left: Column {
relation: None,
name: VALUE_COLUMN_NAME.into(),
},
op: Operator::Eq,
right: lit(1.82),
}],
),
];
for (input, exp, mut value_exprs) in cases {
let rewritten = input.rewrite(&mut rewriter).unwrap();
assert_eq!(rewritten, exp);
assert_eq!(rewriter.value_exprs, &mut value_exprs);
}
// Test case with single field.
let mut rewriter = FieldValueRewriter {
value_exprs: &mut vec![],
};
let input = binary_expr(col(VALUE_COLUMN_NAME), Operator::Gt, lit(1.88));
let rewritten = input.rewrite(&mut rewriter).unwrap();
assert_eq!(rewritten, lit(true));
assert_eq!(
rewriter.value_exprs,
&mut vec![BinaryExpr {
left: Column {
relation: None,
name: VALUE_COLUMN_NAME.into(),
},
op: Operator::Gt,
right: lit(1.88),
}]
);
}
#[test]
fn test_field_column_rewriter() {
let mut field_columns = BTreeSet::new();
let mut rewriter = FieldColumnRewriter {
field_projections: &mut field_columns,
};
let cases = vec![
(
binary_expr(col("f1"), Operator::Eq, lit(1.82)),
binary_expr(col("f1"), Operator::Eq, lit(1.82)),
vec![],
),
(
// TODO - should be rewritten and project onto all field columns
binary_expr(col(FIELD_COLUMN_NAME), Operator::NotEq, lit("foo")),
binary_expr(col(FIELD_COLUMN_NAME), Operator::NotEq, lit("foo")),
vec![],
),
(
binary_expr(col(FIELD_COLUMN_NAME), Operator::Eq, lit("f1")),
lit(true),
vec!["f1"],
),
(
binary_expr(
binary_expr(col(FIELD_COLUMN_NAME), Operator::Eq, lit("f1")),
Operator::Or,
binary_expr(col(FIELD_COLUMN_NAME), Operator::Eq, lit("f2")),
),
binary_expr(lit(true), Operator::Or, lit(true)),
vec!["f1", "f2"],
),
];
for (input, exp_expr, field_columns) in cases {
let rewritten = input.rewrite(&mut rewriter).unwrap();
assert_eq!(rewritten, exp_expr);
let mut exp_field_columns = field_columns
.into_iter()
.map(String::from)
.collect::<BTreeSet<String>>();
assert_eq!(rewriter.field_projections, &mut exp_field_columns);
}
}
}

View File

@ -11,11 +11,9 @@ use arrow::{
bitmap::Bitmap, bitmap::Bitmap,
datatypes::DataType as ArrowDataType, datatypes::DataType as ArrowDataType,
}; };
use predicate::rpc_predicate::{FIELD_COLUMN_NAME, MEASUREMENT_COLUMN_NAME};
use crate::{ use crate::exec::{field::FieldIndex, seriesset::SeriesSet};
exec::{field::FieldIndex, seriesset::SeriesSet},
frontend::influxrpc::{FIELD_COLUMN_NAME, MEASUREMENT_COLUMN_NAME},
};
use snafu::Snafu; use snafu::Snafu;
#[derive(Debug, Snafu)] #[derive(Debug, Snafu)]

File diff suppressed because it is too large Load Diff

View File

@ -117,13 +117,16 @@ pub trait QueryDatabase: Debug + Send + Sync {
/// Return the partition keys for data in this DB /// Return the partition keys for data in this DB
fn partition_addrs(&self) -> Vec<PartitionAddr>; fn partition_addrs(&self) -> Vec<PartitionAddr>;
/// Returns a list of table names in this DB
fn table_names(&self) -> Vec<String>;
/// Schema for a specific table if the table exists. /// Schema for a specific table if the table exists.
fn table_schema(&self, table_name: &str) -> Option<Arc<Schema>>; fn table_schema(&self, table_name: &str) -> Option<Arc<Schema>>;
/// Returns a set of chunks within the partition with data that may match /// Returns a set of chunks within the partition with data that may match
/// the provided predicate. If possible, chunks which have no rows that can /// the provided predicate. If possible, chunks which have no rows that can
/// possibly match the predicate may be omitted. /// possibly match the predicate may be omitted.
fn chunks(&self, predicate: &Predicate) -> Vec<Arc<Self::Chunk>>; fn chunks(&self, table_name: &str, predicate: &Predicate) -> Vec<Arc<Self::Chunk>>;
/// Return a summary of all chunks in this database, in all partitions /// Return a summary of all chunks in this database, in all partitions
fn chunk_summaries(&self) -> Vec<ChunkSummary>; fn chunk_summaries(&self) -> Vec<ChunkSummary>;

View File

@ -12,6 +12,12 @@ impl FieldListPlan {
Self::default() Self::default()
} }
pub fn with_capacity(capacity: usize) -> Self {
Self {
plans: Vec::with_capacity(capacity),
}
}
/// Append a new plan to this list of plans /// Append a new plan to this list of plans
pub fn append(mut self, plan: LogicalPlan) -> Self { pub fn append(mut self, plan: LogicalPlan) -> Self {
self.plans.push(plan); self.plans.push(plan);

View File

@ -25,6 +25,7 @@ use data_types::{
use datafusion::physical_plan::SendableRecordBatchStream; use datafusion::physical_plan::SendableRecordBatchStream;
use datafusion_util::stream_from_batches; use datafusion_util::stream_from_batches;
use futures::StreamExt; use futures::StreamExt;
use hashbrown::HashSet;
use observability_deps::tracing::debug; use observability_deps::tracing::debug;
use parking_lot::Mutex; use parking_lot::Mutex;
use schema::selection::Selection; use schema::selection::Selection;
@ -118,11 +119,12 @@ impl QueryDatabase for TestDatabase {
.collect() .collect()
} }
fn chunks(&self, _predicate: &Predicate) -> Vec<Arc<Self::Chunk>> { fn chunks(&self, table_name: &str, _predicate: &Predicate) -> Vec<Arc<Self::Chunk>> {
let partitions = self.partitions.lock(); let partitions = self.partitions.lock();
partitions partitions
.values() .values()
.flat_map(|x| x.values()) .flat_map(|x| x.values())
.filter(|x| x.table_name == table_name)
.cloned() .cloned()
.collect() .collect()
} }
@ -155,6 +157,18 @@ impl QueryDatabase for TestDatabase {
) -> QueryCompletedToken<'_> { ) -> QueryCompletedToken<'_> {
QueryCompletedToken::new(|| {}) QueryCompletedToken::new(|| {})
} }
fn table_names(&self) -> Vec<String> {
let mut values = HashSet::new();
let partitions = self.partitions.lock();
for chunks in partitions.values() {
for chunk in chunks.values() {
values.get_or_insert_owned(&chunk.table_name);
}
}
values.into_iter().collect()
}
} }
impl ExecutionContextProvider for TestDatabase { impl ExecutionContextProvider for TestDatabase {
@ -900,14 +914,7 @@ impl QueryChunk for TestChunk {
return Ok(predicate_match); return Ok(predicate_match);
} }
// otherwise fall back to basic filtering based on table name predicate. Ok(PredicateMatch::Unknown)
let predicate_match = if !predicate.should_include_table(&self.table_name) {
PredicateMatch::Zero
} else {
PredicateMatch::Unknown
};
Ok(predicate_match)
} }
fn column_values( fn column_values(

View File

@ -1,6 +1,7 @@
use arrow::datatypes::DataType; use arrow::datatypes::DataType;
use datafusion::logical_plan::{col, lit}; use datafusion::logical_plan::{col, lit};
use predicate::predicate::{Predicate, PredicateBuilder}; use predicate::predicate::PredicateBuilder;
use predicate::rpc_predicate::InfluxRpcPredicate;
use query::{ use query::{
exec::fieldlist::{Field, FieldList}, exec::fieldlist::{Field, FieldList},
frontend::influxrpc::InfluxRpcPlanner, frontend::influxrpc::InfluxRpcPlanner,
@ -15,7 +16,7 @@ use crate::scenarios::*;
/// output /// output
async fn run_field_columns_test_case<D>( async fn run_field_columns_test_case<D>(
db_setup: D, db_setup: D,
predicate: Predicate, predicate: InfluxRpcPredicate,
expected_fields: FieldList, expected_fields: FieldList,
) where ) where
D: DbSetup, D: DbSetup,
@ -49,17 +50,16 @@ async fn run_field_columns_test_case<D>(
#[tokio::test] #[tokio::test]
async fn test_field_columns_empty_database() { async fn test_field_columns_empty_database() {
let predicate = PredicateBuilder::default().build();
let expected_fields = FieldList::default(); let expected_fields = FieldList::default();
run_field_columns_test_case(NoData {}, predicate, expected_fields).await; run_field_columns_test_case(NoData {}, InfluxRpcPredicate::default(), expected_fields).await;
} }
#[tokio::test] #[tokio::test]
async fn test_field_columns_no_predicate() { async fn test_field_columns_no_predicate() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.table("NoSuchTable")
.add_expr(col("state").eq(lit("MA"))) // state=MA .add_expr(col("state").eq(lit("MA"))) // state=MA
.build(); .build();
let predicate = InfluxRpcPredicate::new_table("NoSuchTable", predicate);
let expected_fields = FieldList::default(); let expected_fields = FieldList::default();
run_field_columns_test_case(TwoMeasurementsManyFields {}, predicate, expected_fields).await; run_field_columns_test_case(TwoMeasurementsManyFields {}, predicate, expected_fields).await;
} }
@ -70,9 +70,9 @@ async fn test_field_columns_no_predicate() {
async fn test_field_columns_with_pred() { async fn test_field_columns_with_pred() {
// get only fields from h20 (but both chunks) // get only fields from h20 (but both chunks)
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.table("h2o")
.add_expr(col("state").eq(lit("MA"))) // state=MA .add_expr(col("state").eq(lit("MA"))) // state=MA
.build(); .build();
let predicate = InfluxRpcPredicate::new_table("h2o", predicate);
let expected_fields = FieldList { let expected_fields = FieldList {
fields: vec![ fields: vec![
@ -103,6 +103,7 @@ async fn test_field_columns_measurement_pred() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.add_expr(col("_measurement").eq(lit("h2o"))) .add_expr(col("_measurement").eq(lit("h2o")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_fields = FieldList { let expected_fields = FieldList {
fields: vec![ fields: vec![
@ -130,10 +131,10 @@ async fn test_field_columns_measurement_pred() {
#[tokio::test] #[tokio::test]
async fn test_field_columns_with_ts_pred() { async fn test_field_columns_with_ts_pred() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.table("h2o")
.timestamp_range(200, 300) .timestamp_range(200, 300)
.add_expr(col("state").eq(lit("MA"))) // state=MA .add_expr(col("state").eq(lit("MA"))) // state=MA
.build(); .build();
let predicate = InfluxRpcPredicate::new_table("h2o", predicate);
let expected_fields = FieldList { let expected_fields = FieldList {
fields: vec![Field { fields: vec![Field {
@ -151,6 +152,7 @@ async fn test_field_name_plan() {
test_helpers::maybe_start_logging(); test_helpers::maybe_start_logging();
let predicate = PredicateBuilder::default().timestamp_range(0, 2000).build(); let predicate = PredicateBuilder::default().timestamp_range(0, 2000).build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_fields = FieldList { let expected_fields = FieldList {
fields: vec![ fields: vec![
@ -185,6 +187,7 @@ async fn test_field_name_plan_with_delete() {
test_helpers::maybe_start_logging(); test_helpers::maybe_start_logging();
let predicate = PredicateBuilder::default().timestamp_range(0, 2000).build(); let predicate = PredicateBuilder::default().timestamp_range(0, 2000).build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_fields = FieldList { let expected_fields = FieldList {
fields: vec![ fields: vec![

View File

@ -11,7 +11,8 @@ use data_types::{
timestamp::TimestampRange, timestamp::TimestampRange,
}; };
use datafusion::logical_plan::{col, lit}; use datafusion::logical_plan::{col, lit};
use predicate::predicate::{Predicate, PredicateBuilder, EMPTY_PREDICATE}; use predicate::predicate::PredicateBuilder;
use predicate::rpc_predicate::InfluxRpcPredicate;
use query::frontend::influxrpc::InfluxRpcPlanner; use query::frontend::influxrpc::InfluxRpcPlanner;
#[derive(Debug)] #[derive(Debug)]
@ -120,7 +121,7 @@ impl DbSetup for TwoMeasurementsMultiSeriesWithDeleteAll {
/// output /// output
async fn run_read_filter_test_case<D>( async fn run_read_filter_test_case<D>(
db_setup: D, db_setup: D,
predicate: Predicate, predicate: InfluxRpcPredicate,
expected_results: Vec<&str>, expected_results: Vec<&str>,
) where ) where
D: DbSetup, D: DbSetup,
@ -152,15 +153,13 @@ async fn run_read_filter_test_case<D>(
#[tokio::test] #[tokio::test]
async fn test_read_filter_no_data_no_pred() { async fn test_read_filter_no_data_no_pred() {
let predicate = EMPTY_PREDICATE;
let expected_results = vec![] as Vec<&str>; let expected_results = vec![] as Vec<&str>;
run_read_filter_test_case(NoData {}, predicate, expected_results).await; run_read_filter_test_case(NoData {}, InfluxRpcPredicate::default(), expected_results).await;
} }
#[tokio::test] #[tokio::test]
async fn test_read_filter_data_no_pred() { async fn test_read_filter_data_no_pred() {
let predicate = EMPTY_PREDICATE;
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100, 250], values: [70.4, 72.4]", "Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100, 250], values: [70.4, 72.4]",
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [200, 350], values: [90.0, 90.0]", "Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [200, 350], values: [90.0, 90.0]",
@ -168,7 +167,12 @@ async fn test_read_filter_data_no_pred() {
"Series tags={_measurement=o2, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]", "Series tags={_measurement=o2, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100, 250], values: [50.4, 53.4]",
]; ];
run_read_filter_test_case(TwoMeasurementsMultiSeries {}, predicate, expected_results).await; run_read_filter_test_case(
TwoMeasurementsMultiSeries {},
InfluxRpcPredicate::default(),
expected_results,
)
.await;
} }
#[tokio::test] #[tokio::test]
@ -178,6 +182,7 @@ async fn test_read_filter_data_exclusive_predicate() {
// range.start <= ts < range.end // range.start <= ts < range.end
.timestamp_range(349, 350) .timestamp_range(349, 350)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![]; let expected_results = vec![];
@ -190,6 +195,7 @@ async fn test_read_filter_data_inclusive_predicate() {
// should return 350 row! // should return 350 row!
.timestamp_range(350, 351) .timestamp_range(350, 351)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [350], values: [90.0]", "Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [350], values: [90.0]",
@ -204,6 +210,7 @@ async fn test_read_filter_data_exact_predicate() {
// should return 250 rows! // should return 250 rows!
.timestamp_range(250, 251) .timestamp_range(250, 251)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [250], values: [72.4]", "Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [250], values: [72.4]",
@ -220,6 +227,7 @@ async fn test_read_filter_data_tag_predicate() {
// region = region // region = region
.add_expr(col("region").eq(col("region"))) .add_expr(col("region").eq(col("region")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
// expect both series to be returned // expect both series to be returned
let expected_results = vec![ let expected_results = vec![
@ -232,7 +240,6 @@ async fn test_read_filter_data_tag_predicate() {
#[tokio::test] #[tokio::test]
async fn test_read_filter_data_no_pred_with_delete() { async fn test_read_filter_data_no_pred_with_delete() {
let predicate = EMPTY_PREDICATE;
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100], values: [70.4]", "Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100], values: [70.4]",
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [350], values: [90.0]", "Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [350], values: [90.0]",
@ -242,7 +249,7 @@ async fn test_read_filter_data_no_pred_with_delete() {
run_read_filter_test_case( run_read_filter_test_case(
TwoMeasurementsMultiSeriesWithDelete {}, TwoMeasurementsMultiSeriesWithDelete {},
predicate, InfluxRpcPredicate::default(),
expected_results, expected_results,
) )
.await; .await;
@ -250,7 +257,6 @@ async fn test_read_filter_data_no_pred_with_delete() {
#[tokio::test] #[tokio::test]
async fn test_read_filter_data_no_pred_with_delete_all() { async fn test_read_filter_data_no_pred_with_delete_all() {
let predicate = EMPTY_PREDICATE;
// nothing from h2o table because all rows were deleted // nothing from h2o table because all rows were deleted
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=o2, city=Boston, state=MA, _field=reading}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]", "Series tags={_measurement=o2, city=Boston, state=MA, _field=reading}\n FloatPoints timestamps: [100, 250], values: [50.0, 51.0]",
@ -259,7 +265,7 @@ async fn test_read_filter_data_no_pred_with_delete_all() {
run_read_filter_test_case( run_read_filter_test_case(
TwoMeasurementsMultiSeriesWithDeleteAll {}, TwoMeasurementsMultiSeriesWithDeleteAll {},
predicate, InfluxRpcPredicate::default(),
expected_results, expected_results,
) )
.await; .await;
@ -272,6 +278,7 @@ async fn test_read_filter_data_filter() {
.timestamp_range(200, 300) .timestamp_range(200, 300)
.add_expr(col("state").eq(lit("CA"))) // state=CA .add_expr(col("state").eq(lit("CA"))) // state=CA
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [200], values: [90.0]", "Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [200], values: [90.0]",
@ -289,6 +296,7 @@ async fn test_read_filter_data_filter() {
.timestamp_range(200, 300) .timestamp_range(200, 300)
.add_expr(col("state").not_eq(lit("MA"))) // state=CA .add_expr(col("state").not_eq(lit("MA"))) // state=CA
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
run_read_filter_test_case(TwoMeasurementsMultiSeries {}, predicate, expected_results).await; run_read_filter_test_case(TwoMeasurementsMultiSeries {}, predicate, expected_results).await;
} }
@ -301,6 +309,8 @@ async fn test_read_filter_data_filter_with_delete() {
.add_expr(col("state").eq(lit("CA"))) // state=CA .add_expr(col("state").eq(lit("CA"))) // state=CA
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![]; let expected_results = vec![];
run_read_filter_test_case( run_read_filter_test_case(
@ -316,6 +326,8 @@ async fn test_read_filter_data_filter_with_delete() {
.add_expr(col("state").not_eq(lit("MA"))) // state=CA .add_expr(col("state").not_eq(lit("MA"))) // state=CA
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
run_read_filter_test_case( run_read_filter_test_case(
TwoMeasurementsMultiSeriesWithDelete {}, TwoMeasurementsMultiSeriesWithDelete {},
predicate, predicate,
@ -330,6 +342,8 @@ async fn test_read_filter_data_filter_with_delete() {
.add_expr(col("_measurement").eq(lit("h2o"))) .add_expr(col("_measurement").eq(lit("h2o")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100], values: [70.4]", "Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [100], values: [70.4]",
]; ];
@ -349,6 +363,7 @@ async fn test_read_filter_data_filter_fields() {
.field_columns(vec!["other_temp"]) .field_columns(vec!["other_temp"])
.add_expr(col("state").eq(lit("CA"))) // state=CA .add_expr(col("state").eq(lit("CA"))) // state=CA
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
// Only expect other_temp in this location // Only expect other_temp in this location
let expected_results = vec![ let expected_results = vec![
@ -367,6 +382,7 @@ async fn test_read_filter_data_filter_measurement_pred() {
.timestamp_range(200, 400) .timestamp_range(200, 400)
.add_expr(col("_measurement").eq(lit("o2"))) .add_expr(col("_measurement").eq(lit("o2")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
// Only expect other_temp in this location // Only expect other_temp in this location
let expected_results = vec![ let expected_results = vec![
@ -381,6 +397,7 @@ async fn test_read_filter_data_pred_refers_to_non_existent_column() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.add_expr(col("tag_not_in_h20").eq(lit("foo"))) .add_expr(col("tag_not_in_h20").eq(lit("foo")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![] as Vec<&str>; let expected_results = vec![] as Vec<&str>;
@ -392,6 +409,7 @@ async fn test_read_filter_data_pred_refers_to_non_existent_column_with_delete()
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.add_expr(col("tag_not_in_h20").eq(lit("foo"))) .add_expr(col("tag_not_in_h20").eq(lit("foo")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![] as Vec<&str>; let expected_results = vec![] as Vec<&str>;
@ -404,6 +422,7 @@ async fn test_read_filter_data_pred_no_columns() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.add_expr(lit("foo").eq(lit("foo"))) .add_expr(lit("foo").eq(lit("foo")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=cpu, region=west, _field=user}\n FloatPoints timestamps: [100, 150], values: [23.2, 21.0]", "Series tags={_measurement=cpu, region=west, _field=user}\n FloatPoints timestamps: [100, 150], values: [23.2, 21.0]",
@ -419,6 +438,7 @@ async fn test_read_filter_data_pred_no_columns_with_delete() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.add_expr(lit("foo").eq(lit("foo"))) .add_expr(lit("foo").eq(lit("foo")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=cpu, region=west, _field=user}\n FloatPoints timestamps: [100], values: [23.2]", "Series tags={_measurement=cpu, region=west, _field=user}\n FloatPoints timestamps: [100], values: [23.2]",
@ -434,6 +454,7 @@ async fn test_read_filter_data_pred_no_columns_with_delete_all() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.add_expr(lit("foo").eq(lit("foo"))) .add_expr(lit("foo").eq(lit("foo")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
// Only table disk has no deleted data // Only table disk has no deleted data
let expected_results = vec![ let expected_results = vec![
@ -450,6 +471,7 @@ async fn test_read_filter_data_pred_refers_to_good_and_non_existent_columns() {
.add_expr(col("state").eq(lit("MA"))) .add_expr(col("state").eq(lit("MA")))
.add_expr(col("tag_not_in_h20").eq(lit("foo"))) .add_expr(col("tag_not_in_h20").eq(lit("foo")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![] as Vec<&str>; let expected_results = vec![] as Vec<&str>;
@ -475,6 +497,7 @@ async fn test_read_filter_data_pred_using_regex_match() {
// will match CA state // will match CA state
.build_regex_match_expr("state", "C.*") .build_regex_match_expr("state", "C.*")
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [200], values: [90.0]", "Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [200], values: [90.0]",
@ -490,6 +513,7 @@ async fn test_read_filter_data_pred_using_regex_match_with_delete() {
// will match CA state // will match CA state
.build_regex_match_expr("state", "C.*") .build_regex_match_expr("state", "C.*")
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
// the selected row was soft deleted // the selected row was soft deleted
let expected_results = vec![]; let expected_results = vec![];
@ -506,6 +530,7 @@ async fn test_read_filter_data_pred_using_regex_match_with_delete() {
// will match CA state // will match CA state
.build_regex_match_expr("state", "C.*") .build_regex_match_expr("state", "C.*")
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [350], values: [90.0]", "Series tags={_measurement=h2o, city=LA, state=CA, _field=temp}\n FloatPoints timestamps: [350], values: [90.0]",
@ -534,6 +559,7 @@ async fn test_read_filter_data_pred_using_regex_not_match() {
// will filter out any rows with a state that matches "CA" // will filter out any rows with a state that matches "CA"
.build_regex_not_match_expr("state", "C.*") .build_regex_not_match_expr("state", "C.*")
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [250], values: [72.4]", "Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [250], values: [72.4]",
@ -550,6 +576,7 @@ async fn test_read_filter_data_pred_regex_escape() {
// Came from InfluxQL like `SELECT value FROM db0.rp0.status_code WHERE url =~ /https\:\/\/influxdb\.com/`, // Came from InfluxQL like `SELECT value FROM db0.rp0.status_code WHERE url =~ /https\:\/\/influxdb\.com/`,
.build_regex_match_expr("url", r#"https\://influxdb\.com"#) .build_regex_match_expr("url", r#"https\://influxdb\.com"#)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
// expect one series with influxdb.com // expect one series with influxdb.com
let expected_results = vec![ let expected_results = vec![
@ -580,6 +607,7 @@ async fn test_read_filter_data_pred_not_match_regex_escape() {
// Came from InfluxQL like `SELECT value FROM db0.rp0.status_code WHERE url !~ /https\:\/\/influxdb\.com/`, // Came from InfluxQL like `SELECT value FROM db0.rp0.status_code WHERE url !~ /https\:\/\/influxdb\.com/`,
.build_regex_not_match_expr("url", r#"https\://influxdb\.com"#) .build_regex_not_match_expr("url", r#"https\://influxdb\.com"#)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
// expect one series with influxdb.com // expect one series with influxdb.com
let expected_results = vec![ let expected_results = vec![
@ -600,6 +628,7 @@ async fn test_read_filter_data_pred_unsupported_in_scan() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.add_expr(col("state").eq(lit("CA")).or(col("reading").gt(lit(0)))) .add_expr(col("state").eq(lit("CA")).or(col("reading").gt(lit(0))))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
// Note these results include data from both o2 and h2o // Note these results include data from both o2 and h2o
let expected_results = vec![ let expected_results = vec![
@ -622,6 +651,7 @@ async fn test_read_filter_data_pred_unsupported_in_scan_with_delete() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.add_expr(col("state").eq(lit("CA")).or(col("reading").gt(lit(0)))) .add_expr(col("state").eq(lit("CA")).or(col("reading").gt(lit(0))))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
// Note these results include data from both o2 and h2o // Note these results include data from both o2 and h2o
let expected_results = vec![ let expected_results = vec![
@ -710,7 +740,6 @@ impl DbSetup for MeasurementsSortableTagsWithDelete {
#[tokio::test] #[tokio::test]
async fn test_read_filter_data_plan_order() { async fn test_read_filter_data_plan_order() {
test_helpers::maybe_start_logging(); test_helpers::maybe_start_logging();
let predicate = Predicate::default();
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=h2o, city=Boston, state=CA, _field=temp}\n FloatPoints timestamps: [250], values: [70.3]", "Series tags={_measurement=h2o, city=Boston, state=CA, _field=temp}\n FloatPoints timestamps: [250], values: [70.3]",
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=other}\n FloatPoints timestamps: [250], values: [5.0]", "Series tags={_measurement=h2o, city=Boston, state=MA, _field=other}\n FloatPoints timestamps: [250], values: [5.0]",
@ -720,13 +749,17 @@ async fn test_read_filter_data_plan_order() {
"Series tags={_measurement=h2o, city=Kingston, state=MA, zz_tag=B, _field=temp}\n FloatPoints timestamps: [100], values: [70.2]", "Series tags={_measurement=h2o, city=Kingston, state=MA, zz_tag=B, _field=temp}\n FloatPoints timestamps: [100], values: [70.2]",
]; ];
run_read_filter_test_case(MeasurementsSortableTags {}, predicate, expected_results).await; run_read_filter_test_case(
MeasurementsSortableTags {},
InfluxRpcPredicate::default(),
expected_results,
)
.await;
} }
#[tokio::test] #[tokio::test]
async fn test_read_filter_data_plan_order_with_delete() { async fn test_read_filter_data_plan_order_with_delete() {
test_helpers::maybe_start_logging(); test_helpers::maybe_start_logging();
let predicate = Predicate::default();
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=other}\n FloatPoints timestamps: [250], values: [5.0]", "Series tags={_measurement=h2o, city=Boston, state=MA, _field=other}\n FloatPoints timestamps: [250], values: [5.0]",
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [250], values: [70.5]", "Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [250], values: [70.5]",
@ -737,7 +770,7 @@ async fn test_read_filter_data_plan_order_with_delete() {
run_read_filter_test_case( run_read_filter_test_case(
MeasurementsSortableTagsWithDelete {}, MeasurementsSortableTagsWithDelete {},
predicate, InfluxRpcPredicate::default(),
expected_results, expected_results,
) )
.await; .await;
@ -772,6 +805,7 @@ async fn test_read_filter_filter_on_value() {
.add_expr(col("_value").eq(lit(1.77))) .add_expr(col("_value").eq(lit(1.77)))
.add_expr(col("_field").eq(lit("load4"))) .add_expr(col("_field").eq(lit("load4")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=system, host=host.local, _field=load4}\n FloatPoints timestamps: [1527018806000000000, 1527018826000000000], values: [1.77, 1.77]", "Series tags={_measurement=system, host=host.local, _field=load4}\n FloatPoints timestamps: [1527018806000000000, 1527018826000000000], values: [1.77, 1.77]",
@ -788,6 +822,7 @@ async fn test_read_filter_on_field() {
// (_field = 'temp') // (_field = 'temp')
let p1 = col("_field").eq(lit("temp")); let p1 = col("_field").eq(lit("temp"));
let predicate = PredicateBuilder::default().add_expr(p1).build(); let predicate = PredicateBuilder::default().add_expr(p1).build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [50, 100000], values: [70.4, 70.4]", "Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [50, 100000], values: [70.4, 70.4]",
@ -808,6 +843,7 @@ async fn test_read_filter_on_field_single_measurement() {
.eq(lit("temp")) .eq(lit("temp"))
.and(col("_measurement").eq(lit("h2o"))); .and(col("_measurement").eq(lit("h2o")));
let predicate = PredicateBuilder::default().add_expr(p1).build(); let predicate = PredicateBuilder::default().add_expr(p1).build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_results = vec![ let expected_results = vec![
"Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [50, 100000], values: [70.4, 70.4]", "Series tags={_measurement=h2o, city=Boston, state=MA, _field=temp}\n FloatPoints timestamps: [50, 100000], values: [70.4, 70.4]",
@ -830,6 +866,7 @@ async fn test_read_filter_on_field_multi_measurement() {
.eq(lit("temp")) .eq(lit("temp"))
.and(col("_measurement").eq(lit("o2"))); .and(col("_measurement").eq(lit("o2")));
let predicate = PredicateBuilder::default().add_expr(p1.or(p2)).build(); let predicate = PredicateBuilder::default().add_expr(p1.or(p2)).build();
let predicate = InfluxRpcPredicate::new(None, predicate);
// SHOULD NOT contain temp from h2o // SHOULD NOT contain temp from h2o
let expected_results = vec![ let expected_results = vec![

View File

@ -16,14 +16,15 @@ use datafusion::{
logical_plan::{binary_expr, Operator}, logical_plan::{binary_expr, Operator},
prelude::*, prelude::*,
}; };
use predicate::predicate::{Predicate, PredicateBuilder}; use predicate::predicate::PredicateBuilder;
use predicate::rpc_predicate::InfluxRpcPredicate;
use query::{frontend::influxrpc::InfluxRpcPlanner, group_by::Aggregate}; use query::{frontend::influxrpc::InfluxRpcPlanner, group_by::Aggregate};
/// runs read_group(predicate) and compares it to the expected /// runs read_group(predicate) and compares it to the expected
/// output /// output
async fn run_read_group_test_case<D>( async fn run_read_group_test_case<D>(
db_setup: D, db_setup: D,
predicate: Predicate, predicate: InfluxRpcPredicate,
agg: Aggregate, agg: Aggregate,
group_columns: Vec<&str>, group_columns: Vec<&str>,
expected_results: Vec<&str>, expected_results: Vec<&str>,
@ -57,12 +58,18 @@ async fn run_read_group_test_case<D>(
#[tokio::test] #[tokio::test]
async fn test_read_group_no_data_no_pred() { async fn test_read_group_no_data_no_pred() {
let predicate = Predicate::default();
let agg = Aggregate::Mean; let agg = Aggregate::Mean;
let group_columns = vec![] as Vec<&str>; let group_columns = vec![] as Vec<&str>;
let expected_results = vec![] as Vec<&str>; let expected_results = vec![] as Vec<&str>;
run_read_group_test_case(NoData {}, predicate, agg, group_columns, expected_results).await; run_read_group_test_case(
NoData {},
InfluxRpcPredicate::default(),
agg,
group_columns,
expected_results,
)
.await;
} }
struct OneMeasurementNoTags {} struct OneMeasurementNoTags {}
@ -137,7 +144,6 @@ impl DbSetup for OneMeasurementNoTagsWithDeleteAllWithAndWithoutChunk {
#[tokio::test] #[tokio::test]
async fn test_read_group_data_no_tag_columns() { async fn test_read_group_data_no_tag_columns() {
let predicate = Predicate::default();
// Count // Count
let agg = Aggregate::Count; let agg = Aggregate::Count;
let group_columns = vec![]; let group_columns = vec![];
@ -148,7 +154,7 @@ async fn test_read_group_data_no_tag_columns() {
run_read_group_test_case( run_read_group_test_case(
OneMeasurementNoTags {}, OneMeasurementNoTags {},
predicate.clone(), InfluxRpcPredicate::default(),
agg, agg,
group_columns.clone(), group_columns.clone(),
expected_results, expected_results,
@ -164,7 +170,7 @@ async fn test_read_group_data_no_tag_columns() {
run_read_group_test_case( run_read_group_test_case(
OneMeasurementNoTags {}, OneMeasurementNoTags {},
predicate, InfluxRpcPredicate::default(),
agg, agg,
group_columns, group_columns,
expected_results, expected_results,
@ -174,8 +180,6 @@ async fn test_read_group_data_no_tag_columns() {
#[tokio::test] #[tokio::test]
async fn test_read_group_data_no_tag_columns_count_with_delete() { async fn test_read_group_data_no_tag_columns_count_with_delete() {
let predicate = Predicate::default();
let agg = Aggregate::Count; let agg = Aggregate::Count;
let group_columns = vec![]; let group_columns = vec![];
let expected_results = vec![ let expected_results = vec![
@ -184,7 +188,7 @@ async fn test_read_group_data_no_tag_columns_count_with_delete() {
]; ];
run_read_group_test_case( run_read_group_test_case(
OneMeasurementNoTagsWithDelete {}, OneMeasurementNoTagsWithDelete {},
predicate.clone(), InfluxRpcPredicate::default(),
agg, agg,
group_columns.clone(), group_columns.clone(),
expected_results, expected_results,
@ -194,8 +198,6 @@ async fn test_read_group_data_no_tag_columns_count_with_delete() {
#[tokio::test] #[tokio::test]
async fn test_read_group_data_no_tag_columns_min_with_delete() { async fn test_read_group_data_no_tag_columns_min_with_delete() {
let predicate = Predicate::default();
let agg = Aggregate::Min; let agg = Aggregate::Min;
let group_columns = vec![]; let group_columns = vec![];
let expected_results = vec![ let expected_results = vec![
@ -205,7 +207,7 @@ async fn test_read_group_data_no_tag_columns_min_with_delete() {
run_read_group_test_case( run_read_group_test_case(
OneMeasurementNoTagsWithDelete {}, OneMeasurementNoTagsWithDelete {},
predicate.clone(), InfluxRpcPredicate::default(),
agg, agg,
group_columns.clone(), group_columns.clone(),
expected_results, expected_results,
@ -215,14 +217,13 @@ async fn test_read_group_data_no_tag_columns_min_with_delete() {
#[tokio::test] #[tokio::test]
async fn test_read_group_data_no_tag_columns_count_with_delete_all() { async fn test_read_group_data_no_tag_columns_count_with_delete_all() {
let predicate = Predicate::default();
let agg = Aggregate::Count; let agg = Aggregate::Count;
let group_columns = vec![]; let group_columns = vec![];
let expected_results = vec![]; let expected_results = vec![];
run_read_group_test_case( run_read_group_test_case(
OneMeasurementNoTagsWithDeleteAllWithAndWithoutChunk {}, OneMeasurementNoTagsWithDeleteAllWithAndWithoutChunk {},
predicate.clone(), InfluxRpcPredicate::default(),
agg, agg,
group_columns.clone(), group_columns.clone(),
expected_results, expected_results,
@ -232,14 +233,13 @@ async fn test_read_group_data_no_tag_columns_count_with_delete_all() {
#[tokio::test] #[tokio::test]
async fn test_read_group_data_no_tag_columns_min_with_delete_all() { async fn test_read_group_data_no_tag_columns_min_with_delete_all() {
let predicate = Predicate::default();
let agg = Aggregate::Min; let agg = Aggregate::Min;
let group_columns = vec![]; let group_columns = vec![];
let expected_results = vec![]; let expected_results = vec![];
run_read_group_test_case( run_read_group_test_case(
OneMeasurementNoTagsWithDeleteAllWithAndWithoutChunk {}, OneMeasurementNoTagsWithDeleteAllWithAndWithoutChunk {},
predicate, InfluxRpcPredicate::default(),
agg, agg,
group_columns, group_columns,
expected_results, expected_results,
@ -274,6 +274,7 @@ async fn test_read_group_data_pred() {
.add_expr(col("city").eq(lit("LA"))) .add_expr(col("city").eq(lit("LA")))
.timestamp_range(190, 210) .timestamp_range(190, 210)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Sum; let agg = Aggregate::Sum;
let group_columns = vec!["state"]; let group_columns = vec!["state"];
let expected_results = vec![ let expected_results = vec![
@ -297,6 +298,7 @@ async fn test_read_group_data_field_restriction() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.field_columns(vec!["temp"]) .field_columns(vec!["temp"])
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Sum; let agg = Aggregate::Sum;
let group_columns = vec!["state"]; let group_columns = vec!["state"];
let expected_results = vec![ let expected_results = vec![
@ -352,6 +354,7 @@ async fn test_grouped_series_set_plan_sum() {
// fiter out first Cambridge row // fiter out first Cambridge row
.timestamp_range(100, 1000) .timestamp_range(100, 1000)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Sum; let agg = Aggregate::Sum;
let group_columns = vec!["state"]; let group_columns = vec!["state"];
@ -386,6 +389,7 @@ async fn test_grouped_series_set_plan_count() {
// fiter out first Cambridge row // fiter out first Cambridge row
.timestamp_range(100, 1000) .timestamp_range(100, 1000)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Count; let agg = Aggregate::Count;
let group_columns = vec!["state"]; let group_columns = vec!["state"];
@ -420,6 +424,7 @@ async fn test_grouped_series_set_plan_mean() {
// fiter out first Cambridge row // fiter out first Cambridge row
.timestamp_range(100, 1000) .timestamp_range(100, 1000)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Mean; let agg = Aggregate::Mean;
let group_columns = vec!["state"]; let group_columns = vec!["state"];
@ -471,6 +476,7 @@ async fn test_grouped_series_set_plan_count_measurement_pred() {
.or(col("_measurement").eq(lit("o2"))), .or(col("_measurement").eq(lit("o2"))),
) )
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Count; let agg = Aggregate::Count;
let group_columns = vec!["state"]; let group_columns = vec!["state"];
@ -517,6 +523,7 @@ async fn test_grouped_series_set_plan_first() {
// fiter out first row (ts 1000) // fiter out first row (ts 1000)
.timestamp_range(1001, 4001) .timestamp_range(1001, 4001)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::First; let agg = Aggregate::First;
let group_columns = vec!["state"]; let group_columns = vec!["state"];
@ -546,10 +553,10 @@ async fn test_grouped_series_set_plan_first_with_nulls() {
// "h2o,state=MA,city=Boston temp=70.4 50", // "h2o,state=MA,city=Boston temp=70.4 50",
// "h2o,state=MA,city=Boston other_temp=70.4 250", // "h2o,state=MA,city=Boston other_temp=70.4 250",
// "h2o,state=MA,city=Boston temp=70.4,moisture=43.0 100000" // "h2o,state=MA,city=Boston temp=70.4,moisture=43.0 100000"
.table("h2o")
.add_expr(col("state").eq(lit("MA"))) .add_expr(col("state").eq(lit("MA")))
.add_expr(col("city").eq(lit("Boston"))) .add_expr(col("city").eq(lit("Boston")))
.build(); .build();
let predicate = InfluxRpcPredicate::new_table("h2o", predicate);
let agg = Aggregate::First; let agg = Aggregate::First;
let group_columns = vec!["state"]; let group_columns = vec!["state"];
@ -578,6 +585,7 @@ async fn test_grouped_series_set_plan_last() {
// fiter out last row (ts 4000) // fiter out last row (ts 4000)
.timestamp_range(100, 3999) .timestamp_range(100, 3999)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Last; let agg = Aggregate::Last;
let group_columns = vec!["state"]; let group_columns = vec!["state"];
@ -607,10 +615,10 @@ async fn test_grouped_series_set_plan_last_with_nulls() {
// "h2o,state=MA,city=Boston temp=70.4 50", // "h2o,state=MA,city=Boston temp=70.4 50",
// "h2o,state=MA,city=Boston other_temp=70.4 250", // "h2o,state=MA,city=Boston other_temp=70.4 250",
// "h2o,state=MA,city=Boston temp=70.4,moisture=43.0 100000" // "h2o,state=MA,city=Boston temp=70.4,moisture=43.0 100000"
.table("h2o")
.add_expr(col("state").eq(lit("MA"))) .add_expr(col("state").eq(lit("MA")))
.add_expr(col("city").eq(lit("Boston"))) .add_expr(col("city").eq(lit("Boston")))
.build(); .build();
let predicate = InfluxRpcPredicate::new_table("h2o", predicate);
let agg = Aggregate::Last; let agg = Aggregate::Last;
let group_columns = vec!["state"]; let group_columns = vec!["state"];
@ -660,6 +668,7 @@ async fn test_grouped_series_set_plan_min() {
// fiter out last row (ts 4000) // fiter out last row (ts 4000)
.timestamp_range(100, 3999) .timestamp_range(100, 3999)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Min; let agg = Aggregate::Min;
let group_columns = vec!["state"]; let group_columns = vec!["state"];
@ -707,6 +716,7 @@ async fn test_grouped_series_set_plan_max() {
// fiter out first row (ts 1000) // fiter out first row (ts 1000)
.timestamp_range(1001, 4001) .timestamp_range(1001, 4001)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Max; let agg = Aggregate::Max;
let group_columns = vec!["state"]; let group_columns = vec!["state"];
@ -755,9 +765,6 @@ impl DbSetup for MeasurementForGroupKeys {
#[tokio::test] #[tokio::test]
async fn test_grouped_series_set_plan_group_by_state_city() { async fn test_grouped_series_set_plan_group_by_state_city() {
// no predicate
let predicate = PredicateBuilder::default().build();
let agg = Aggregate::Sum; let agg = Aggregate::Sum;
let group_columns = vec!["state", "city"]; let group_columns = vec!["state", "city"];
@ -773,7 +780,7 @@ async fn test_grouped_series_set_plan_group_by_state_city() {
run_read_group_test_case( run_read_group_test_case(
MeasurementForGroupKeys {}, MeasurementForGroupKeys {},
predicate, InfluxRpcPredicate::default(),
agg, agg,
group_columns, group_columns,
expected_results, expected_results,
@ -783,9 +790,6 @@ async fn test_grouped_series_set_plan_group_by_state_city() {
#[tokio::test] #[tokio::test]
async fn test_grouped_series_set_plan_group_by_city_state() { async fn test_grouped_series_set_plan_group_by_city_state() {
// no predicate
let predicate = PredicateBuilder::default().build();
let agg = Aggregate::Sum; let agg = Aggregate::Sum;
let group_columns = vec!["city", "state"]; let group_columns = vec!["city", "state"];
@ -802,7 +806,7 @@ async fn test_grouped_series_set_plan_group_by_city_state() {
run_read_group_test_case( run_read_group_test_case(
MeasurementForGroupKeys {}, MeasurementForGroupKeys {},
predicate, InfluxRpcPredicate::default(),
agg, agg,
group_columns, group_columns,
expected_results, expected_results,
@ -812,9 +816,6 @@ async fn test_grouped_series_set_plan_group_by_city_state() {
#[tokio::test] #[tokio::test]
async fn test_grouped_series_set_plan_group_aggregate_none() { async fn test_grouped_series_set_plan_group_aggregate_none() {
// no predicate
let predicate = PredicateBuilder::default().build();
let agg = Aggregate::None; let agg = Aggregate::None;
let group_columns = vec!["city", "state"]; let group_columns = vec!["city", "state"];
@ -831,7 +832,7 @@ async fn test_grouped_series_set_plan_group_aggregate_none() {
run_read_group_test_case( run_read_group_test_case(
MeasurementForGroupKeys {}, MeasurementForGroupKeys {},
predicate, InfluxRpcPredicate::default(),
agg, agg,
group_columns, group_columns,
expected_results, expected_results,
@ -863,9 +864,6 @@ impl DbSetup for MeasurementForGroupByField {
#[tokio::test] #[tokio::test]
async fn test_grouped_series_set_plan_group_by_field_none() { async fn test_grouped_series_set_plan_group_by_field_none() {
// no predicate
let predicate = PredicateBuilder::default().build();
let agg = Aggregate::None; let agg = Aggregate::None;
let group_columns = vec!["_field"]; let group_columns = vec!["_field"];
@ -886,7 +884,7 @@ async fn test_grouped_series_set_plan_group_by_field_none() {
run_read_group_test_case( run_read_group_test_case(
MeasurementForGroupByField {}, MeasurementForGroupByField {},
predicate, InfluxRpcPredicate::default(),
agg, agg,
group_columns, group_columns,
expected_results, expected_results,
@ -896,9 +894,6 @@ async fn test_grouped_series_set_plan_group_by_field_none() {
#[tokio::test] #[tokio::test]
async fn test_grouped_series_set_plan_group_by_field_and_tag_none() { async fn test_grouped_series_set_plan_group_by_field_and_tag_none() {
// no predicate
let predicate = PredicateBuilder::default().build();
let agg = Aggregate::None; let agg = Aggregate::None;
let group_columns = vec!["_field", "region"]; let group_columns = vec!["_field", "region"];
@ -923,7 +918,7 @@ async fn test_grouped_series_set_plan_group_by_field_and_tag_none() {
run_read_group_test_case( run_read_group_test_case(
MeasurementForGroupByField {}, MeasurementForGroupByField {},
predicate, InfluxRpcPredicate::default(),
agg, agg,
group_columns, group_columns,
expected_results, expected_results,
@ -933,9 +928,6 @@ async fn test_grouped_series_set_plan_group_by_field_and_tag_none() {
#[tokio::test] #[tokio::test]
async fn test_grouped_series_set_plan_group_by_tag_and_field_none() { async fn test_grouped_series_set_plan_group_by_tag_and_field_none() {
// no predicate
let predicate = PredicateBuilder::default().build();
let agg = Aggregate::None; let agg = Aggregate::None;
// note group by the tag first then the field.... Output shoud be // note group by the tag first then the field.... Output shoud be
// sorted on on region first and then _field // sorted on on region first and then _field
@ -960,7 +952,7 @@ async fn test_grouped_series_set_plan_group_by_tag_and_field_none() {
run_read_group_test_case( run_read_group_test_case(
MeasurementForGroupByField {}, MeasurementForGroupByField {},
predicate, InfluxRpcPredicate::default(),
agg, agg,
group_columns, group_columns,
expected_results, expected_results,
@ -970,9 +962,6 @@ async fn test_grouped_series_set_plan_group_by_tag_and_field_none() {
#[tokio::test] #[tokio::test]
async fn test_grouped_series_set_plan_group_measurement_tag_count() { async fn test_grouped_series_set_plan_group_measurement_tag_count() {
// no predicate
let predicate = PredicateBuilder::default().build();
let agg = Aggregate::Count; let agg = Aggregate::Count;
let group_columns = vec!["_measurement", "region"]; let group_columns = vec!["_measurement", "region"];
@ -994,7 +983,7 @@ async fn test_grouped_series_set_plan_group_measurement_tag_count() {
run_read_group_test_case( run_read_group_test_case(
MeasurementForGroupByField {}, MeasurementForGroupByField {},
predicate, InfluxRpcPredicate::default(),
agg, agg,
group_columns, group_columns,
expected_results, expected_results,
@ -1004,8 +993,7 @@ async fn test_grouped_series_set_plan_group_measurement_tag_count() {
#[tokio::test] #[tokio::test]
async fn test_grouped_series_set_plan_group_field_start_stop() { async fn test_grouped_series_set_plan_group_field_start_stop() {
// no predicate let predicate = InfluxRpcPredicate::new_table("o2", Default::default());
let predicate = PredicateBuilder::default().table("o2").build();
let agg = Aggregate::Count; let agg = Aggregate::Count;
@ -1045,8 +1033,7 @@ async fn test_grouped_series_set_plan_group_field_start_stop() {
#[tokio::test] #[tokio::test]
async fn test_grouped_series_set_plan_group_field_pred_and_null_fields() { async fn test_grouped_series_set_plan_group_field_pred_and_null_fields() {
// no predicate let predicate = InfluxRpcPredicate::new_table("o2", Default::default());
let predicate = PredicateBuilder::default().table("o2").build();
let agg = Aggregate::Count; let agg = Aggregate::Count;
let group_columns = vec!["state", "_field"]; let group_columns = vec!["state", "_field"];
@ -1081,9 +1068,9 @@ async fn test_grouped_series_set_plan_group_field_pred_and_null_fields() {
async fn test_grouped_series_set_plan_group_field_pred_filter_on_field() { async fn test_grouped_series_set_plan_group_field_pred_filter_on_field() {
// no predicate // no predicate
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.table("o2")
.add_expr(col("_field").eq(lit("reading"))) .add_expr(col("_field").eq(lit("reading")))
.build(); .build();
let predicate = InfluxRpcPredicate::new_table("o2", predicate);
let agg = Aggregate::Count; let agg = Aggregate::Count;
let group_columns = vec!["state", "_field"]; let group_columns = vec!["state", "_field"];
@ -1140,6 +1127,8 @@ async fn test_grouped_series_set_plan_group_field_pred_filter_on_value() {
.add_expr(col("_value").eq(lit(1.77))) .add_expr(col("_value").eq(lit(1.77)))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Max; let agg = Aggregate::Max;
let group_columns = vec!["_field"]; let group_columns = vec!["_field"];
@ -1172,6 +1161,8 @@ async fn test_grouped_series_set_plan_group_field_pred_filter_on_multiple_value(
)) ))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Max; let agg = Aggregate::Max;
let group_columns = vec!["_field"]; let group_columns = vec!["_field"];
@ -1202,6 +1193,8 @@ async fn test_grouped_series_set_plan_group_field_pred_filter_on_value_sum() {
.add_expr(col("_value").eq(lit(1.77))) .add_expr(col("_value").eq(lit(1.77)))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Sum; let agg = Aggregate::Sum;
let group_columns = vec!["_field"]; let group_columns = vec!["_field"];

View File

@ -7,7 +7,8 @@ use async_trait::async_trait;
use data_types::{delete_predicate::DeletePredicate, timestamp::TimestampRange}; use data_types::{delete_predicate::DeletePredicate, timestamp::TimestampRange};
use datafusion::prelude::*; use datafusion::prelude::*;
use db::{test_helpers::write_lp, utils::make_db}; use db::{test_helpers::write_lp, utils::make_db};
use predicate::predicate::{Predicate, PredicateBuilder}; use predicate::predicate::PredicateBuilder;
use predicate::rpc_predicate::InfluxRpcPredicate;
use query::{ use query::{
frontend::influxrpc::InfluxRpcPlanner, frontend::influxrpc::InfluxRpcPlanner,
group_by::{Aggregate, WindowDuration}, group_by::{Aggregate, WindowDuration},
@ -17,7 +18,7 @@ use query::{
/// output /// output
async fn run_read_window_aggregate_test_case<D>( async fn run_read_window_aggregate_test_case<D>(
db_setup: D, db_setup: D,
predicate: Predicate, predicate: InfluxRpcPredicate,
agg: Aggregate, agg: Aggregate,
every: WindowDuration, every: WindowDuration,
offset: WindowDuration, offset: WindowDuration,
@ -58,14 +59,20 @@ async fn run_read_window_aggregate_test_case<D>(
#[tokio::test] #[tokio::test]
async fn test_read_window_aggregate_no_data_no_pred() { async fn test_read_window_aggregate_no_data_no_pred() {
let predicate = Predicate::default();
let agg = Aggregate::Mean; let agg = Aggregate::Mean;
let every = WindowDuration::from_nanoseconds(200); let every = WindowDuration::from_nanoseconds(200);
let offset = WindowDuration::from_nanoseconds(0); let offset = WindowDuration::from_nanoseconds(0);
let expected_results = vec![] as Vec<&str>; let expected_results = vec![] as Vec<&str>;
run_read_window_aggregate_test_case(NoData {}, predicate, agg, every, offset, expected_results) run_read_window_aggregate_test_case(
.await; NoData {},
InfluxRpcPredicate::default(),
agg,
every,
offset,
expected_results,
)
.await;
} }
struct MeasurementForWindowAggregate {} struct MeasurementForWindowAggregate {}
@ -107,6 +114,7 @@ async fn test_read_window_aggregate_nanoseconds() {
.add_expr(col("city").eq(lit("Boston")).or(col("city").eq(lit("LA")))) .add_expr(col("city").eq(lit("Boston")).or(col("city").eq(lit("LA"))))
.timestamp_range(100, 450) .timestamp_range(100, 450)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Mean; let agg = Aggregate::Mean;
let every = WindowDuration::from_nanoseconds(200); let every = WindowDuration::from_nanoseconds(200);
@ -140,6 +148,7 @@ async fn test_read_window_aggregate_nanoseconds_measurement_pred() {
) )
.timestamp_range(100, 450) .timestamp_range(100, 450)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Mean; let agg = Aggregate::Mean;
let every = WindowDuration::from_nanoseconds(200); let every = WindowDuration::from_nanoseconds(200);
@ -167,6 +176,7 @@ async fn test_read_window_aggregate_nanoseconds_measurement_count() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.timestamp_range(100, 450) .timestamp_range(100, 450)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Count; let agg = Aggregate::Count;
let every = WindowDuration::from_nanoseconds(200); let every = WindowDuration::from_nanoseconds(200);
@ -247,8 +257,6 @@ impl DbSetup for MeasurementForWindowAggregateMonths {
#[tokio::test] #[tokio::test]
async fn test_read_window_aggregate_months() { async fn test_read_window_aggregate_months() {
let predicate = PredicateBuilder::default().build();
let agg = Aggregate::Mean; let agg = Aggregate::Mean;
let every = WindowDuration::from_months(1, false); let every = WindowDuration::from_months(1, false);
let offset = WindowDuration::from_months(0, false); let offset = WindowDuration::from_months(0, false);
@ -260,7 +268,7 @@ async fn test_read_window_aggregate_months() {
run_read_window_aggregate_test_case( run_read_window_aggregate_test_case(
MeasurementForWindowAggregateMonths {}, MeasurementForWindowAggregateMonths {},
predicate, InfluxRpcPredicate::default(),
agg, agg,
every, every,
offset, offset,
@ -359,6 +367,7 @@ async fn test_grouped_series_set_plan_group_aggregate_min_defect_2697() {
// time >= '2021-01-01T00:00:01.000000001Z' AND time <= '2021-01-01T00:00:01.000000031Z' // time >= '2021-01-01T00:00:01.000000001Z' AND time <= '2021-01-01T00:00:01.000000031Z'
.timestamp_range(1609459201000000001, 1609459201000000031) .timestamp_range(1609459201000000001, 1609459201000000031)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Min; let agg = Aggregate::Min;
let every = WindowDuration::from_nanoseconds(10); let every = WindowDuration::from_nanoseconds(10);
@ -391,6 +400,7 @@ async fn test_grouped_series_set_plan_group_aggregate_min_defect_2697_with_delet
// time >= '2021-01-01T00:00:01.000000001Z' AND time <= '2021-01-01T00:00:01.000000031Z' // time >= '2021-01-01T00:00:01.000000001Z' AND time <= '2021-01-01T00:00:01.000000031Z'
.timestamp_range(1609459201000000001, 1609459201000000031) .timestamp_range(1609459201000000001, 1609459201000000031)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Min; let agg = Aggregate::Min;
let every = WindowDuration::from_nanoseconds(10); let every = WindowDuration::from_nanoseconds(10);
@ -433,6 +443,7 @@ async fn test_grouped_series_set_plan_group_aggregate_sum_defect_2697() {
// time >= '2021-01-01T00:00:01.000000001Z' AND time <= '2021-01-01T00:00:01.000000031Z' // time >= '2021-01-01T00:00:01.000000001Z' AND time <= '2021-01-01T00:00:01.000000031Z'
.timestamp_range(1609459201000000001, 1609459201000000031) .timestamp_range(1609459201000000001, 1609459201000000031)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Sum; let agg = Aggregate::Sum;
let every = WindowDuration::from_nanoseconds(10); let every = WindowDuration::from_nanoseconds(10);
@ -468,6 +479,7 @@ async fn test_grouped_series_set_plan_group_aggregate_filter_on_field() {
.timestamp_range(1609459201000000001, 1609459201000000031) .timestamp_range(1609459201000000001, 1609459201000000031)
.add_expr(col("_field").eq(lit("foo"))) .add_expr(col("_field").eq(lit("foo")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Sum; let agg = Aggregate::Sum;
let every = WindowDuration::from_nanoseconds(10); let every = WindowDuration::from_nanoseconds(10);
@ -497,6 +509,7 @@ async fn test_grouped_series_set_plan_group_aggregate_sum_defect_2697_with_delet
// time >= '2021-01-01T00:00:01.000000001Z' AND time <= '2021-01-01T00:00:01.000000031Z' // time >= '2021-01-01T00:00:01.000000001Z' AND time <= '2021-01-01T00:00:01.000000031Z'
.timestamp_range(1609459201000000001, 1609459201000000031) .timestamp_range(1609459201000000001, 1609459201000000031)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Sum; let agg = Aggregate::Sum;
let every = WindowDuration::from_nanoseconds(10); let every = WindowDuration::from_nanoseconds(10);
@ -564,6 +577,7 @@ async fn test_read_window_aggregate_overflow() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.timestamp_range(1609459201000000001, 1609459201000000024) .timestamp_range(1609459201000000001, 1609459201000000024)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let agg = Aggregate::Max; let agg = Aggregate::Max;
// Note the giant window (every=9223372036854775807) // Note the giant window (every=9223372036854775807)

View File

@ -1,6 +1,7 @@
//! Tests for the Influx gRPC queries //! Tests for the Influx gRPC queries
use datafusion::logical_plan::{col, lit}; use datafusion::logical_plan::{col, lit};
use predicate::predicate::{Predicate, PredicateBuilder, EMPTY_PREDICATE}; use predicate::predicate::PredicateBuilder;
use predicate::rpc_predicate::InfluxRpcPredicate;
use query::{ use query::{
exec::stringset::{IntoStringSet, StringSetRef}, exec::stringset::{IntoStringSet, StringSetRef},
frontend::influxrpc::InfluxRpcPlanner, frontend::influxrpc::InfluxRpcPlanner,
@ -10,8 +11,11 @@ use crate::scenarios::*;
/// runs table_names(predicate) and compares it to the expected /// runs table_names(predicate) and compares it to the expected
/// output /// output
async fn run_table_names_test_case<D>(db_setup: D, predicate: Predicate, expected_names: Vec<&str>) async fn run_table_names_test_case<D>(
where db_setup: D,
predicate: InfluxRpcPredicate,
expected_names: Vec<&str>,
) where
D: DbSetup, D: DbSetup,
{ {
test_helpers::maybe_start_logging(); test_helpers::maybe_start_logging();
@ -47,12 +51,17 @@ where
#[tokio::test] #[tokio::test]
async fn list_table_names_no_data_no_pred() { async fn list_table_names_no_data_no_pred() {
run_table_names_test_case(NoData {}, EMPTY_PREDICATE, vec![]).await; run_table_names_test_case(NoData {}, InfluxRpcPredicate::default(), vec![]).await;
} }
#[tokio::test] #[tokio::test]
async fn list_table_names_no_data_pred() { async fn list_table_names_no_data_pred() {
run_table_names_test_case(TwoMeasurements {}, EMPTY_PREDICATE, vec!["cpu", "disk"]).await; run_table_names_test_case(
TwoMeasurements {},
InfluxRpcPredicate::default(),
vec!["cpu", "disk"],
)
.await;
} }
#[tokio::test] #[tokio::test]
@ -70,12 +79,12 @@ async fn list_table_names_no_data_passes() {
async fn list_table_names_no_non_null_data_passes() { async fn list_table_names_no_non_null_data_passes() {
// only a single row with a null field passes this predicate (expect no table names) // only a single row with a null field passes this predicate (expect no table names)
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.table("o2")
// only get last row of o2 (timestamp = 300) // only get last row of o2 (timestamp = 300)
.timestamp_range(200, 400) .timestamp_range(200, 400)
// model predicate like _field='reading' which last row does not have // model predicate like _field='reading' which last row does not have
.field_columns(vec!["reading"]) .field_columns(vec!["reading"])
.build(); .build();
let predicate = InfluxRpcPredicate::new_table("o2", predicate);
run_table_names_test_case(TwoMeasurementsManyFields {}, predicate, vec![]).await; run_table_names_test_case(TwoMeasurementsManyFields {}, predicate, vec![]).await;
} }
@ -86,7 +95,6 @@ async fn list_table_names_no_non_null_general_data_passes() {
// (expect no table names) -- has a general purpose predicate to // (expect no table names) -- has a general purpose predicate to
// force a generic plan // force a generic plan
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.table("o2")
// only get last row of o2 (timestamp = 300) // only get last row of o2 (timestamp = 300)
.timestamp_range(200, 400) .timestamp_range(200, 400)
// model predicate like _field='reading' which last row does not have // model predicate like _field='reading' which last row does not have
@ -94,6 +102,7 @@ async fn list_table_names_no_non_null_general_data_passes() {
// (state = CA) OR (temp > 50) // (state = CA) OR (temp > 50)
.add_expr(col("state").eq(lit("CA")).or(col("temp").gt(lit(50)))) .add_expr(col("state").eq(lit("CA")).or(col("temp").gt(lit(50))))
.build(); .build();
let predicate = InfluxRpcPredicate::new_table("o2", predicate);
run_table_names_test_case(TwoMeasurementsManyFields {}, predicate, vec![]).await; run_table_names_test_case(TwoMeasurementsManyFields {}, predicate, vec![]).await;
} }
@ -102,7 +111,7 @@ async fn list_table_names_no_non_null_general_data_passes() {
async fn list_table_names_no_data_pred_with_delete() { async fn list_table_names_no_data_pred_with_delete() {
run_table_names_test_case( run_table_names_test_case(
TwoMeasurementsWithDelete {}, TwoMeasurementsWithDelete {},
EMPTY_PREDICATE, InfluxRpcPredicate::default(),
vec!["cpu", "disk"], vec!["cpu", "disk"],
) )
.await; .await;
@ -112,7 +121,7 @@ async fn list_table_names_no_data_pred_with_delete() {
async fn list_table_names_no_data_pred_with_delete_all() { async fn list_table_names_no_data_pred_with_delete_all() {
run_table_names_test_case( run_table_names_test_case(
TwoMeasurementsWithDeleteAll {}, TwoMeasurementsWithDeleteAll {},
EMPTY_PREDICATE, InfluxRpcPredicate::default(),
vec!["disk"], vec!["disk"],
) )
.await; .await;
@ -203,8 +212,11 @@ async fn list_table_names_data_pred_250_300_with_delete_all() {
// https://github.com/influxdata/influxdb_iox/issues/762 // https://github.com/influxdata/influxdb_iox/issues/762
// make a single timestamp predicate between r1 and r2 // make a single timestamp predicate between r1 and r2
fn tsp(r1: i64, r2: i64) -> Predicate { fn tsp(r1: i64, r2: i64) -> InfluxRpcPredicate {
PredicateBuilder::default().timestamp_range(r1, r2).build() InfluxRpcPredicate::new(
None,
PredicateBuilder::default().timestamp_range(r1, r2).build(),
)
} }
fn to_stringset(v: &[&str]) -> StringSetRef { fn to_stringset(v: &[&str]) -> StringSetRef {

View File

@ -1,5 +1,6 @@
use datafusion::logical_plan::{col, lit}; use datafusion::logical_plan::{col, lit};
use predicate::predicate::{Predicate, PredicateBuilder}; use predicate::predicate::PredicateBuilder;
use predicate::rpc_predicate::InfluxRpcPredicate;
use query::{ use query::{
exec::stringset::{IntoStringSet, StringSetRef}, exec::stringset::{IntoStringSet, StringSetRef},
frontend::influxrpc::InfluxRpcPlanner, frontend::influxrpc::InfluxRpcPlanner,
@ -12,8 +13,11 @@ use crate::scenarios::*;
/// ///
/// runs table_column_names(predicate) and compares it to the expected /// runs table_column_names(predicate) and compares it to the expected
/// output /// output
async fn run_tag_keys_test_case<D>(db_setup: D, predicate: Predicate, expected_tag_keys: Vec<&str>) async fn run_tag_keys_test_case<D>(
where db_setup: D,
predicate: InfluxRpcPredicate,
expected_tag_keys: Vec<&str>,
) where
D: DbSetup, D: DbSetup,
{ {
test_helpers::maybe_start_logging(); test_helpers::maybe_start_logging();
@ -48,18 +52,27 @@ where
#[tokio::test] #[tokio::test]
async fn list_tag_columns_with_no_tags() { async fn list_tag_columns_with_no_tags() {
let predicate = PredicateBuilder::default().build(); run_tag_keys_test_case(
run_tag_keys_test_case(OneMeasurementNoTags {}, predicate, vec![]).await; OneMeasurementNoTags {},
InfluxRpcPredicate::default(),
vec![],
)
.await;
let predicate = PredicateBuilder::default().timestamp_range(0, 1000).build(); let predicate = PredicateBuilder::default().timestamp_range(0, 1000).build();
let predicate = InfluxRpcPredicate::new(None, predicate);
run_tag_keys_test_case(OneMeasurementNoTags {}, predicate, vec![]).await; run_tag_keys_test_case(OneMeasurementNoTags {}, predicate, vec![]).await;
} }
#[tokio::test] #[tokio::test]
async fn list_tag_columns_no_predicate() { async fn list_tag_columns_no_predicate() {
let predicate = PredicateBuilder::default().build();
let expected_tag_keys = vec!["borough", "city", "county", "state"]; let expected_tag_keys = vec!["borough", "city", "county", "state"];
run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(
TwoMeasurementsManyNulls {},
InfluxRpcPredicate::default(),
expected_tag_keys,
)
.await;
} }
// NGA todo: add delete tests when TwoMeasurementsManyNullsWithDelete available // NGA todo: add delete tests when TwoMeasurementsManyNullsWithDelete available
@ -69,6 +82,7 @@ async fn list_tag_columns_timestamp() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.timestamp_range(150, 201) .timestamp_range(150, 201)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec!["city", "state"]; let expected_tag_keys = vec!["city", "state"];
run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await;
} }
@ -78,6 +92,7 @@ async fn list_tag_columns_predicate() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.add_expr(col("state").eq(lit("MA"))) // state=MA .add_expr(col("state").eq(lit("MA"))) // state=MA
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec!["city", "county", "state"]; let expected_tag_keys = vec!["city", "county", "state"];
run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await;
} }
@ -91,6 +106,7 @@ async fn list_tag_columns_measurement_pred() {
.timestamp_range(450, 550) .timestamp_range(450, 550)
.add_expr(col("_measurement").eq(lit("o2"))) // _measurement=o2 .add_expr(col("_measurement").eq(lit("o2"))) // _measurement=o2
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec!["city", "state"]; let expected_tag_keys = vec!["city", "state"];
run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await;
} }
@ -101,13 +117,14 @@ async fn list_tag_columns_timestamp_and_predicate() {
.timestamp_range(150, 201) .timestamp_range(150, 201)
.add_expr(col("state").eq(lit("MA"))) // state=MA .add_expr(col("state").eq(lit("MA"))) // state=MA
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec!["city", "state"]; let expected_tag_keys = vec!["city", "state"];
run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await;
} }
#[tokio::test] #[tokio::test]
async fn list_tag_columns_measurement_name() { async fn list_tag_columns_measurement_name() {
let predicate = PredicateBuilder::default().table("o2").build(); let predicate = InfluxRpcPredicate::new_table("o2", Default::default());
let expected_tag_keys = vec!["borough", "city", "state"]; let expected_tag_keys = vec!["borough", "city", "state"];
run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await;
} }
@ -115,9 +132,9 @@ async fn list_tag_columns_measurement_name() {
#[tokio::test] #[tokio::test]
async fn list_tag_columns_measurement_name_and_timestamp() { async fn list_tag_columns_measurement_name_and_timestamp() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.table("o2")
.timestamp_range(150, 201) .timestamp_range(150, 201)
.build(); .build();
let predicate = InfluxRpcPredicate::new_table("o2", predicate);
let expected_tag_keys = vec!["city", "state"]; let expected_tag_keys = vec!["city", "state"];
run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await;
} }
@ -125,9 +142,9 @@ async fn list_tag_columns_measurement_name_and_timestamp() {
#[tokio::test] #[tokio::test]
async fn list_tag_columns_measurement_name_and_predicate() { async fn list_tag_columns_measurement_name_and_predicate() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.table("o2")
.add_expr(col("state").eq(lit("NY"))) // state=NY .add_expr(col("state").eq(lit("NY"))) // state=NY
.build(); .build();
let predicate = InfluxRpcPredicate::new_table("o2", predicate);
let expected_tag_keys = vec!["borough", "city", "state"]; let expected_tag_keys = vec!["borough", "city", "state"];
run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await;
} }
@ -135,10 +152,10 @@ async fn list_tag_columns_measurement_name_and_predicate() {
#[tokio::test] #[tokio::test]
async fn list_tag_columns_measurement_name_and_predicate_and_timestamp() { async fn list_tag_columns_measurement_name_and_predicate_and_timestamp() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.table("o2")
.timestamp_range(1, 550) .timestamp_range(1, 550)
.add_expr(col("state").eq(lit("NY"))) // state=NY .add_expr(col("state").eq(lit("NY"))) // state=NY
.build(); .build();
let predicate = InfluxRpcPredicate::new_table("o2", predicate);
let expected_tag_keys = vec!["city", "state"]; let expected_tag_keys = vec!["city", "state"];
run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(TwoMeasurementsManyNulls {}, predicate, expected_tag_keys).await;
} }
@ -149,6 +166,7 @@ async fn list_tag_name_end_to_end() {
.timestamp_range(0, 10000) .timestamp_range(0, 10000)
.add_expr(col("host").eq(lit("server01"))) .add_expr(col("host").eq(lit("server01")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec!["host", "name", "region"]; let expected_tag_keys = vec!["host", "name", "region"];
run_tag_keys_test_case(EndToEndTest {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(EndToEndTest {}, predicate, expected_tag_keys).await;
} }
@ -159,6 +177,7 @@ async fn list_tag_name_end_to_end_with_delete() {
.timestamp_range(0, 10000) .timestamp_range(0, 10000)
.add_expr(col("host").eq(lit("server01"))) .add_expr(col("host").eq(lit("server01")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec!["host", "region"]; let expected_tag_keys = vec!["host", "region"];
run_tag_keys_test_case(EndToEndTestWithDelete {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(EndToEndTestWithDelete {}, predicate, expected_tag_keys).await;
} }
@ -169,6 +188,7 @@ async fn list_tag_name_max_time() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.timestamp_range(-9223372036854775806, 9223372036854775806) .timestamp_range(-9223372036854775806, 9223372036854775806)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec!["host"]; let expected_tag_keys = vec!["host"];
run_tag_keys_test_case(MeasurementWithMaxTime {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(MeasurementWithMaxTime {}, predicate, expected_tag_keys).await;
} }
@ -180,6 +200,7 @@ async fn list_tag_name_max_i64() {
// outside valid timestamp range // outside valid timestamp range
.timestamp_range(i64::MIN, i64::MAX) .timestamp_range(i64::MIN, i64::MAX)
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec!["host"]; let expected_tag_keys = vec!["host"];
run_tag_keys_test_case(MeasurementWithMaxTime {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(MeasurementWithMaxTime {}, predicate, expected_tag_keys).await;
} }
@ -190,6 +211,7 @@ async fn list_tag_name_max_time_less_one() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.timestamp_range(-9223372036854775806, 9223372036854775805) // one less than max timestamp .timestamp_range(-9223372036854775806, 9223372036854775805) // one less than max timestamp
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec![]; let expected_tag_keys = vec![];
run_tag_keys_test_case(MeasurementWithMaxTime {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(MeasurementWithMaxTime {}, predicate, expected_tag_keys).await;
} }
@ -200,6 +222,7 @@ async fn list_tag_name_max_time_greater_one() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.timestamp_range(-9223372036854775805, 9223372036854775806) // one more than min timestamp .timestamp_range(-9223372036854775805, 9223372036854775806) // one more than min timestamp
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec![]; let expected_tag_keys = vec![];
run_tag_keys_test_case(MeasurementWithMaxTime {}, predicate, expected_tag_keys).await; run_tag_keys_test_case(MeasurementWithMaxTime {}, predicate, expected_tag_keys).await;
} }

View File

@ -1,5 +1,6 @@
use datafusion::logical_plan::{col, lit}; use datafusion::logical_plan::{col, lit};
use predicate::predicate::{Predicate, PredicateBuilder}; use predicate::predicate::PredicateBuilder;
use predicate::rpc_predicate::InfluxRpcPredicate;
use query::{ use query::{
exec::stringset::{IntoStringSet, StringSetRef}, exec::stringset::{IntoStringSet, StringSetRef},
frontend::influxrpc::InfluxRpcPlanner, frontend::influxrpc::InfluxRpcPlanner,
@ -12,7 +13,7 @@ use crate::scenarios::*;
async fn run_tag_values_test_case<D>( async fn run_tag_values_test_case<D>(
db_setup: D, db_setup: D,
tag_name: &str, tag_name: &str,
predicate: Predicate, predicate: InfluxRpcPredicate,
expected_tag_values: Vec<&str>, expected_tag_values: Vec<&str>,
) where ) where
D: DbSetup, D: DbSetup,
@ -50,6 +51,7 @@ async fn run_tag_values_test_case<D>(
#[tokio::test] #[tokio::test]
async fn list_tag_values_no_tag() { async fn list_tag_values_no_tag() {
let predicate = PredicateBuilder::default().build(); let predicate = PredicateBuilder::default().build();
let predicate = InfluxRpcPredicate::new(None, predicate);
// If the tag is not present, expect no values back (not error) // If the tag is not present, expect no values back (not error)
let tag_name = "tag_not_in_chunks"; let tag_name = "tag_not_in_chunks";
let expected_tag_keys = vec![]; let expected_tag_keys = vec![];
@ -66,13 +68,12 @@ async fn list_tag_values_no_tag() {
#[tokio::test] #[tokio::test]
async fn list_tag_values_no_predicate_state_col() { async fn list_tag_values_no_predicate_state_col() {
let predicate = PredicateBuilder::default().build();
let tag_name = "state"; let tag_name = "state";
let expected_tag_keys = vec!["CA", "MA", "NY"]; let expected_tag_keys = vec!["CA", "MA", "NY"];
run_tag_values_test_case( run_tag_values_test_case(
TwoMeasurementsManyNulls {}, TwoMeasurementsManyNulls {},
tag_name, tag_name,
predicate, InfluxRpcPredicate::default(),
expected_tag_keys, expected_tag_keys,
) )
.await; .await;
@ -80,13 +81,12 @@ async fn list_tag_values_no_predicate_state_col() {
#[tokio::test] #[tokio::test]
async fn list_tag_values_no_predicate_state_col_with_delete() { async fn list_tag_values_no_predicate_state_col_with_delete() {
let predicate = PredicateBuilder::default().build();
let tag_name = "state"; let tag_name = "state";
let expected_tag_keys = vec!["CA", "MA"]; let expected_tag_keys = vec!["CA", "MA"];
run_tag_values_test_case( run_tag_values_test_case(
OneMeasurementManyNullTagsWithDelete {}, OneMeasurementManyNullTagsWithDelete {},
tag_name, tag_name,
predicate, InfluxRpcPredicate::default(),
expected_tag_keys, expected_tag_keys,
) )
.await; .await;
@ -94,13 +94,12 @@ async fn list_tag_values_no_predicate_state_col_with_delete() {
#[tokio::test] #[tokio::test]
async fn list_tag_values_no_predicate_state_col_with_delete_all() { async fn list_tag_values_no_predicate_state_col_with_delete_all() {
let predicate = PredicateBuilder::default().build();
let tag_name = "state"; let tag_name = "state";
let expected_tag_keys = vec![]; let expected_tag_keys = vec![];
run_tag_values_test_case( run_tag_values_test_case(
OneMeasurementManyNullTagsWithDeleteAll {}, OneMeasurementManyNullTagsWithDeleteAll {},
tag_name, tag_name,
predicate, InfluxRpcPredicate::default(),
expected_tag_keys, expected_tag_keys,
) )
.await; .await;
@ -109,12 +108,11 @@ async fn list_tag_values_no_predicate_state_col_with_delete_all() {
#[tokio::test] #[tokio::test]
async fn list_tag_values_no_predicate_city_col() { async fn list_tag_values_no_predicate_city_col() {
let tag_name = "city"; let tag_name = "city";
let predicate = PredicateBuilder::default().build();
let expected_tag_keys = vec!["Boston", "LA", "NYC"]; let expected_tag_keys = vec!["Boston", "LA", "NYC"];
run_tag_values_test_case( run_tag_values_test_case(
TwoMeasurementsManyNulls {}, TwoMeasurementsManyNulls {},
tag_name, tag_name,
predicate, InfluxRpcPredicate::default(),
expected_tag_keys, expected_tag_keys,
) )
.await; .await;
@ -124,6 +122,7 @@ async fn list_tag_values_no_predicate_city_col() {
async fn list_tag_values_timestamp_pred_state_col() { async fn list_tag_values_timestamp_pred_state_col() {
let tag_name = "state"; let tag_name = "state";
let predicate = PredicateBuilder::default().timestamp_range(50, 201).build(); let predicate = PredicateBuilder::default().timestamp_range(50, 201).build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec!["CA", "MA"]; let expected_tag_keys = vec!["CA", "MA"];
run_tag_values_test_case( run_tag_values_test_case(
TwoMeasurementsManyNulls {}, TwoMeasurementsManyNulls {},
@ -140,6 +139,7 @@ async fn list_tag_values_state_pred_state_col() {
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.add_expr(col("state").eq(lit("MA"))) // state=MA .add_expr(col("state").eq(lit("MA"))) // state=MA
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec!["Boston"]; let expected_tag_keys = vec!["Boston"];
run_tag_values_test_case( run_tag_values_test_case(
TwoMeasurementsManyNulls {}, TwoMeasurementsManyNulls {},
@ -157,6 +157,7 @@ async fn list_tag_values_timestamp_and_state_pred_state_col() {
.timestamp_range(150, 301) .timestamp_range(150, 301)
.add_expr(col("state").eq(lit("MA"))) // state=MA .add_expr(col("state").eq(lit("MA"))) // state=MA
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec!["MA"]; let expected_tag_keys = vec!["MA"];
run_tag_values_test_case( run_tag_values_test_case(
TwoMeasurementsManyNulls {}, TwoMeasurementsManyNulls {},
@ -170,7 +171,7 @@ async fn list_tag_values_timestamp_and_state_pred_state_col() {
#[tokio::test] #[tokio::test]
async fn list_tag_values_table_pred_state_col() { async fn list_tag_values_table_pred_state_col() {
let tag_name = "state"; let tag_name = "state";
let predicate = PredicateBuilder::default().table("h2o").build(); let predicate = InfluxRpcPredicate::new_table("h2o", Default::default());
let expected_tag_keys = vec!["CA", "MA"]; let expected_tag_keys = vec!["CA", "MA"];
run_tag_values_test_case( run_tag_values_test_case(
TwoMeasurementsManyNulls {}, TwoMeasurementsManyNulls {},
@ -184,7 +185,7 @@ async fn list_tag_values_table_pred_state_col() {
#[tokio::test] #[tokio::test]
async fn list_tag_values_table_pred_city_col() { async fn list_tag_values_table_pred_city_col() {
let tag_name = "city"; let tag_name = "city";
let predicate = PredicateBuilder::default().table("o2").build(); let predicate = InfluxRpcPredicate::new_table("o2", Default::default());
let expected_tag_keys = vec!["Boston", "NYC"]; let expected_tag_keys = vec!["Boston", "NYC"];
run_tag_values_test_case( run_tag_values_test_case(
TwoMeasurementsManyNulls {}, TwoMeasurementsManyNulls {},
@ -198,10 +199,8 @@ async fn list_tag_values_table_pred_city_col() {
#[tokio::test] #[tokio::test]
async fn list_tag_values_table_and_timestamp_and_table_pred_state_col() { async fn list_tag_values_table_and_timestamp_and_table_pred_state_col() {
let tag_name = "state"; let tag_name = "state";
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default().timestamp_range(50, 201).build();
.table("o2") let predicate = InfluxRpcPredicate::new_table("o2", predicate);
.timestamp_range(50, 201)
.build();
let expected_tag_keys = vec!["MA"]; let expected_tag_keys = vec!["MA"];
run_tag_values_test_case( run_tag_values_test_case(
TwoMeasurementsManyNulls {}, TwoMeasurementsManyNulls {},
@ -216,9 +215,9 @@ async fn list_tag_values_table_and_timestamp_and_table_pred_state_col() {
async fn list_tag_values_table_and_state_pred_state_col() { async fn list_tag_values_table_and_state_pred_state_col() {
let tag_name = "state"; let tag_name = "state";
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.table("o2")
.add_expr(col("state").eq(lit("NY"))) // state=NY .add_expr(col("state").eq(lit("NY"))) // state=NY
.build(); .build();
let predicate = InfluxRpcPredicate::new_table("o2", predicate);
let expected_tag_keys = vec!["NY"]; let expected_tag_keys = vec!["NY"];
run_tag_values_test_case( run_tag_values_test_case(
TwoMeasurementsManyNulls {}, TwoMeasurementsManyNulls {},
@ -233,10 +232,10 @@ async fn list_tag_values_table_and_state_pred_state_col() {
async fn list_tag_values_table_and_timestamp_and_state_pred_state_col() { async fn list_tag_values_table_and_timestamp_and_state_pred_state_col() {
let tag_name = "state"; let tag_name = "state";
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.table("o2")
.timestamp_range(1, 550) .timestamp_range(1, 550)
.add_expr(col("state").eq(lit("NY"))) // state=NY .add_expr(col("state").eq(lit("NY"))) // state=NY
.build(); .build();
let predicate = InfluxRpcPredicate::new_table("o2", predicate);
let expected_tag_keys = vec!["NY"]; let expected_tag_keys = vec!["NY"];
run_tag_values_test_case( run_tag_values_test_case(
TwoMeasurementsManyNulls {}, TwoMeasurementsManyNulls {},
@ -251,10 +250,10 @@ async fn list_tag_values_table_and_timestamp_and_state_pred_state_col() {
async fn list_tag_values_table_and_timestamp_and_state_pred_state_col_no_rows() { async fn list_tag_values_table_and_timestamp_and_state_pred_state_col_no_rows() {
let tag_name = "state"; let tag_name = "state";
let predicate = PredicateBuilder::default() let predicate = PredicateBuilder::default()
.table("o2")
.timestamp_range(1, 300) // filters out the NY row .timestamp_range(1, 300) // filters out the NY row
.add_expr(col("state").eq(lit("NY"))) // state=NY .add_expr(col("state").eq(lit("NY"))) // state=NY
.build(); .build();
let predicate = InfluxRpcPredicate::new_table("o2", predicate);
let expected_tag_keys = vec![]; let expected_tag_keys = vec![];
run_tag_values_test_case( run_tag_values_test_case(
@ -273,6 +272,7 @@ async fn list_tag_values_measurement_pred() {
.timestamp_range(1, 600) // filters out the NY row .timestamp_range(1, 600) // filters out the NY row
.add_expr(col("_measurement").not_eq(lit("o2"))) .add_expr(col("_measurement").not_eq(lit("o2")))
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec!["CA", "MA"]; let expected_tag_keys = vec!["CA", "MA"];
run_tag_values_test_case( run_tag_values_test_case(
@ -298,6 +298,7 @@ async fn list_tag_values_measurement_pred_and_or() {
.or(col("temp").gt(lit(70.0))), .or(col("temp").gt(lit(70.0))),
) )
.build(); .build();
let predicate = InfluxRpcPredicate::new(None, predicate);
let expected_tag_keys = vec!["Boston", "LA", "NYC"]; let expected_tag_keys = vec!["Boston", "LA", "NYC"];
run_tag_values_test_case( run_tag_values_test_case(
@ -312,19 +313,17 @@ async fn list_tag_values_measurement_pred_and_or() {
#[tokio::test] #[tokio::test]
async fn list_tag_values_field_col() { async fn list_tag_values_field_col() {
let db_setup = TwoMeasurementsManyNulls {}; let db_setup = TwoMeasurementsManyNulls {};
let predicate = PredicateBuilder::default().build();
for scenario in db_setup.make().await { for scenario in db_setup.make().await {
let DbScenario { let DbScenario {
scenario_name, db, .. scenario_name, db, ..
} = scenario; } = scenario;
println!("Running scenario '{}'", scenario_name); println!("Running scenario '{}'", scenario_name);
println!("Predicate: '{:#?}'", predicate);
let planner = InfluxRpcPlanner::new(); let planner = InfluxRpcPlanner::new();
// Test: temp is a field, not a tag // Test: temp is a field, not a tag
let tag_name = "temp"; let tag_name = "temp";
let plan_result = planner.tag_values(db.as_ref(), tag_name, predicate.clone()); let plan_result = planner.tag_values(db.as_ref(), tag_name, InfluxRpcPredicate::default());
assert_eq!( assert_eq!(
plan_result.unwrap_err().to_string(), plan_result.unwrap_err().to_string(),

View File

@ -8,6 +8,7 @@ use db::{
}; };
use metric::{Attributes, Metric, U64Counter}; use metric::{Attributes, Metric, U64Counter};
use predicate::predicate::PredicateBuilder; use predicate::predicate::PredicateBuilder;
use predicate::rpc_predicate::InfluxRpcPredicate;
use query::{ use query::{
exec::{stringset::StringSet, ExecutionContextProvider, ExecutorType}, exec::{stringset::StringSet, ExecutionContextProvider, ExecutorType},
frontend::{influxrpc::InfluxRpcPlanner, sql::SqlQueryPlanner}, frontend::{influxrpc::InfluxRpcPlanner, sql::SqlQueryPlanner},
@ -117,6 +118,7 @@ async fn chunk_pruning_influxrpc() {
// bar < 3.0 // bar < 3.0
.add_expr(col("bar").lt(lit(3.0))) .add_expr(col("bar").lt(lit(3.0)))
.build(); .build();
let rpc_predicate = InfluxRpcPredicate::new(None, predicate);
let mut expected = StringSet::new(); let mut expected = StringSet::new();
expected.insert("cpu".into()); expected.insert("cpu".into());
@ -124,7 +126,7 @@ async fn chunk_pruning_influxrpc() {
let ctx = db.executor().new_context(query::exec::ExecutorType::Query); let ctx = db.executor().new_context(query::exec::ExecutorType::Query);
let plan = InfluxRpcPlanner::new() let plan = InfluxRpcPlanner::new()
.table_names(db.as_ref(), predicate) .table_names(db.as_ref(), rpc_predicate)
.unwrap(); .unwrap();
let result = ctx.to_string_set(plan).await.unwrap(); let result = ctx.to_string_set(plan).await.unwrap();

View File

@ -664,8 +664,8 @@ impl DbSetup for TwoMeasurementsManyFieldsTwoChunks {
]; ];
write_lp(&db, &lp_lines.join("\n")); write_lp(&db, &lp_lines.join("\n"));
assert_eq!(count_mutable_buffer_chunks(&db), 4); assert_eq!(count_mutable_buffer_chunks(&db), 2);
assert_eq!(count_read_buffer_chunks(&db), 2); assert_eq!(count_read_buffer_chunks(&db), 1);
assert_eq!(count_object_store_chunks(&db), 0); assert_eq!(count_object_store_chunks(&db), 0);
vec![DbScenario { vec![DbScenario {

View File

@ -2,8 +2,9 @@
use super::DbScenario; use super::DbScenario;
use data_types::{chunk_metadata::ChunkId, delete_predicate::DeletePredicate}; use data_types::{chunk_metadata::ChunkId, delete_predicate::DeletePredicate};
use db::test_helpers::chunk_ids_rub;
use db::{ use db::{
test_helpers::{read_buffer_table_chunk_ids, write_lp}, test_helpers::write_lp,
utils::{count_mub_table_chunks, count_os_table_chunks, count_rub_table_chunks, make_db}, utils::{count_mub_table_chunks, count_os_table_chunks, count_rub_table_chunks, make_db},
Db, Db,
}; };
@ -373,7 +374,7 @@ pub async fn make_chunk_with_deletes_at_different_stages(
if let ChunkStage::Os = chunk_stage { if let ChunkStage::Os = chunk_stage {
for table in &tables { for table in &tables {
// retrieve its chunk_id first // retrieve its chunk_id first
let rub_chunk_ids = read_buffer_table_chunk_ids(&db, table.as_str(), partition_key); let rub_chunk_ids = chunk_ids_rub(&db, Some(table.as_str()), Some(partition_key));
assert_eq!(rub_chunk_ids.len(), 1); assert_eq!(rub_chunk_ids.len(), 1);
db.unload_read_buffer(table.as_str(), partition_key, rub_chunk_ids[0]) db.unload_read_buffer(table.as_str(), partition_key, rub_chunk_ids[0])
.unwrap(); .unwrap();

View File

@ -1,7 +1,6 @@
//! Tests for the table_names implementation //! Tests for the table_names implementation
use arrow::datatypes::DataType; use arrow::datatypes::DataType;
use predicate::predicate::PredicateBuilder;
use query::{QueryChunk, QueryChunkMeta, QueryDatabase}; use query::{QueryChunk, QueryChunkMeta, QueryDatabase};
use schema::selection::Selection; use schema::selection::Selection;
use schema::{builder::SchemaBuilder, sort::SortKey, Schema, TIME_COLUMN_NAME}; use schema::{builder::SchemaBuilder, sort::SortKey, Schema, TIME_COLUMN_NAME};
@ -35,9 +34,8 @@ async fn run_table_schema_test_case<D>(
// Make sure at least one table has data // Make sure at least one table has data
let mut chunks_with_table = 0; let mut chunks_with_table = 0;
let predicate = PredicateBuilder::new().table(table_name).build();
for chunk in db.chunks(&predicate) { for chunk in db.chunks(table_name, &Default::default()) {
if chunk.table_name().as_ref() == table_name { if chunk.table_name().as_ref() == table_name {
chunks_with_table += 1; chunks_with_table += 1;
let actual_schema = chunk.schema().select(selection).unwrap(); let actual_schema = chunk.schema().select(selection).unwrap();

View File

@ -138,7 +138,7 @@ async fn delete_predicate_preservation() {
// ==================== check: delete predicates ==================== // ==================== check: delete predicates ====================
let closure_check_delete_predicates = |db: &Db| { let closure_check_delete_predicates = |db: &Db| {
for chunk in db.chunks(&Default::default()) { for chunk in db.chunks(table_name, &Default::default()) {
let partition_key = chunk.addr().partition_key.as_ref(); let partition_key = chunk.addr().partition_key.as_ref();
if partition_key == "part_b" { if partition_key == "part_b" {
// Strictly speaking not required because the chunk was persisted AFTER the delete predicate was // Strictly speaking not required because the chunk was persisted AFTER the delete predicate was

View File

@ -5,7 +5,8 @@ use std::io::Read;
// current-thread executor // current-thread executor
use db::Db; use db::Db;
use flate2::read::GzDecoder; use flate2::read::GzDecoder;
use predicate::predicate::{Predicate, PredicateBuilder}; use predicate::predicate::PredicateBuilder;
use predicate::rpc_predicate::InfluxRpcPredicate;
use query::{ use query::{
exec::{Executor, ExecutorType}, exec::{Executor, ExecutorType},
frontend::influxrpc::InfluxRpcPlanner, frontend::influxrpc::InfluxRpcPlanner,
@ -59,11 +60,14 @@ fn execute_benchmark_group(c: &mut Criterion, scenarios: &[DbScenario]) {
let planner = InfluxRpcPlanner::new(); let planner = InfluxRpcPlanner::new();
let predicates = vec![ let predicates = vec![
(PredicateBuilder::default().build(), "no_pred"), (InfluxRpcPredicate::default(), "no_pred"),
( (
PredicateBuilder::default() InfluxRpcPredicate::new(
.add_expr(col("tag3").eq(lit("value49"))) None,
.build(), PredicateBuilder::default()
.add_expr(col("tag3").eq(lit("value49")))
.build(),
),
"with_pred_tag_3=value49", "with_pred_tag_3=value49",
), ),
]; ];
@ -73,7 +77,9 @@ fn execute_benchmark_group(c: &mut Criterion, scenarios: &[DbScenario]) {
let mut group = c.benchmark_group(format!("read_filter/{}", scenario_name)); let mut group = c.benchmark_group(format!("read_filter/{}", scenario_name));
for (predicate, pred_name) in &predicates { for (predicate, pred_name) in &predicates {
let chunks = db.partition_chunk_summaries("2021-04-26T13").len(); let chunks = db
.filtered_chunk_summaries(None, Some("2021-04-26T13"))
.len();
// The number of expected frames, based on the expected number of // The number of expected frames, based on the expected number of
// individual series keys. // individual series keys.
let exp_data_frames = if predicate.is_empty() { 10000 } else { 200 } * chunks; let exp_data_frames = if predicate.is_empty() { 10000 } else { 200 } * chunks;
@ -105,7 +111,7 @@ async fn build_and_execute_plan(
planner: &InfluxRpcPlanner, planner: &InfluxRpcPlanner,
executor: &Executor, executor: &Executor,
db: &Db, db: &Db,
predicate: Predicate, predicate: InfluxRpcPredicate,
exp_data_frames: usize, exp_data_frames: usize,
) { ) {
let plan = planner let plan = planner

View File

@ -5,7 +5,8 @@ use std::io::Read;
// current-thread executor // current-thread executor
use db::Db; use db::Db;
use flate2::read::GzDecoder; use flate2::read::GzDecoder;
use predicate::predicate::{Predicate, PredicateBuilder}; use predicate::predicate::PredicateBuilder;
use predicate::rpc_predicate::InfluxRpcPredicate;
use query::{ use query::{
exec::{Executor, ExecutorType}, exec::{Executor, ExecutorType},
frontend::influxrpc::InfluxRpcPlanner, frontend::influxrpc::InfluxRpcPlanner,
@ -60,11 +61,14 @@ fn execute_benchmark_group(c: &mut Criterion, scenarios: &[DbScenario]) {
let planner = InfluxRpcPlanner::new(); let planner = InfluxRpcPlanner::new();
let predicates = vec![ let predicates = vec![
(PredicateBuilder::default().build(), "no_pred"), (InfluxRpcPredicate::default(), "no_pred"),
( (
PredicateBuilder::default() InfluxRpcPredicate::new(
.add_expr(col("tag3").eq(lit("value49"))) None,
.build(), PredicateBuilder::default()
.add_expr(col("tag3").eq(lit("value49")))
.build(),
),
"with_pred_tag_3=value49", "with_pred_tag_3=value49",
), ),
]; ];
@ -112,7 +116,7 @@ async fn build_and_execute_plan(
planner: &InfluxRpcPlanner, planner: &InfluxRpcPlanner,
executor: &Executor, executor: &Executor,
db: &Db, db: &Db,
predicate: Predicate, predicate: InfluxRpcPredicate,
agg: Aggregate, agg: Aggregate,
group: &[&str], group: &[&str],
exp_frames: usize, exp_frames: usize,

View File

@ -5,7 +5,8 @@ use std::io::Read;
// current-thread executor // current-thread executor
use db::Db; use db::Db;
use flate2::read::GzDecoder; use flate2::read::GzDecoder;
use predicate::predicate::{Predicate, PredicateBuilder}; use predicate::predicate::PredicateBuilder;
use predicate::rpc_predicate::InfluxRpcPredicate;
use query::{ use query::{
exec::{Executor, ExecutorType}, exec::{Executor, ExecutorType},
frontend::influxrpc::InfluxRpcPlanner, frontend::influxrpc::InfluxRpcPlanner,
@ -58,11 +59,14 @@ fn execute_benchmark_group(c: &mut Criterion, scenarios: &[DbScenario]) {
let planner = InfluxRpcPlanner::new(); let planner = InfluxRpcPlanner::new();
let predicates = vec![ let predicates = vec![
(PredicateBuilder::default().build(), "no_pred"), (InfluxRpcPredicate::default(), "no_pred"),
( (
PredicateBuilder::default() InfluxRpcPredicate::new(
.add_expr(col("tag2").eq(lit("value321"))) None,
.build(), PredicateBuilder::default()
.add_expr(col("tag2").eq(lit("value321")))
.build(),
),
"with_pred", "with_pred",
), ),
]; ];
@ -105,7 +109,7 @@ async fn run_tag_values_query(
executor: &Executor, executor: &Executor,
db: &Db, db: &Db,
tag_key: &str, tag_key: &str,
predicate: Predicate, predicate: InfluxRpcPredicate,
) { ) {
let plan = planner let plan = planner
.tag_values(db, tag_key, predicate) .tag_values(db, tag_key, predicate)