feat: expr analyzer for buffer to filter table chunks
parent
1d8d3d66fc
commit
daa3fe700a
|
@ -9,14 +9,28 @@ pub mod paths;
|
||||||
pub mod persister;
|
pub mod persister;
|
||||||
pub mod write_buffer;
|
pub mod write_buffer;
|
||||||
|
|
||||||
|
use anyhow::Context;
|
||||||
use async_trait::async_trait;
|
use async_trait::async_trait;
|
||||||
use data_types::{NamespaceName, TimestampMinMax};
|
use data_types::{NamespaceName, TimestampMinMax};
|
||||||
use datafusion::{catalog::Session, error::DataFusionError, prelude::Expr};
|
use datafusion::{
|
||||||
|
catalog::Session,
|
||||||
|
common::{Column, DFSchema},
|
||||||
|
error::DataFusionError,
|
||||||
|
execution::context::ExecutionProps,
|
||||||
|
physical_expr::{
|
||||||
|
analyze, create_physical_expr,
|
||||||
|
utils::{Guarantee, LiteralGuarantee},
|
||||||
|
AnalysisContext, ExprBoundaries,
|
||||||
|
},
|
||||||
|
prelude::Expr,
|
||||||
|
scalar::ScalarValue,
|
||||||
|
};
|
||||||
|
use hashbrown::{HashMap, HashSet};
|
||||||
use influxdb3_cache::{
|
use influxdb3_cache::{
|
||||||
distinct_cache::{CreateDistinctCacheArgs, DistinctCacheProvider},
|
distinct_cache::{CreateDistinctCacheArgs, DistinctCacheProvider},
|
||||||
last_cache::LastCacheProvider,
|
last_cache::LastCacheProvider,
|
||||||
};
|
};
|
||||||
use influxdb3_catalog::catalog::{Catalog, CatalogSequenceNumber, DatabaseSchema};
|
use influxdb3_catalog::catalog::{Catalog, CatalogSequenceNumber, DatabaseSchema, TableDefinition};
|
||||||
use influxdb3_id::{ColumnId, DbId, ParquetFileId, SerdeVecMap, TableId};
|
use influxdb3_id::{ColumnId, DbId, ParquetFileId, SerdeVecMap, TableId};
|
||||||
use influxdb3_wal::{
|
use influxdb3_wal::{
|
||||||
DistinctCacheDefinition, LastCacheDefinition, SnapshotSequenceNumber, Wal,
|
DistinctCacheDefinition, LastCacheDefinition, SnapshotSequenceNumber, Wal,
|
||||||
|
@ -24,6 +38,8 @@ use influxdb3_wal::{
|
||||||
};
|
};
|
||||||
use iox_query::QueryChunk;
|
use iox_query::QueryChunk;
|
||||||
use iox_time::Time;
|
use iox_time::Time;
|
||||||
|
use observability_deps::tracing::{debug, info, warn};
|
||||||
|
use schema::{InfluxColumnType, TIME_COLUMN_NAME};
|
||||||
use serde::{Deserialize, Serialize};
|
use serde::{Deserialize, Serialize};
|
||||||
use std::{fmt::Debug, sync::Arc, time::Duration};
|
use std::{fmt::Debug, sync::Arc, time::Duration};
|
||||||
use thiserror::Error;
|
use thiserror::Error;
|
||||||
|
@ -41,6 +57,9 @@ pub enum Error {
|
||||||
|
|
||||||
#[error("persister error: {0}")]
|
#[error("persister error: {0}")]
|
||||||
Persister(#[from] persister::Error),
|
Persister(#[from] persister::Error),
|
||||||
|
|
||||||
|
#[error(transparent)]
|
||||||
|
Anyhow(#[from] anyhow::Error),
|
||||||
}
|
}
|
||||||
|
|
||||||
pub type Result<T, E = Error> = std::result::Result<T, E>;
|
pub type Result<T, E = Error> = std::result::Result<T, E>;
|
||||||
|
@ -90,7 +109,16 @@ pub trait Bufferer: Debug + Send + Sync + 'static {
|
||||||
fn wal(&self) -> Arc<dyn Wal>;
|
fn wal(&self) -> Arc<dyn Wal>;
|
||||||
|
|
||||||
/// Returns the parquet files for a given database and table
|
/// Returns the parquet files for a given database and table
|
||||||
fn parquet_files(&self, db_id: DbId, table_id: TableId) -> Vec<ParquetFile>;
|
fn parquet_files(&self, db_id: DbId, table_id: TableId) -> Vec<ParquetFile> {
|
||||||
|
self.parquet_files_filtered(db_id, table_id, &BufferFilter::default())
|
||||||
|
}
|
||||||
|
|
||||||
|
fn parquet_files_filtered(
|
||||||
|
&self,
|
||||||
|
db_id: DbId,
|
||||||
|
table_id: TableId,
|
||||||
|
filter: &BufferFilter,
|
||||||
|
) -> Vec<ParquetFile>;
|
||||||
|
|
||||||
/// A channel to watch for when new persisted snapshots are created
|
/// A channel to watch for when new persisted snapshots are created
|
||||||
fn watch_persisted_snapshots(&self) -> tokio::sync::watch::Receiver<Option<PersistedSnapshot>>;
|
fn watch_persisted_snapshots(&self) -> tokio::sync::watch::Receiver<Option<PersistedSnapshot>>;
|
||||||
|
@ -462,6 +490,153 @@ pub(crate) mod test_help {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
#[derive(Debug, Default)]
|
||||||
|
pub struct BufferFilter {
|
||||||
|
time_lower_bound: Option<i64>,
|
||||||
|
time_upper_bound: Option<i64>,
|
||||||
|
guarantees: HashMap<ColumnId, BufferGuarantee>,
|
||||||
|
}
|
||||||
|
|
||||||
|
#[derive(Debug)]
|
||||||
|
pub struct BufferGuarantee {
|
||||||
|
pub guarantee: Guarantee,
|
||||||
|
pub literals: HashSet<Arc<str>>,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl BufferFilter {
|
||||||
|
pub fn generate(table_def: &Arc<TableDefinition>, exprs: &[Expr]) -> Result<Self> {
|
||||||
|
let mut time_lower_bound = None;
|
||||||
|
let mut time_upper_bound = None;
|
||||||
|
let arrow_schema = table_def.schema.as_arrow();
|
||||||
|
let mut guarantees = HashMap::new();
|
||||||
|
let df_schema = DFSchema::try_from(Arc::clone(&arrow_schema))
|
||||||
|
.context("table schema was not able to convert to datafusion schema")?;
|
||||||
|
let props = ExecutionProps::new();
|
||||||
|
info!(?exprs, "analyzing expressions");
|
||||||
|
for expr in exprs.iter().filter(|e| {
|
||||||
|
matches!(
|
||||||
|
e,
|
||||||
|
Expr::BinaryExpr(_) | Expr::Not(_) | Expr::Between(_) | Expr::InList(_)
|
||||||
|
)
|
||||||
|
}) {
|
||||||
|
let Ok(physical_expr) = create_physical_expr(expr, &df_schema, &props) else {
|
||||||
|
continue;
|
||||||
|
};
|
||||||
|
if expr
|
||||||
|
.column_refs()
|
||||||
|
.contains(&Column::new_unqualified(TIME_COLUMN_NAME))
|
||||||
|
{
|
||||||
|
debug!(">>> has time col expr");
|
||||||
|
let time_col_index = arrow_schema
|
||||||
|
.fields()
|
||||||
|
.iter()
|
||||||
|
.position(|f| f.name() == TIME_COLUMN_NAME)
|
||||||
|
.expect("table should have a time column");
|
||||||
|
// Determine time bounds, if provided:
|
||||||
|
let boundaries = ExprBoundaries::try_new_unbounded(&arrow_schema)
|
||||||
|
.context("unable to create unbounded expr boundaries on incoming expression")?;
|
||||||
|
let analysis = analyze(
|
||||||
|
&physical_expr,
|
||||||
|
AnalysisContext::new(boundaries),
|
||||||
|
&arrow_schema,
|
||||||
|
)
|
||||||
|
.inspect_err(|error| {
|
||||||
|
warn!(?physical_expr, ?arrow_schema, ?error, "failed to analyze")
|
||||||
|
})
|
||||||
|
.context("unable to analyze provided filters")?;
|
||||||
|
// Set the time boundaries by the analyzed expression, if they have not already been
|
||||||
|
// set. If they have been set, we remove the bounds, because it is not clear how to
|
||||||
|
// evaluate multiple intervals.
|
||||||
|
if let Some(ExprBoundaries { interval, .. }) =
|
||||||
|
analysis.boundaries.get(time_col_index)
|
||||||
|
{
|
||||||
|
debug!(?interval, ">>> got the interval");
|
||||||
|
if let ScalarValue::TimestampNanosecond(Some(lower), _) = interval.lower() {
|
||||||
|
if time_lower_bound.take().is_none() {
|
||||||
|
time_lower_bound.replace(*lower);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if let ScalarValue::TimestampNanosecond(Some(upper), _) = interval.upper() {
|
||||||
|
if time_upper_bound.take().is_none() {
|
||||||
|
time_upper_bound.replace(*upper);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Determine any literal guarantees made on tag columns:
|
||||||
|
let literal_guarantees = LiteralGuarantee::analyze(&physical_expr);
|
||||||
|
for LiteralGuarantee {
|
||||||
|
column,
|
||||||
|
guarantee,
|
||||||
|
literals,
|
||||||
|
} in literal_guarantees
|
||||||
|
{
|
||||||
|
// NOTE: only retaining string literals for matching
|
||||||
|
// on tag columns for the buffer index:
|
||||||
|
let Some((column_id, InfluxColumnType::Tag)) = table_def
|
||||||
|
.column_definition(column.name())
|
||||||
|
.map(|def| (def.id, def.data_type))
|
||||||
|
else {
|
||||||
|
continue;
|
||||||
|
};
|
||||||
|
let literals = literals
|
||||||
|
.into_iter()
|
||||||
|
.filter_map(|l| match l {
|
||||||
|
ScalarValue::Utf8(Some(s)) | ScalarValue::Utf8View(Some(s)) => {
|
||||||
|
Some(Arc::<str>::from(s.as_str()))
|
||||||
|
}
|
||||||
|
_ => None,
|
||||||
|
})
|
||||||
|
.collect::<HashSet<Arc<str>>>();
|
||||||
|
guarantees
|
||||||
|
.entry(column_id)
|
||||||
|
.and_modify(|e: &mut BufferGuarantee| {
|
||||||
|
// NOTE: it seems unlikely that there would be
|
||||||
|
// multiple literal guarantees on a single
|
||||||
|
// column from the Expr set. But we handle
|
||||||
|
// that here:
|
||||||
|
use Guarantee::*;
|
||||||
|
match (e.guarantee, guarantee) {
|
||||||
|
(In, In) | (NotIn, NotIn) => {
|
||||||
|
e.literals = e.literals.union(&literals).cloned().collect()
|
||||||
|
}
|
||||||
|
(In, NotIn) => {
|
||||||
|
e.literals = e.literals.difference(&literals).cloned().collect()
|
||||||
|
}
|
||||||
|
(NotIn, In) => {
|
||||||
|
e.literals = literals.difference(&e.literals).cloned().collect()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
})
|
||||||
|
.or_insert(BufferGuarantee {
|
||||||
|
guarantee,
|
||||||
|
literals,
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Ok(Self {
|
||||||
|
time_lower_bound,
|
||||||
|
time_upper_bound,
|
||||||
|
guarantees,
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn test_time_stamp_min_max(&self, min: i64, max: i64) -> bool {
|
||||||
|
match (self.time_lower_bound, self.time_upper_bound) {
|
||||||
|
(None, None) => true,
|
||||||
|
(None, Some(u)) => min <= u,
|
||||||
|
(Some(l), None) => max >= l,
|
||||||
|
(Some(l), Some(u)) => min <= u && max >= l,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn guarantees(&self) -> impl Iterator<Item = (&ColumnId, &BufferGuarantee)> {
|
||||||
|
self.guarantees.iter()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
#[cfg(test)]
|
#[cfg(test)]
|
||||||
mod tests {
|
mod tests {
|
||||||
use influxdb3_catalog::catalog::CatalogSequenceNumber;
|
use influxdb3_catalog::catalog::CatalogSequenceNumber;
|
||||||
|
|
|
@ -12,8 +12,8 @@ use crate::write_buffer::queryable_buffer::QueryableBuffer;
|
||||||
use crate::write_buffer::validator::WriteValidator;
|
use crate::write_buffer::validator::WriteValidator;
|
||||||
use crate::{chunk::ParquetChunk, DatabaseManager};
|
use crate::{chunk::ParquetChunk, DatabaseManager};
|
||||||
use crate::{
|
use crate::{
|
||||||
BufferedWriteRequest, Bufferer, ChunkContainer, DistinctCacheManager, LastCacheManager,
|
BufferFilter, BufferedWriteRequest, Bufferer, ChunkContainer, DistinctCacheManager,
|
||||||
ParquetFile, PersistedSnapshot, Precision, WriteBuffer, WriteLineError,
|
LastCacheManager, ParquetFile, PersistedSnapshot, Precision, WriteBuffer, WriteLineError,
|
||||||
};
|
};
|
||||||
use async_trait::async_trait;
|
use async_trait::async_trait;
|
||||||
use data_types::{
|
use data_types::{
|
||||||
|
@ -319,30 +319,35 @@ impl WriteBufferImpl {
|
||||||
DataFusionError::Execution(format!("database {} not found", database_name))
|
DataFusionError::Execution(format!("database {} not found", database_name))
|
||||||
})?;
|
})?;
|
||||||
|
|
||||||
let (table_id, table_schema) =
|
let table_def = db_schema.table_definition(table_name).ok_or_else(|| {
|
||||||
db_schema.table_id_and_schema(table_name).ok_or_else(|| {
|
|
||||||
DataFusionError::Execution(format!(
|
DataFusionError::Execution(format!(
|
||||||
"table {} not found in db {}",
|
"table {} not found in db {}",
|
||||||
table_name, database_name
|
table_name, database_name
|
||||||
))
|
))
|
||||||
})?;
|
})?;
|
||||||
|
|
||||||
|
let buffer_filter = BufferFilter::generate(&table_def, filters)
|
||||||
|
.inspect_err(|error| warn!(?error, "buffer filter generation failed"))
|
||||||
|
.map_err(|error| DataFusionError::External(Box::new(error)))?;
|
||||||
|
|
||||||
let mut chunks = self.buffer.get_table_chunks(
|
let mut chunks = self.buffer.get_table_chunks(
|
||||||
Arc::clone(&db_schema),
|
Arc::clone(&db_schema),
|
||||||
table_name,
|
table_name,
|
||||||
filters,
|
&buffer_filter,
|
||||||
projection,
|
projection,
|
||||||
ctx,
|
ctx,
|
||||||
)?;
|
)?;
|
||||||
|
|
||||||
let parquet_files = self.persisted_files.get_files(db_schema.id, table_id);
|
let parquet_files =
|
||||||
|
self.persisted_files
|
||||||
|
.get_files(db_schema.id, table_def.table_id, &buffer_filter);
|
||||||
|
|
||||||
let mut chunk_order = chunks.len() as i64;
|
let mut chunk_order = chunks.len() as i64;
|
||||||
|
|
||||||
for parquet_file in parquet_files {
|
for parquet_file in parquet_files {
|
||||||
let parquet_chunk = parquet_chunk_from_file(
|
let parquet_chunk = parquet_chunk_from_file(
|
||||||
&parquet_file,
|
&parquet_file,
|
||||||
&table_schema,
|
&table_def.schema,
|
||||||
self.persister.object_store_url().clone(),
|
self.persister.object_store_url().clone(),
|
||||||
self.persister.object_store(),
|
self.persister.object_store(),
|
||||||
chunk_order,
|
chunk_order,
|
||||||
|
@ -427,8 +432,13 @@ impl Bufferer for WriteBufferImpl {
|
||||||
Arc::clone(&self.wal)
|
Arc::clone(&self.wal)
|
||||||
}
|
}
|
||||||
|
|
||||||
fn parquet_files(&self, db_id: DbId, table_id: TableId) -> Vec<ParquetFile> {
|
fn parquet_files_filtered(
|
||||||
self.buffer.persisted_parquet_files(db_id, table_id)
|
&self,
|
||||||
|
db_id: DbId,
|
||||||
|
table_id: TableId,
|
||||||
|
filter: &BufferFilter,
|
||||||
|
) -> Vec<ParquetFile> {
|
||||||
|
self.buffer.persisted_parquet_files(db_id, table_id, filter)
|
||||||
}
|
}
|
||||||
|
|
||||||
fn watch_persisted_snapshots(&self) -> Receiver<Option<PersistedSnapshot>> {
|
fn watch_persisted_snapshots(&self) -> Receiver<Option<PersistedSnapshot>> {
|
||||||
|
@ -2092,7 +2102,9 @@ mod tests {
|
||||||
verify_snapshot_count(1, &wbuf.persister).await;
|
verify_snapshot_count(1, &wbuf.persister).await;
|
||||||
|
|
||||||
// get the path for the created parquet file:
|
// get the path for the created parquet file:
|
||||||
let persisted_files = wbuf.persisted_files().get_files(db_id, tbl_id);
|
let persisted_files =
|
||||||
|
wbuf.persisted_files()
|
||||||
|
.get_files(db_id, tbl_id, &BufferFilter::default());
|
||||||
assert_eq!(1, persisted_files.len());
|
assert_eq!(1, persisted_files.len());
|
||||||
let path = ObjPath::from(persisted_files[0].path.as_str());
|
let path = ObjPath::from(persisted_files[0].path.as_str());
|
||||||
|
|
||||||
|
@ -2198,7 +2210,9 @@ mod tests {
|
||||||
verify_snapshot_count(1, &wbuf.persister).await;
|
verify_snapshot_count(1, &wbuf.persister).await;
|
||||||
|
|
||||||
// get the path for the created parquet file:
|
// get the path for the created parquet file:
|
||||||
let persisted_files = wbuf.persisted_files().get_files(db_id, tbl_id);
|
let persisted_files =
|
||||||
|
wbuf.persisted_files()
|
||||||
|
.get_files(db_id, tbl_id, &BufferFilter::default());
|
||||||
assert_eq!(1, persisted_files.len());
|
assert_eq!(1, persisted_files.len());
|
||||||
let path = ObjPath::from(persisted_files[0].path.as_str());
|
let path = ObjPath::from(persisted_files[0].path.as_str());
|
||||||
|
|
||||||
|
|
|
@ -2,6 +2,7 @@
|
||||||
//! When queries come in they will combine whatever chunks exist from `QueryableBuffer` with
|
//! When queries come in they will combine whatever chunks exist from `QueryableBuffer` with
|
||||||
//! the persisted files to get the full set of data to query.
|
//! the persisted files to get the full set of data to query.
|
||||||
|
|
||||||
|
use crate::BufferFilter;
|
||||||
use crate::{ParquetFile, PersistedSnapshot};
|
use crate::{ParquetFile, PersistedSnapshot};
|
||||||
use hashbrown::HashMap;
|
use hashbrown::HashMap;
|
||||||
use influxdb3_id::DbId;
|
use influxdb3_id::DbId;
|
||||||
|
@ -47,7 +48,12 @@ impl PersistedFiles {
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Get the list of files for a given database and table, always return in descending order of min_time
|
/// Get the list of files for a given database and table, always return in descending order of min_time
|
||||||
pub fn get_files(&self, db_id: DbId, table_id: TableId) -> Vec<ParquetFile> {
|
pub fn get_files(
|
||||||
|
&self,
|
||||||
|
db_id: DbId,
|
||||||
|
table_id: TableId,
|
||||||
|
filter: &BufferFilter,
|
||||||
|
) -> Vec<ParquetFile> {
|
||||||
let three_days_ago = (self.time_provider.now() - crate::THREE_DAYS).timestamp_nanos();
|
let three_days_ago = (self.time_provider.now() - crate::THREE_DAYS).timestamp_nanos();
|
||||||
let mut files = {
|
let mut files = {
|
||||||
let inner = self.inner.read();
|
let inner = self.inner.read();
|
||||||
|
@ -58,6 +64,7 @@ impl PersistedFiles {
|
||||||
.cloned()
|
.cloned()
|
||||||
.unwrap_or_default()
|
.unwrap_or_default()
|
||||||
.into_iter()
|
.into_iter()
|
||||||
|
.filter(|file| filter.test_time_stamp_min_max(file.min_time, file.max_time))
|
||||||
.filter(|file| dbg!(file.min_time) > dbg!(three_days_ago))
|
.filter(|file| dbg!(file.min_time) > dbg!(three_days_ago))
|
||||||
.collect::<Vec<_>>()
|
.collect::<Vec<_>>()
|
||||||
};
|
};
|
||||||
|
|
|
@ -3,7 +3,7 @@ use crate::paths::ParquetFilePath;
|
||||||
use crate::persister::Persister;
|
use crate::persister::Persister;
|
||||||
use crate::write_buffer::persisted_files::PersistedFiles;
|
use crate::write_buffer::persisted_files::PersistedFiles;
|
||||||
use crate::write_buffer::table_buffer::TableBuffer;
|
use crate::write_buffer::table_buffer::TableBuffer;
|
||||||
use crate::{ParquetFile, ParquetFileId, PersistedSnapshot};
|
use crate::{BufferFilter, ParquetFile, ParquetFileId, PersistedSnapshot};
|
||||||
use anyhow::Context;
|
use anyhow::Context;
|
||||||
use arrow::record_batch::RecordBatch;
|
use arrow::record_batch::RecordBatch;
|
||||||
use async_trait::async_trait;
|
use async_trait::async_trait;
|
||||||
|
@ -13,7 +13,6 @@ use data_types::{
|
||||||
};
|
};
|
||||||
use datafusion::catalog::Session;
|
use datafusion::catalog::Session;
|
||||||
use datafusion::common::DataFusionError;
|
use datafusion::common::DataFusionError;
|
||||||
use datafusion::logical_expr::Expr;
|
|
||||||
use datafusion_util::stream_from_batches;
|
use datafusion_util::stream_from_batches;
|
||||||
use hashbrown::HashMap;
|
use hashbrown::HashMap;
|
||||||
use influxdb3_cache::distinct_cache::DistinctCacheProvider;
|
use influxdb3_cache::distinct_cache::DistinctCacheProvider;
|
||||||
|
@ -100,7 +99,7 @@ impl QueryableBuffer {
|
||||||
&self,
|
&self,
|
||||||
db_schema: Arc<DatabaseSchema>,
|
db_schema: Arc<DatabaseSchema>,
|
||||||
table_name: &str,
|
table_name: &str,
|
||||||
filters: &[Expr],
|
buffer_filter: &BufferFilter,
|
||||||
_projection: Option<&Vec<usize>>,
|
_projection: Option<&Vec<usize>>,
|
||||||
_ctx: &dyn Session,
|
_ctx: &dyn Session,
|
||||||
) -> Result<Vec<Arc<dyn QueryChunk>>, DataFusionError> {
|
) -> Result<Vec<Arc<dyn QueryChunk>>, DataFusionError> {
|
||||||
|
@ -120,7 +119,7 @@ impl QueryableBuffer {
|
||||||
};
|
};
|
||||||
|
|
||||||
Ok(table_buffer
|
Ok(table_buffer
|
||||||
.partitioned_record_batches(Arc::clone(&table_def), filters)
|
.partitioned_record_batches(Arc::clone(&table_def), buffer_filter)
|
||||||
.map_err(|e| DataFusionError::Execution(format!("error getting batches {}", e)))?
|
.map_err(|e| DataFusionError::Execution(format!("error getting batches {}", e)))?
|
||||||
.into_iter()
|
.into_iter()
|
||||||
.filter(|(_, (ts_min_max, _))| {
|
.filter(|(_, (ts_min_max, _))| {
|
||||||
|
@ -416,8 +415,13 @@ impl QueryableBuffer {
|
||||||
receiver
|
receiver
|
||||||
}
|
}
|
||||||
|
|
||||||
pub fn persisted_parquet_files(&self, db_id: DbId, table_id: TableId) -> Vec<ParquetFile> {
|
pub fn persisted_parquet_files(
|
||||||
self.persisted_files.get_files(db_id, table_id)
|
&self,
|
||||||
|
db_id: DbId,
|
||||||
|
table_id: TableId,
|
||||||
|
filter: &BufferFilter,
|
||||||
|
) -> Vec<ParquetFile> {
|
||||||
|
self.persisted_files.get_files(db_id, table_id, filter)
|
||||||
}
|
}
|
||||||
|
|
||||||
pub fn persisted_snapshot_notify_rx(
|
pub fn persisted_snapshot_notify_rx(
|
||||||
|
@ -877,9 +881,11 @@ mod tests {
|
||||||
// validate we have a single persisted file
|
// validate we have a single persisted file
|
||||||
let db = catalog.db_schema("testdb").unwrap();
|
let db = catalog.db_schema("testdb").unwrap();
|
||||||
let table = db.table_definition("foo").unwrap();
|
let table = db.table_definition("foo").unwrap();
|
||||||
let files = queryable_buffer
|
let files = queryable_buffer.persisted_files.get_files(
|
||||||
.persisted_files
|
db.id,
|
||||||
.get_files(db.id, table.table_id);
|
table.table_id,
|
||||||
|
&BufferFilter::default(),
|
||||||
|
);
|
||||||
assert_eq!(files.len(), 1);
|
assert_eq!(files.len(), 1);
|
||||||
|
|
||||||
// now force another snapshot, persisting the data to parquet file
|
// now force another snapshot, persisting the data to parquet file
|
||||||
|
@ -908,9 +914,11 @@ mod tests {
|
||||||
.unwrap();
|
.unwrap();
|
||||||
|
|
||||||
// validate we have two persisted files
|
// validate we have two persisted files
|
||||||
let files = queryable_buffer
|
let files = queryable_buffer.persisted_files.get_files(
|
||||||
.persisted_files
|
db.id,
|
||||||
.get_files(db.id, table.table_id);
|
table.table_id,
|
||||||
|
&BufferFilter::default(),
|
||||||
|
);
|
||||||
assert_eq!(files.len(), 2);
|
assert_eq!(files.len(), 2);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -8,20 +8,22 @@ use arrow::array::{
|
||||||
use arrow::datatypes::{GenericStringType, Int32Type};
|
use arrow::datatypes::{GenericStringType, Int32Type};
|
||||||
use arrow::record_batch::RecordBatch;
|
use arrow::record_batch::RecordBatch;
|
||||||
use data_types::TimestampMinMax;
|
use data_types::TimestampMinMax;
|
||||||
use datafusion::logical_expr::{BinaryExpr, Expr};
|
use datafusion::physical_expr::utils::Guarantee;
|
||||||
use hashbrown::HashMap;
|
use hashbrown::{HashMap, HashSet};
|
||||||
use influxdb3_catalog::catalog::TableDefinition;
|
use influxdb3_catalog::catalog::TableDefinition;
|
||||||
use influxdb3_id::ColumnId;
|
use influxdb3_id::ColumnId;
|
||||||
use influxdb3_wal::{FieldData, Row};
|
use influxdb3_wal::{FieldData, Row};
|
||||||
use observability_deps::tracing::{debug, error};
|
use observability_deps::tracing::error;
|
||||||
use schema::sort::SortKey;
|
use schema::sort::SortKey;
|
||||||
use schema::{InfluxColumnType, InfluxFieldType, Schema, SchemaBuilder};
|
use schema::{InfluxColumnType, InfluxFieldType, Schema, SchemaBuilder};
|
||||||
use std::collections::btree_map::Entry;
|
use std::collections::btree_map::Entry;
|
||||||
use std::collections::{BTreeMap, HashSet};
|
use std::collections::BTreeMap;
|
||||||
use std::mem::size_of;
|
use std::mem::size_of;
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
use thiserror::Error;
|
use thiserror::Error;
|
||||||
|
|
||||||
|
use crate::{BufferFilter, BufferGuarantee};
|
||||||
|
|
||||||
#[derive(Debug, Error)]
|
#[derive(Debug, Error)]
|
||||||
pub enum Error {
|
pub enum Error {
|
||||||
#[error("Field not found in table buffer: {0}")]
|
#[error("Field not found in table buffer: {0}")]
|
||||||
|
@ -71,11 +73,13 @@ impl TableBuffer {
|
||||||
pub fn partitioned_record_batches(
|
pub fn partitioned_record_batches(
|
||||||
&self,
|
&self,
|
||||||
table_def: Arc<TableDefinition>,
|
table_def: Arc<TableDefinition>,
|
||||||
filter: &[Expr],
|
filter: &BufferFilter,
|
||||||
) -> Result<HashMap<i64, (TimestampMinMax, Vec<RecordBatch>)>> {
|
) -> Result<HashMap<i64, (TimestampMinMax, Vec<RecordBatch>)>> {
|
||||||
let mut batches = HashMap::new();
|
let mut batches = HashMap::new();
|
||||||
let schema = table_def.schema.as_arrow();
|
let schema = table_def.schema.as_arrow();
|
||||||
for sc in &self.snapshotting_chunks {
|
for sc in self.snapshotting_chunks.iter().filter(|sc| {
|
||||||
|
filter.test_time_stamp_min_max(sc.timestamp_min_max.min, sc.timestamp_min_max.max)
|
||||||
|
}) {
|
||||||
let cols: std::result::Result<Vec<_>, _> = schema
|
let cols: std::result::Result<Vec<_>, _> = schema
|
||||||
.fields()
|
.fields()
|
||||||
.iter()
|
.iter()
|
||||||
|
@ -95,7 +99,11 @@ impl TableBuffer {
|
||||||
*ts = ts.union(&sc.timestamp_min_max);
|
*ts = ts.union(&sc.timestamp_min_max);
|
||||||
v.push(rb);
|
v.push(rb);
|
||||||
}
|
}
|
||||||
for (t, c) in &self.chunk_time_to_chunks {
|
for (t, c) in self
|
||||||
|
.chunk_time_to_chunks
|
||||||
|
.iter()
|
||||||
|
.filter(|(_, c)| filter.test_time_stamp_min_max(c.timestamp_min, c.timestamp_max))
|
||||||
|
{
|
||||||
let ts_min_max = TimestampMinMax::new(c.timestamp_min, c.timestamp_max);
|
let ts_min_max = TimestampMinMax::new(c.timestamp_min, c.timestamp_max);
|
||||||
let (ts, v) = batches
|
let (ts, v) = batches
|
||||||
.entry(*t)
|
.entry(*t)
|
||||||
|
@ -106,40 +114,6 @@ impl TableBuffer {
|
||||||
Ok(batches)
|
Ok(batches)
|
||||||
}
|
}
|
||||||
|
|
||||||
pub fn record_batches(
|
|
||||||
&self,
|
|
||||||
table_def: Arc<TableDefinition>,
|
|
||||||
filter: &[Expr],
|
|
||||||
) -> Result<Vec<RecordBatch>> {
|
|
||||||
let mut batches =
|
|
||||||
Vec::with_capacity(self.snapshotting_chunks.len() + self.chunk_time_to_chunks.len());
|
|
||||||
let schema = table_def.schema.as_arrow();
|
|
||||||
|
|
||||||
for sc in &self.snapshotting_chunks {
|
|
||||||
let cols: std::result::Result<Vec<_>, _> = schema
|
|
||||||
.fields()
|
|
||||||
.iter()
|
|
||||||
.map(|f| {
|
|
||||||
let col = sc
|
|
||||||
.record_batch
|
|
||||||
.column_by_name(f.name())
|
|
||||||
.ok_or(Error::FieldNotFound(f.name().to_string()));
|
|
||||||
col.cloned()
|
|
||||||
})
|
|
||||||
.collect();
|
|
||||||
let cols = cols?;
|
|
||||||
let rb = RecordBatch::try_new(schema.clone(), cols)?;
|
|
||||||
|
|
||||||
batches.push(rb);
|
|
||||||
}
|
|
||||||
|
|
||||||
for c in self.chunk_time_to_chunks.values() {
|
|
||||||
batches.push(c.record_batch(Arc::clone(&table_def), filter)?)
|
|
||||||
}
|
|
||||||
|
|
||||||
Ok(batches)
|
|
||||||
}
|
|
||||||
|
|
||||||
pub fn timestamp_min_max(&self) -> TimestampMinMax {
|
pub fn timestamp_min_max(&self) -> TimestampMinMax {
|
||||||
let (min, max) = if self.chunk_time_to_chunks.is_empty() {
|
let (min, max) = if self.chunk_time_to_chunks.is_empty() {
|
||||||
(0, 0)
|
(0, 0)
|
||||||
|
@ -265,7 +239,6 @@ impl MutableTableChunk {
|
||||||
self.timestamp_max = self.timestamp_max.max(*v);
|
self.timestamp_max = self.timestamp_max.max(*v);
|
||||||
|
|
||||||
let b = self.data.entry(f.id).or_insert_with(|| {
|
let b = self.data.entry(f.id).or_insert_with(|| {
|
||||||
debug!("Creating new timestamp builder");
|
|
||||||
let mut time_builder = TimestampNanosecondBuilder::new();
|
let mut time_builder = TimestampNanosecondBuilder::new();
|
||||||
// append nulls for all previous rows
|
// append nulls for all previous rows
|
||||||
time_builder.append_nulls(row_index + self.row_count);
|
time_builder.append_nulls(row_index + self.row_count);
|
||||||
|
@ -383,7 +356,6 @@ impl MutableTableChunk {
|
||||||
// add nulls for any columns not present
|
// add nulls for any columns not present
|
||||||
for (name, builder) in &mut self.data {
|
for (name, builder) in &mut self.data {
|
||||||
if !value_added.contains(name) {
|
if !value_added.contains(name) {
|
||||||
debug!("Adding null for column {}", name);
|
|
||||||
match builder {
|
match builder {
|
||||||
Builder::Bool(b) => b.append_null(),
|
Builder::Bool(b) => b.append_null(),
|
||||||
Builder::F64(b) => b.append_null(),
|
Builder::F64(b) => b.append_null(),
|
||||||
|
@ -408,18 +380,16 @@ impl MutableTableChunk {
|
||||||
fn record_batch(
|
fn record_batch(
|
||||||
&self,
|
&self,
|
||||||
table_def: Arc<TableDefinition>,
|
table_def: Arc<TableDefinition>,
|
||||||
filter: &[Expr],
|
filter: &BufferFilter,
|
||||||
) -> Result<RecordBatch> {
|
) -> Result<RecordBatch> {
|
||||||
let row_ids = self
|
let row_ids = self.index.get_rows_from_index_for_filter(filter);
|
||||||
.index
|
|
||||||
.get_rows_from_index_for_filter(Arc::clone(&table_def), filter);
|
|
||||||
let schema = table_def.schema.as_arrow();
|
let schema = table_def.schema.as_arrow();
|
||||||
|
|
||||||
let mut cols = Vec::with_capacity(schema.fields().len());
|
let mut cols = Vec::with_capacity(schema.fields().len());
|
||||||
|
|
||||||
for f in schema.fields() {
|
for f in schema.fields() {
|
||||||
match row_ids {
|
match row_ids {
|
||||||
Some(row_ids) => {
|
Some(ref row_ids) => {
|
||||||
let b = table_def
|
let b = table_def
|
||||||
.column_name_to_id(f.name().as_str())
|
.column_name_to_id(f.name().as_str())
|
||||||
.and_then(|id| self.data.get(&id));
|
.and_then(|id| self.data.get(&id));
|
||||||
|
@ -576,7 +546,7 @@ impl std::fmt::Debug for MutableTableChunk {
|
||||||
#[derive(Debug, Clone)]
|
#[derive(Debug, Clone)]
|
||||||
struct BufferIndex {
|
struct BufferIndex {
|
||||||
// column id -> string value -> row indexes
|
// column id -> string value -> row indexes
|
||||||
columns: HashMap<ColumnId, HashMap<String, Vec<usize>>>,
|
columns: HashMap<ColumnId, HashMap<String, HashSet<usize>>>,
|
||||||
}
|
}
|
||||||
|
|
||||||
impl BufferIndex {
|
impl BufferIndex {
|
||||||
|
@ -594,34 +564,52 @@ impl BufferIndex {
|
||||||
if let Some(column) = self.columns.get_mut(&column_id) {
|
if let Some(column) = self.columns.get_mut(&column_id) {
|
||||||
column
|
column
|
||||||
.entry_ref(value)
|
.entry_ref(value)
|
||||||
.and_modify(|c| c.push(row_index))
|
.and_modify(|c| {
|
||||||
.or_insert(vec![row_index]);
|
c.insert(row_index);
|
||||||
|
})
|
||||||
|
.or_insert([row_index].into_iter().collect());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
fn get_rows_from_index_for_filter(
|
fn get_rows_from_index_for_filter(&self, filter: &BufferFilter) -> Option<HashSet<usize>> {
|
||||||
&self,
|
let mut row_ids = HashSet::new();
|
||||||
table_def: Arc<TableDefinition>,
|
for (
|
||||||
filter: &[Expr],
|
col_id,
|
||||||
) -> Option<&Vec<usize>> {
|
BufferGuarantee {
|
||||||
for expr in filter {
|
guarantee,
|
||||||
if let Expr::BinaryExpr(BinaryExpr { left, op, right }) = expr {
|
literals,
|
||||||
if *op == datafusion::logical_expr::Operator::Eq {
|
},
|
||||||
if let Expr::Column(c) = left.as_ref() {
|
) in filter.guarantees()
|
||||||
if let Expr::Literal(datafusion::scalar::ScalarValue::Utf8(Some(v))) =
|
|
||||||
right.as_ref()
|
|
||||||
{
|
{
|
||||||
return table_def
|
let Some(row_map) = self.columns.get(col_id) else {
|
||||||
.column_name_to_id(c.name())
|
continue;
|
||||||
.and_then(|id| self.columns.get(&id))
|
};
|
||||||
.and_then(|m| m.get(v.as_str()));
|
match guarantee {
|
||||||
|
Guarantee::In => {
|
||||||
|
for literal in literals {
|
||||||
|
let Some(row) = row_map.get(literal.as_ref()) else {
|
||||||
|
continue;
|
||||||
|
};
|
||||||
|
row_ids = row_ids.union(row).copied().collect();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Guarantee::NotIn => {
|
||||||
|
row_ids.extend(row_map.values().flatten().copied());
|
||||||
|
for literal in literals {
|
||||||
|
let Some(row) = row_map.get(literal.as_ref()) else {
|
||||||
|
continue;
|
||||||
|
};
|
||||||
|
row_ids = row_ids.difference(row).copied().collect();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if row_ids.is_empty() {
|
||||||
None
|
None
|
||||||
|
} else {
|
||||||
|
Some(row_ids.into_iter().collect())
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
#[allow(dead_code)]
|
#[allow(dead_code)]
|
||||||
|
@ -695,7 +683,7 @@ impl Builder {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
fn get_rows(&self, rows: &[usize]) -> ArrayRef {
|
fn get_rows(&self, rows: &HashSet<usize>) -> ArrayRef {
|
||||||
match self {
|
match self {
|
||||||
Self::Bool(b) => {
|
Self::Bool(b) => {
|
||||||
let b = b.finish_cloned();
|
let b = b.finish_cloned();
|
||||||
|
@ -795,82 +783,79 @@ impl Builder {
|
||||||
|
|
||||||
#[cfg(test)]
|
#[cfg(test)]
|
||||||
mod tests {
|
mod tests {
|
||||||
|
use crate::{write_buffer::validator::WriteValidator, Precision};
|
||||||
|
|
||||||
use super::*;
|
use super::*;
|
||||||
use arrow_util::{assert_batches_eq, assert_batches_sorted_eq};
|
use arrow_util::assert_batches_sorted_eq;
|
||||||
use datafusion::common::Column;
|
use data_types::NamespaceName;
|
||||||
use influxdb3_id::TableId;
|
use datafusion::prelude::{col, lit, lit_timestamp_nano, Expr};
|
||||||
use influxdb3_wal::Field;
|
use influxdb3_catalog::catalog::{Catalog, DatabaseSchema};
|
||||||
use schema::InfluxFieldType;
|
use iox_time::Time;
|
||||||
|
|
||||||
|
struct TestWriter {
|
||||||
|
catalog: Arc<Catalog>,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl TestWriter {
|
||||||
|
const DB_NAME: &str = "test-db";
|
||||||
|
|
||||||
|
fn new() -> Self {
|
||||||
|
let catalog = Arc::new(Catalog::new("test-node".into(), "test-instance".into()));
|
||||||
|
Self { catalog }
|
||||||
|
}
|
||||||
|
|
||||||
|
fn write_to_rows(&self, lp: impl AsRef<str>, ingest_time_sec: i64) -> Vec<Row> {
|
||||||
|
let db = NamespaceName::try_from(Self::DB_NAME).unwrap();
|
||||||
|
let ingest_time_ns = ingest_time_sec * 1_000_000_000;
|
||||||
|
let validator =
|
||||||
|
WriteValidator::initialize(db, Arc::clone(&self.catalog), ingest_time_ns).unwrap();
|
||||||
|
validator
|
||||||
|
.v1_parse_lines_and_update_schema(
|
||||||
|
lp.as_ref(),
|
||||||
|
false,
|
||||||
|
Time::from_timestamp_nanos(ingest_time_ns),
|
||||||
|
Precision::Nanosecond,
|
||||||
|
)
|
||||||
|
.map(|r| r.into_inner().to_rows())
|
||||||
|
.unwrap()
|
||||||
|
}
|
||||||
|
|
||||||
|
fn db_schema(&self) -> Arc<DatabaseSchema> {
|
||||||
|
self.catalog.db_schema(Self::DB_NAME).unwrap()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
#[test]
|
#[test]
|
||||||
fn partitioned_table_buffer_batches() {
|
fn test_partitioned_table_buffer_batches() {
|
||||||
let table_def = Arc::new(
|
let writer = TestWriter::new();
|
||||||
TableDefinition::new(
|
|
||||||
TableId::new(),
|
|
||||||
"test_table".into(),
|
|
||||||
vec![
|
|
||||||
(ColumnId::from(0), "tag".into(), InfluxColumnType::Tag),
|
|
||||||
(
|
|
||||||
ColumnId::from(1),
|
|
||||||
"val".into(),
|
|
||||||
InfluxColumnType::Field(InfluxFieldType::String),
|
|
||||||
),
|
|
||||||
(
|
|
||||||
ColumnId::from(2),
|
|
||||||
"time".into(),
|
|
||||||
InfluxColumnType::Timestamp,
|
|
||||||
),
|
|
||||||
],
|
|
||||||
vec![0.into()],
|
|
||||||
)
|
|
||||||
.unwrap(),
|
|
||||||
);
|
|
||||||
let mut table_buffer = TableBuffer::new(vec![ColumnId::from(0)], SortKey::empty());
|
|
||||||
|
|
||||||
|
let mut row_batches = Vec::new();
|
||||||
for t in 0..10 {
|
for t in 0..10 {
|
||||||
let offset = t * 10;
|
let offset = t * 10;
|
||||||
let rows = vec![
|
let rows = writer.write_to_rows(
|
||||||
Row {
|
format!(
|
||||||
time: offset + 1,
|
"\
|
||||||
fields: vec![
|
tbl,tag=a val=\"thing {t}-1\" {o1}\n\
|
||||||
Field {
|
tbl,tag=b val=\"thing {t}-2\" {o2}\n\
|
||||||
id: ColumnId::from(0),
|
",
|
||||||
value: FieldData::Tag("a".to_string()),
|
o1 = offset + 1,
|
||||||
},
|
o2 = offset + 2,
|
||||||
Field {
|
),
|
||||||
id: ColumnId::from(1),
|
offset,
|
||||||
value: FieldData::String(format!("thing {t}-1")),
|
);
|
||||||
},
|
row_batches.push((rows, offset));
|
||||||
Field {
|
}
|
||||||
id: ColumnId::from(2),
|
|
||||||
value: FieldData::Timestamp(offset + 1),
|
|
||||||
},
|
|
||||||
],
|
|
||||||
},
|
|
||||||
Row {
|
|
||||||
time: offset + 2,
|
|
||||||
fields: vec![
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(0),
|
|
||||||
value: FieldData::Tag("b".to_string()),
|
|
||||||
},
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(1),
|
|
||||||
value: FieldData::String(format!("thing {t}-2")),
|
|
||||||
},
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(2),
|
|
||||||
value: FieldData::Timestamp(offset + 2),
|
|
||||||
},
|
|
||||||
],
|
|
||||||
},
|
|
||||||
];
|
|
||||||
|
|
||||||
|
let table_def = writer.db_schema().table_definition("tbl").unwrap();
|
||||||
|
let tag_col_id = table_def.column_name_to_id("tag").unwrap();
|
||||||
|
|
||||||
|
let mut table_buffer = TableBuffer::new(vec![tag_col_id], SortKey::empty());
|
||||||
|
for (rows, offset) in row_batches {
|
||||||
table_buffer.buffer_chunk(offset, &rows);
|
table_buffer.buffer_chunk(offset, &rows);
|
||||||
}
|
}
|
||||||
|
|
||||||
let partitioned_batches = table_buffer
|
let partitioned_batches = table_buffer
|
||||||
.partitioned_record_batches(Arc::clone(&table_def), &[])
|
.partitioned_record_batches(Arc::clone(&table_def), &BufferFilter::default())
|
||||||
.unwrap();
|
.unwrap();
|
||||||
|
|
||||||
assert_eq!(10, partitioned_batches.len());
|
assert_eq!(10, partitioned_batches.len());
|
||||||
|
@ -902,209 +887,165 @@ mod tests {
|
||||||
}
|
}
|
||||||
|
|
||||||
#[test]
|
#[test]
|
||||||
fn tag_row_index() {
|
fn test_row_index_tag_filtering() {
|
||||||
let table_def = Arc::new(
|
let writer = TestWriter::new();
|
||||||
TableDefinition::new(
|
let rows = writer.write_to_rows(
|
||||||
TableId::new(),
|
"\
|
||||||
"test_table".into(),
|
tbl,tag=a value=1i 1\n\
|
||||||
vec![
|
tbl,tag=b value=2i 1\n\
|
||||||
(ColumnId::from(0), "tag".into(), InfluxColumnType::Tag),
|
tbl,tag=a value=3i 2\n\
|
||||||
(
|
tbl,tag=b value=4i 2\n\
|
||||||
ColumnId::from(1),
|
tbl,tag=a value=5i 3\n\
|
||||||
"value".into(),
|
tbl,tag=c value=6i 3",
|
||||||
InfluxColumnType::Field(InfluxFieldType::Integer),
|
0,
|
||||||
),
|
|
||||||
(
|
|
||||||
ColumnId::from(2),
|
|
||||||
"time".into(),
|
|
||||||
InfluxColumnType::Timestamp,
|
|
||||||
),
|
|
||||||
],
|
|
||||||
vec![0.into()],
|
|
||||||
)
|
|
||||||
.unwrap(),
|
|
||||||
);
|
);
|
||||||
let mut table_buffer = TableBuffer::new(vec![ColumnId::from(0)], SortKey::empty());
|
let table_def = writer.db_schema().table_definition("tbl").unwrap();
|
||||||
|
let tag_id = table_def.column_name_to_id("tag").unwrap();
|
||||||
let rows = vec![
|
let mut table_buffer = TableBuffer::new(vec![tag_id], SortKey::empty());
|
||||||
Row {
|
|
||||||
time: 1,
|
|
||||||
fields: vec![
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(0),
|
|
||||||
value: FieldData::Tag("a".to_string()),
|
|
||||||
},
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(1),
|
|
||||||
value: FieldData::Integer(1),
|
|
||||||
},
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(2),
|
|
||||||
value: FieldData::Timestamp(1),
|
|
||||||
},
|
|
||||||
],
|
|
||||||
},
|
|
||||||
Row {
|
|
||||||
time: 2,
|
|
||||||
fields: vec![
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(0),
|
|
||||||
value: FieldData::Tag("b".to_string()),
|
|
||||||
},
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(1),
|
|
||||||
value: FieldData::Integer(2),
|
|
||||||
},
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(2),
|
|
||||||
value: FieldData::Timestamp(2),
|
|
||||||
},
|
|
||||||
],
|
|
||||||
},
|
|
||||||
Row {
|
|
||||||
time: 3,
|
|
||||||
fields: vec![
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(0),
|
|
||||||
value: FieldData::Tag("a".to_string()),
|
|
||||||
},
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(1),
|
|
||||||
value: FieldData::Integer(3),
|
|
||||||
},
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(2),
|
|
||||||
value: FieldData::Timestamp(3),
|
|
||||||
},
|
|
||||||
],
|
|
||||||
},
|
|
||||||
];
|
|
||||||
|
|
||||||
table_buffer.buffer_chunk(0, &rows);
|
table_buffer.buffer_chunk(0, &rows);
|
||||||
|
|
||||||
let filter = &[Expr::BinaryExpr(BinaryExpr {
|
struct TestCase<'a> {
|
||||||
left: Box::new(Expr::Column(Column {
|
filter: &'a [Expr],
|
||||||
relation: None,
|
expected_rows: &'a [usize],
|
||||||
name: "tag".to_string(),
|
expected_output: &'a [&'a str],
|
||||||
})),
|
}
|
||||||
op: datafusion::logical_expr::Operator::Eq,
|
|
||||||
right: Box::new(Expr::Literal(datafusion::scalar::ScalarValue::Utf8(Some(
|
|
||||||
"a".to_string(),
|
|
||||||
)))),
|
|
||||||
})];
|
|
||||||
let a_rows = table_buffer
|
|
||||||
.chunk_time_to_chunks
|
|
||||||
.get(&0)
|
|
||||||
.unwrap()
|
|
||||||
.index
|
|
||||||
.get_rows_from_index_for_filter(Arc::clone(&table_def), filter)
|
|
||||||
.unwrap();
|
|
||||||
assert_eq!(a_rows, &[0, 2]);
|
|
||||||
|
|
||||||
let a = table_buffer
|
let test_cases = [
|
||||||
.record_batches(Arc::clone(&table_def), filter)
|
TestCase {
|
||||||
.unwrap();
|
filter: &[col("tag").eq(lit("a"))],
|
||||||
let expected_a = vec![
|
expected_rows: &[0, 2, 4],
|
||||||
|
expected_output: &[
|
||||||
"+-----+--------------------------------+-------+",
|
"+-----+--------------------------------+-------+",
|
||||||
"| tag | time | value |",
|
"| tag | time | value |",
|
||||||
"+-----+--------------------------------+-------+",
|
"+-----+--------------------------------+-------+",
|
||||||
"| a | 1970-01-01T00:00:00.000000001Z | 1 |",
|
"| a | 1970-01-01T00:00:00.000000001Z | 1 |",
|
||||||
"| a | 1970-01-01T00:00:00.000000003Z | 3 |",
|
"| a | 1970-01-01T00:00:00.000000002Z | 3 |",
|
||||||
|
"| a | 1970-01-01T00:00:00.000000003Z | 5 |",
|
||||||
"+-----+--------------------------------+-------+",
|
"+-----+--------------------------------+-------+",
|
||||||
|
],
|
||||||
|
},
|
||||||
|
TestCase {
|
||||||
|
filter: &[col("tag").eq(lit("b"))],
|
||||||
|
expected_rows: &[1, 3],
|
||||||
|
expected_output: &[
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
"| tag | time | value |",
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
"| b | 1970-01-01T00:00:00.000000001Z | 2 |",
|
||||||
|
"| b | 1970-01-01T00:00:00.000000002Z | 4 |",
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
],
|
||||||
|
},
|
||||||
|
TestCase {
|
||||||
|
filter: &[col("tag").eq(lit("c"))],
|
||||||
|
expected_rows: &[5],
|
||||||
|
expected_output: &[
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
"| tag | time | value |",
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
"| c | 1970-01-01T00:00:00.000000003Z | 6 |",
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
],
|
||||||
|
},
|
||||||
|
TestCase {
|
||||||
|
filter: &[col("tag").eq(lit("a")).or(col("tag").eq(lit("c")))],
|
||||||
|
expected_rows: &[0, 2, 4, 5],
|
||||||
|
expected_output: &[
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
"| tag | time | value |",
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
"| a | 1970-01-01T00:00:00.000000001Z | 1 |",
|
||||||
|
"| a | 1970-01-01T00:00:00.000000002Z | 3 |",
|
||||||
|
"| a | 1970-01-01T00:00:00.000000003Z | 5 |",
|
||||||
|
"| c | 1970-01-01T00:00:00.000000003Z | 6 |",
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
],
|
||||||
|
},
|
||||||
|
TestCase {
|
||||||
|
filter: &[col("tag").not_eq(lit("a"))],
|
||||||
|
expected_rows: &[1, 3, 5],
|
||||||
|
expected_output: &[
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
"| tag | time | value |",
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
"| b | 1970-01-01T00:00:00.000000001Z | 2 |",
|
||||||
|
"| b | 1970-01-01T00:00:00.000000002Z | 4 |",
|
||||||
|
"| c | 1970-01-01T00:00:00.000000003Z | 6 |",
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
],
|
||||||
|
},
|
||||||
|
TestCase {
|
||||||
|
filter: &[col("tag").in_list(vec![lit("a"), lit("c")], false)],
|
||||||
|
expected_rows: &[0, 2, 4, 5],
|
||||||
|
expected_output: &[
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
"| tag | time | value |",
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
"| a | 1970-01-01T00:00:00.000000001Z | 1 |",
|
||||||
|
"| a | 1970-01-01T00:00:00.000000002Z | 3 |",
|
||||||
|
"| a | 1970-01-01T00:00:00.000000003Z | 5 |",
|
||||||
|
"| c | 1970-01-01T00:00:00.000000003Z | 6 |",
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
],
|
||||||
|
},
|
||||||
|
TestCase {
|
||||||
|
filter: &[col("tag").in_list(vec![lit("a"), lit("c")], true)],
|
||||||
|
expected_rows: &[1, 3],
|
||||||
|
expected_output: &[
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
"| tag | time | value |",
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
"| b | 1970-01-01T00:00:00.000000001Z | 2 |",
|
||||||
|
"| b | 1970-01-01T00:00:00.000000002Z | 4 |",
|
||||||
|
"+-----+--------------------------------+-------+",
|
||||||
|
],
|
||||||
|
},
|
||||||
];
|
];
|
||||||
assert_batches_eq!(&expected_a, &a);
|
|
||||||
|
|
||||||
let filter = &[Expr::BinaryExpr(BinaryExpr {
|
for t in test_cases {
|
||||||
left: Box::new(Expr::Column(Column {
|
let filter = BufferFilter::generate(&table_def, t.filter).unwrap();
|
||||||
relation: None,
|
let rows = table_buffer
|
||||||
name: "tag".to_string(),
|
|
||||||
})),
|
|
||||||
op: datafusion::logical_expr::Operator::Eq,
|
|
||||||
right: Box::new(Expr::Literal(datafusion::scalar::ScalarValue::Utf8(Some(
|
|
||||||
"b".to_string(),
|
|
||||||
)))),
|
|
||||||
})];
|
|
||||||
|
|
||||||
let b_rows = table_buffer
|
|
||||||
.chunk_time_to_chunks
|
.chunk_time_to_chunks
|
||||||
.get(&0)
|
.get(&0)
|
||||||
.unwrap()
|
.unwrap()
|
||||||
.index
|
.index
|
||||||
.get_rows_from_index_for_filter(Arc::clone(&table_def), filter)
|
.get_rows_from_index_for_filter(&filter)
|
||||||
.unwrap();
|
.unwrap();
|
||||||
assert_eq!(b_rows, &[1]);
|
assert_eq!(
|
||||||
|
rows,
|
||||||
let b = table_buffer
|
HashSet::<usize>::from_iter(t.expected_rows.iter().copied())
|
||||||
.record_batches(Arc::clone(&table_def), filter)
|
);
|
||||||
.unwrap();
|
let batches = table_buffer
|
||||||
let expected_b = vec![
|
.partitioned_record_batches(Arc::clone(&table_def), &filter)
|
||||||
"+-----+--------------------------------+-------+",
|
.unwrap()
|
||||||
"| tag | time | value |",
|
.into_values()
|
||||||
"+-----+--------------------------------+-------+",
|
.flat_map(|(_, batch)| batch.into_iter())
|
||||||
"| b | 1970-01-01T00:00:00.000000002Z | 2 |",
|
.collect::<Vec<RecordBatch>>();
|
||||||
"+-----+--------------------------------+-------+",
|
assert_batches_sorted_eq!(t.expected_output, &batches);
|
||||||
];
|
}
|
||||||
assert_batches_eq!(&expected_b, &b);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#[test]
|
#[test]
|
||||||
fn computed_size_of_buffer() {
|
fn test_computed_size_of_buffer() {
|
||||||
let mut table_buffer = TableBuffer::new(vec![ColumnId::from(0)], SortKey::empty());
|
let writer = TestWriter::new();
|
||||||
|
|
||||||
let rows = vec![
|
let rows = writer.write_to_rows(
|
||||||
Row {
|
"\
|
||||||
time: 1,
|
tbl,tag=a value=1i 1\n\
|
||||||
fields: vec![
|
tbl,tag=b value=2i 2\n\
|
||||||
Field {
|
tbl,tag=this\\ is\\ a\\ long\\ tag\\ value\\ to\\ store value=3i 3\n\
|
||||||
id: ColumnId::from(0),
|
",
|
||||||
value: FieldData::Tag("a".to_string()),
|
0,
|
||||||
},
|
);
|
||||||
Field {
|
|
||||||
id: ColumnId::from(1),
|
|
||||||
value: FieldData::Integer(1),
|
|
||||||
},
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(2),
|
|
||||||
value: FieldData::Timestamp(1),
|
|
||||||
},
|
|
||||||
],
|
|
||||||
},
|
|
||||||
Row {
|
|
||||||
time: 2,
|
|
||||||
fields: vec![
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(0),
|
|
||||||
value: FieldData::Tag("b".to_string()),
|
|
||||||
},
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(1),
|
|
||||||
value: FieldData::Integer(2),
|
|
||||||
},
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(2),
|
|
||||||
value: FieldData::Timestamp(2),
|
|
||||||
},
|
|
||||||
],
|
|
||||||
},
|
|
||||||
Row {
|
|
||||||
time: 3,
|
|
||||||
fields: vec![
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(0),
|
|
||||||
value: FieldData::Tag("this is a long tag value to store".to_string()),
|
|
||||||
},
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(1),
|
|
||||||
value: FieldData::Integer(3),
|
|
||||||
},
|
|
||||||
Field {
|
|
||||||
id: ColumnId::from(2),
|
|
||||||
value: FieldData::Timestamp(3),
|
|
||||||
},
|
|
||||||
],
|
|
||||||
},
|
|
||||||
];
|
|
||||||
|
|
||||||
|
let tag_col_id = writer
|
||||||
|
.db_schema()
|
||||||
|
.table_definition("tbl")
|
||||||
|
.and_then(|tbl| tbl.column_name_to_id("tag"))
|
||||||
|
.unwrap();
|
||||||
|
|
||||||
|
let mut table_buffer = TableBuffer::new(vec![tag_col_id], SortKey::empty());
|
||||||
table_buffer.buffer_chunk(0, &rows);
|
table_buffer.buffer_chunk(0, &rows);
|
||||||
|
|
||||||
let size = table_buffer.computed_size();
|
let size = table_buffer.computed_size();
|
||||||
|
@ -1118,4 +1059,94 @@ mod tests {
|
||||||
assert_eq!(timestamp_min_max.min, 0);
|
assert_eq!(timestamp_min_max.min, 0);
|
||||||
assert_eq!(timestamp_min_max.max, 0);
|
assert_eq!(timestamp_min_max.max, 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
#[test_log::test]
|
||||||
|
fn test_time_filters() {
|
||||||
|
let writer = TestWriter::new();
|
||||||
|
|
||||||
|
let mut row_batches = Vec::new();
|
||||||
|
for offset in 0..100 {
|
||||||
|
let rows = writer.write_to_rows(
|
||||||
|
format!(
|
||||||
|
"\
|
||||||
|
tbl,tag=a val={}\n\
|
||||||
|
tbl,tag=b val={}\n\
|
||||||
|
",
|
||||||
|
offset + 1,
|
||||||
|
offset + 2
|
||||||
|
),
|
||||||
|
offset,
|
||||||
|
);
|
||||||
|
row_batches.push((offset, rows));
|
||||||
|
}
|
||||||
|
let table_def = writer.db_schema().table_definition("tbl").unwrap();
|
||||||
|
let tag_col_id = table_def.column_name_to_id("tag").unwrap();
|
||||||
|
let mut table_buffer = TableBuffer::new(vec![tag_col_id], SortKey::empty());
|
||||||
|
|
||||||
|
for (offset, rows) in row_batches {
|
||||||
|
table_buffer.buffer_chunk(offset, &rows);
|
||||||
|
}
|
||||||
|
|
||||||
|
struct TestCase<'a> {
|
||||||
|
filter: &'a [Expr],
|
||||||
|
expected_output: &'a [&'a str],
|
||||||
|
}
|
||||||
|
|
||||||
|
let test_cases = [
|
||||||
|
TestCase {
|
||||||
|
filter: &[col("time").gt(lit_timestamp_nano(97_000_000_000i64))],
|
||||||
|
expected_output: &[
|
||||||
|
"+-----+----------------------+-------+",
|
||||||
|
"| tag | time | val |",
|
||||||
|
"+-----+----------------------+-------+",
|
||||||
|
"| a | 1970-01-01T00:01:38Z | 99.0 |",
|
||||||
|
"| a | 1970-01-01T00:01:39Z | 100.0 |",
|
||||||
|
"| b | 1970-01-01T00:01:38Z | 100.0 |",
|
||||||
|
"| b | 1970-01-01T00:01:39Z | 101.0 |",
|
||||||
|
"+-----+----------------------+-------+",
|
||||||
|
],
|
||||||
|
},
|
||||||
|
TestCase {
|
||||||
|
filter: &[col("time").lt(lit_timestamp_nano(3_000_000_000i64))],
|
||||||
|
expected_output: &[
|
||||||
|
"+-----+----------------------+-----+",
|
||||||
|
"| tag | time | val |",
|
||||||
|
"+-----+----------------------+-----+",
|
||||||
|
"| a | 1970-01-01T00:00:00Z | 1.0 |",
|
||||||
|
"| a | 1970-01-01T00:00:01Z | 2.0 |",
|
||||||
|
"| a | 1970-01-01T00:00:02Z | 3.0 |",
|
||||||
|
"| b | 1970-01-01T00:00:00Z | 2.0 |",
|
||||||
|
"| b | 1970-01-01T00:00:01Z | 3.0 |",
|
||||||
|
"| b | 1970-01-01T00:00:02Z | 4.0 |",
|
||||||
|
"+-----+----------------------+-----+",
|
||||||
|
],
|
||||||
|
},
|
||||||
|
TestCase {
|
||||||
|
filter: &[col("time")
|
||||||
|
.gt(lit_timestamp_nano(3_000_000_000i64))
|
||||||
|
.and(col("time").lt(lit_timestamp_nano(6_000_000_000i64)))],
|
||||||
|
expected_output: &[
|
||||||
|
"+-----+----------------------+-----+",
|
||||||
|
"| tag | time | val |",
|
||||||
|
"+-----+----------------------+-----+",
|
||||||
|
"| a | 1970-01-01T00:00:04Z | 5.0 |",
|
||||||
|
"| a | 1970-01-01T00:00:05Z | 6.0 |",
|
||||||
|
"| b | 1970-01-01T00:00:04Z | 6.0 |",
|
||||||
|
"| b | 1970-01-01T00:00:05Z | 7.0 |",
|
||||||
|
"+-----+----------------------+-----+",
|
||||||
|
],
|
||||||
|
},
|
||||||
|
];
|
||||||
|
|
||||||
|
for t in test_cases {
|
||||||
|
let filter = BufferFilter::generate(&table_def, t.filter).unwrap();
|
||||||
|
let batches = table_buffer
|
||||||
|
.partitioned_record_batches(Arc::clone(&table_def), &filter)
|
||||||
|
.unwrap()
|
||||||
|
.into_values()
|
||||||
|
.flat_map(|(_, batches)| batches)
|
||||||
|
.collect::<Vec<RecordBatch>>();
|
||||||
|
assert_batches_sorted_eq!(t.expected_output, &batches);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue