diff --git a/Cargo.toml b/Cargo.toml index 24a9561..415ec35 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -8,8 +8,8 @@ name = "server" path = "src/bin/server.rs" [dependencies] -datafusion = "51.0" -datafusion-tracing = "51.0.0" +datafusion = { version = "52.1", default-features = false, features = ["parquet", "nested_expressions", "crypto_expressions", "regex_expressions", "unicode_expressions", "datetime_expressions", "encoding_expressions", "string_expressions", "math_expressions", "sql", "recursive_protection"] } +datafusion-tracing = "52.0.0" instrumented-object-store = "52.0.0" duckdb = { version = "1.4.4", features = ["bundled"] } sqlx = { version = "0.8", features = ["runtime-tokio", "postgres", "sqlite", "mysql", "chrono", "tls-rustls", "bigdecimal"] } @@ -57,7 +57,8 @@ arrow-array-54 = { package = "arrow-array", version = "54" } arrow-ipc-54 = { package = "arrow-ipc", version = "54" } # Arrow 56 for duckdb compatibility (duckdb uses arrow 56, datafusion uses arrow 57) arrow-ipc-56 = { package = "arrow-ipc", version = "56" } -liquid-cache-client = { git = "https://github.com/XiangpengHao/liquid-cache", rev = "5e63d811132230a0e15fb6d4311be2eb5551cb4d" } +# Temporarily disabled - waiting for DataFusion 52 support +# liquid-cache-client = { git = "https://github.com/XiangpengHao/liquid-cache", rev = "5e63d811132230a0e15fb6d4311be2eb5551cb4d" } redis = { version = "0.27", features = ["tokio-comp", "connection-manager"] } [build-dependencies] diff --git a/migrations/postgres/v5.sql b/migrations/postgres/v5.sql new file mode 100644 index 0000000..0ee59ad --- /dev/null +++ b/migrations/postgres/v5.sql @@ -0,0 +1,15 @@ +-- Add indexes table for storing projection/sorted index metadata +CREATE TABLE indexes ( + id SERIAL PRIMARY KEY, + connection_id TEXT NOT NULL, + schema_name TEXT NOT NULL, + table_name TEXT NOT NULL, + index_name TEXT NOT NULL, + sort_columns TEXT NOT NULL, -- JSON array of column names + parquet_path TEXT, -- Path to sorted projection parquet + created_at TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP, + FOREIGN KEY (connection_id) REFERENCES connections(id) ON DELETE CASCADE, + UNIQUE (connection_id, schema_name, table_name, index_name) +); + +CREATE INDEX idx_indexes_table ON indexes(connection_id, schema_name, table_name); diff --git a/migrations/sqlite/v5.sql b/migrations/sqlite/v5.sql new file mode 100644 index 0000000..79e1a05 --- /dev/null +++ b/migrations/sqlite/v5.sql @@ -0,0 +1,15 @@ +-- Add indexes table for storing projection/sorted index metadata +CREATE TABLE indexes ( + id INTEGER PRIMARY KEY AUTOINCREMENT, + connection_id TEXT NOT NULL, + schema_name TEXT NOT NULL, + table_name TEXT NOT NULL, + index_name TEXT NOT NULL, + sort_columns TEXT NOT NULL, -- JSON array of column names + parquet_path TEXT, -- Path to sorted projection parquet + created_at TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP, + FOREIGN KEY (connection_id) REFERENCES connections(id) ON DELETE CASCADE, + UNIQUE (connection_id, schema_name, table_name, index_name) +); + +CREATE INDEX idx_indexes_table ON indexes(connection_id, schema_name, table_name); diff --git a/src/catalog/caching_manager.rs b/src/catalog/caching_manager.rs index 086bbdc..8e59afb 100644 --- a/src/catalog/caching_manager.rs +++ b/src/catalog/caching_manager.rs @@ -5,8 +5,8 @@ //! to the inner catalog directly. use super::{ - CatalogManager, ConnectionInfo, DatasetInfo, OptimisticLock, PendingDeletion, QueryResult, - ResultStatus, ResultUpdate, TableInfo, UploadInfo, + CatalogManager, ConnectionInfo, DatasetInfo, IndexInfo, OptimisticLock, PendingDeletion, + QueryResult, ResultStatus, ResultUpdate, TableInfo, UploadInfo, }; use crate::config::CacheConfig; use crate::secrets::{SecretMetadata, SecretStatus}; @@ -988,4 +988,62 @@ impl CatalogManager for CachingCatalogManager { self.cached_read(&key, || self.inner().list_dataset_table_names(&schema)) .await } + + // Index methods - pass through to inner catalog (no caching for now) + + async fn create_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + sort_columns: &[String], + parquet_path: Option<&str>, + ) -> Result { + self.inner() + .create_index( + connection_id, + schema_name, + table_name, + index_name, + sort_columns, + parquet_path, + ) + .await + } + + async fn get_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + ) -> Result> { + self.inner() + .get_index(connection_id, schema_name, table_name, index_name) + .await + } + + async fn list_indexes( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + ) -> Result> { + self.inner() + .list_indexes(connection_id, schema_name, table_name) + .await + } + + async fn delete_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + ) -> Result> { + self.inner() + .delete_index(connection_id, schema_name, table_name, index_name) + .await + } } diff --git a/src/catalog/manager.rs b/src/catalog/manager.rs index ed00a92..7c308a0 100644 --- a/src/catalog/manager.rs +++ b/src/catalog/manager.rs @@ -172,6 +172,45 @@ pub struct DatasetInfo { pub updated_at: DateTime, } +/// A sorted index (projection) on a table. +#[derive(Debug, Clone, Serialize, Deserialize, FromRow)] +pub struct IndexInfo { + pub id: i32, + pub connection_id: String, + pub schema_name: String, + pub table_name: String, + pub index_name: String, + pub sort_columns: String, // JSON array of column names + pub parquet_path: Option, + pub created_at: DateTime, +} + +impl IndexInfo { + /// Parse sort_columns JSON into a Vec. + pub fn sort_columns_vec(&self) -> Vec { + serde_json::from_str(&self.sort_columns).unwrap_or_default() + } + + /// Parse sort_columns JSON into (column_name, is_descending) pairs. + /// Always returns ASC (false). Strips legacy " ASC"/" DESC" suffixes for backward compat. + pub fn sort_columns_with_direction(&self) -> Vec<(String, bool)> { + let cols: Vec = serde_json::from_str(&self.sort_columns).unwrap_or_default(); + cols.into_iter() + .map(|c| { + let c_upper = c.to_uppercase(); + let name = if c_upper.ends_with(" DESC") { + c[..c.len() - 5].trim().to_string() + } else if c_upper.ends_with(" ASC") { + c[..c.len() - 4].trim().to_string() + } else { + c + }; + (name, false) + }) + .collect() + } +} + /// Async interface for catalog operations. #[async_trait] pub trait CatalogManager: Debug + Send + Sync { @@ -385,4 +424,43 @@ pub trait CatalogManager: Debug + Send + Sync { /// Delete a dataset by ID. Returns the deleted dataset if it existed. async fn delete_dataset(&self, id: &str) -> Result>; + + // Index management methods + + /// Create a new index with its parquet path. Returns the index ID. + async fn create_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + sort_columns: &[String], + parquet_path: Option<&str>, + ) -> Result; + + /// Get an index by name. + async fn get_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + ) -> Result>; + + /// List all indexes for a table. + async fn list_indexes( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + ) -> Result>; + + /// Delete an index. Returns the deleted index if it existed. + async fn delete_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + ) -> Result>; } diff --git a/src/catalog/mock_catalog.rs b/src/catalog/mock_catalog.rs index 09c0709..5d94f3e 100644 --- a/src/catalog/mock_catalog.rs +++ b/src/catalog/mock_catalog.rs @@ -4,8 +4,8 @@ //! in tests to avoid needing a real database. use super::{ - CatalogManager, ConnectionInfo, DatasetInfo, OptimisticLock, PendingDeletion, QueryResult, - ResultStatus, ResultUpdate, TableInfo, UploadInfo, + CatalogManager, ConnectionInfo, DatasetInfo, IndexInfo, OptimisticLock, PendingDeletion, + QueryResult, ResultStatus, ResultUpdate, TableInfo, UploadInfo, }; use crate::secrets::{SecretMetadata, SecretStatus}; use anyhow::Result; @@ -357,4 +357,47 @@ impl CatalogManager for MockCatalog { async fn delete_dataset(&self, _id: &str) -> Result> { Ok(None) } + + // Index methods - not implemented for mock + + async fn create_index( + &self, + _connection_id: &str, + _schema_name: &str, + _table_name: &str, + _index_name: &str, + _sort_columns: &[String], + _parquet_path: Option<&str>, + ) -> Result { + Ok(1) + } + + async fn get_index( + &self, + _connection_id: &str, + _schema_name: &str, + _table_name: &str, + _index_name: &str, + ) -> Result> { + Ok(None) + } + + async fn list_indexes( + &self, + _connection_id: &str, + _schema_name: &str, + _table_name: &str, + ) -> Result> { + Ok(vec![]) + } + + async fn delete_index( + &self, + _connection_id: &str, + _schema_name: &str, + _table_name: &str, + _index_name: &str, + ) -> Result> { + Ok(None) + } } diff --git a/src/catalog/mod.rs b/src/catalog/mod.rs index ce377a0..c805564 100644 --- a/src/catalog/mod.rs +++ b/src/catalog/mod.rs @@ -14,8 +14,8 @@ mod mock_catalog; pub use mock_catalog::MockCatalog; pub use manager::{ - CatalogManager, ConnectionInfo, DatasetInfo, OptimisticLock, PendingDeletion, QueryResult, - QueryResultRow, ResultStatus, ResultUpdate, TableInfo, UploadInfo, + CatalogManager, ConnectionInfo, DatasetInfo, IndexInfo, OptimisticLock, PendingDeletion, + QueryResult, QueryResultRow, ResultStatus, ResultUpdate, TableInfo, UploadInfo, }; pub use postgres_manager::PostgresCatalogManager; pub use sqlite_manager::SqliteCatalogManager; diff --git a/src/catalog/postgres_manager.rs b/src/catalog/postgres_manager.rs index 7f85d07..9c53341 100644 --- a/src/catalog/postgres_manager.rs +++ b/src/catalog/postgres_manager.rs @@ -1,7 +1,7 @@ use crate::catalog::backend::CatalogBackend; use crate::catalog::manager::{ - CatalogManager, ConnectionInfo, DatasetInfo, OptimisticLock, PendingDeletion, QueryResult, - QueryResultRow, ResultStatus, ResultUpdate, TableInfo, UploadInfo, + CatalogManager, ConnectionInfo, DatasetInfo, IndexInfo, OptimisticLock, PendingDeletion, + QueryResult, QueryResultRow, ResultStatus, ResultUpdate, TableInfo, UploadInfo, }; use crate::catalog::migrations::{ run_migrations, wrap_migration_sql, CatalogMigrations, Migration, POSTGRES_MIGRATIONS, @@ -895,6 +895,101 @@ impl CatalogManager for PostgresCatalogManager { Ok(dataset) } + + // Index management methods + + async fn create_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + sort_columns: &[String], + parquet_path: Option<&str>, + ) -> Result { + let sort_columns_json = serde_json::to_string(sort_columns)?; + + let row: (i32,) = sqlx::query_as( + "INSERT INTO indexes (connection_id, schema_name, table_name, index_name, sort_columns, parquet_path) + VALUES ($1, $2, $3, $4, $5, $6) + RETURNING id", + ) + .bind(connection_id) + .bind(schema_name) + .bind(table_name) + .bind(index_name) + .bind(&sort_columns_json) + .bind(parquet_path) + .fetch_one(self.backend.pool()) + .await?; + + Ok(row.0) + } + + async fn get_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + ) -> Result> { + let row: Option = sqlx::query_as( + "SELECT id, connection_id, schema_name, table_name, index_name, sort_columns, parquet_path, created_at + FROM indexes + WHERE connection_id = $1 AND schema_name = $2 AND table_name = $3 AND index_name = $4", + ) + .bind(connection_id) + .bind(schema_name) + .bind(table_name) + .bind(index_name) + .fetch_optional(self.backend.pool()) + .await?; + + Ok(row) + } + + async fn list_indexes( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + ) -> Result> { + let rows: Vec = sqlx::query_as( + "SELECT id, connection_id, schema_name, table_name, index_name, sort_columns, parquet_path, created_at + FROM indexes + WHERE connection_id = $1 AND schema_name = $2 AND table_name = $3 + ORDER BY index_name", + ) + .bind(connection_id) + .bind(schema_name) + .bind(table_name) + .fetch_all(self.backend.pool()) + .await?; + + Ok(rows) + } + + async fn delete_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + ) -> Result> { + let row: Option = sqlx::query_as( + "DELETE FROM indexes + WHERE connection_id = $1 AND schema_name = $2 AND table_name = $3 AND index_name = $4 + RETURNING id, connection_id, schema_name, table_name, index_name, sort_columns, parquet_path, created_at", + ) + .bind(connection_id) + .bind(schema_name) + .bind(table_name) + .bind(index_name) + .fetch_optional(self.backend.pool()) + .await?; + + Ok(row) + } } impl Debug for PostgresCatalogManager { diff --git a/src/catalog/sqlite_manager.rs b/src/catalog/sqlite_manager.rs index d106ec0..f7b2965 100644 --- a/src/catalog/sqlite_manager.rs +++ b/src/catalog/sqlite_manager.rs @@ -1,7 +1,7 @@ use crate::catalog::backend::CatalogBackend; use crate::catalog::manager::{ - CatalogManager, ConnectionInfo, DatasetInfo, OptimisticLock, PendingDeletion, QueryResult, - QueryResultRow, ResultStatus, ResultUpdate, TableInfo, UploadInfo, + CatalogManager, ConnectionInfo, DatasetInfo, IndexInfo, OptimisticLock, PendingDeletion, + QueryResult, QueryResultRow, ResultStatus, ResultUpdate, TableInfo, UploadInfo, }; use crate::catalog::migrations::{ run_migrations, wrap_migration_sql, CatalogMigrations, Migration, SQLITE_MIGRATIONS, @@ -10,7 +10,7 @@ use crate::secrets::{SecretMetadata, SecretStatus}; use anyhow::Result; use async_trait::async_trait; use chrono::{DateTime, Utc}; -use sqlx::{Sqlite, SqlitePool}; +use sqlx::{Row, Sqlite, SqlitePool}; use std::fmt::{self, Debug, Formatter}; use std::str::FromStr; use tracing::warn; @@ -937,6 +937,129 @@ impl CatalogManager for SqliteCatalogManager { Ok(dataset) } + + // Index management methods + + async fn create_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + sort_columns: &[String], + parquet_path: Option<&str>, + ) -> Result { + let sort_columns_json = serde_json::to_string(sort_columns)?; + + let result = sqlx::query( + "INSERT INTO indexes (connection_id, schema_name, table_name, index_name, sort_columns, parquet_path) + VALUES (?, ?, ?, ?, ?, ?) + RETURNING id", + ) + .bind(connection_id) + .bind(schema_name) + .bind(table_name) + .bind(index_name) + .bind(&sort_columns_json) + .bind(parquet_path) + .fetch_one(self.backend.pool()) + .await?; + + Ok(result.get::("id")) + } + + async fn get_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + ) -> Result> { + let row: Option = sqlx::query_as( + "SELECT id, connection_id, schema_name, table_name, index_name, sort_columns, parquet_path, created_at + FROM indexes + WHERE connection_id = ? AND schema_name = ? AND table_name = ? AND index_name = ?", + ) + .bind(connection_id) + .bind(schema_name) + .bind(table_name) + .bind(index_name) + .fetch_optional(self.backend.pool()) + .await?; + + Ok(row.map(|r| r.into_index_info())) + } + + async fn list_indexes( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + ) -> Result> { + let rows: Vec = sqlx::query_as( + "SELECT id, connection_id, schema_name, table_name, index_name, sort_columns, parquet_path, created_at + FROM indexes + WHERE connection_id = ? AND schema_name = ? AND table_name = ? + ORDER BY index_name", + ) + .bind(connection_id) + .bind(schema_name) + .bind(table_name) + .fetch_all(self.backend.pool()) + .await?; + + Ok(rows.into_iter().map(|r| r.into_index_info()).collect()) + } + + async fn delete_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + ) -> Result> { + let row: Option = sqlx::query_as( + "DELETE FROM indexes + WHERE connection_id = ? AND schema_name = ? AND table_name = ? AND index_name = ? + RETURNING id, connection_id, schema_name, table_name, index_name, sort_columns, parquet_path, created_at", + ) + .bind(connection_id) + .bind(schema_name) + .bind(table_name) + .bind(index_name) + .fetch_optional(self.backend.pool()) + .await?; + + Ok(row.map(|r| r.into_index_info())) + } +} + +/// Row type for index queries (SQLite stores timestamps as strings) +#[derive(sqlx::FromRow)] +struct IndexInfoRow { + id: i32, + connection_id: String, + schema_name: String, + table_name: String, + index_name: String, + sort_columns: String, + parquet_path: Option, + created_at: String, +} + +impl IndexInfoRow { + fn into_index_info(self) -> IndexInfo { + IndexInfo { + id: self.id, + connection_id: self.connection_id, + schema_name: self.schema_name, + table_name: self.table_name, + index_name: self.index_name, + sort_columns: self.sort_columns, + parquet_path: self.parquet_path, + created_at: self.created_at.parse().unwrap_or_else(|_| Utc::now()), + } + } } impl CatalogMigrations for SqliteMigrationBackend { diff --git a/src/datafetch/collecting_writer.rs b/src/datafetch/collecting_writer.rs new file mode 100644 index 0000000..34039ca --- /dev/null +++ b/src/datafetch/collecting_writer.rs @@ -0,0 +1,105 @@ +//! A batch writer that collects batches in memory for later processing. + +use datafusion::arrow::array::RecordBatch; +use datafusion::arrow::datatypes::Schema; +use std::sync::Arc; + +use super::batch_writer::{BatchWriteResult, BatchWriter}; +use super::DataFetchError; + +/// A writer that collects all batches in memory. +/// Used when we need to process the data multiple times (e.g., for projections). +pub struct CollectingBatchWriter { + schema: Option>, + batches: Vec, +} + +impl CollectingBatchWriter { + pub fn new() -> Self { + Self { + schema: None, + batches: Vec::new(), + } + } + + /// Get the schema after init() has been called. + pub fn schema(&self) -> Option<&Arc> { + self.schema.as_ref() + } + + /// Consume this writer and return the collected batches. + pub fn into_batches(self) -> Vec { + self.batches + } + + /// Total row count across all batches. + pub fn row_count(&self) -> usize { + self.batches.iter().map(|b| b.num_rows()).sum() + } +} + +impl Default for CollectingBatchWriter { + fn default() -> Self { + Self::new() + } +} + +impl BatchWriter for CollectingBatchWriter { + fn init(&mut self, schema: &Schema) -> Result<(), DataFetchError> { + self.schema = Some(Arc::new(schema.clone())); + Ok(()) + } + + fn write_batch(&mut self, batch: &RecordBatch) -> Result<(), DataFetchError> { + if self.schema.is_none() { + return Err(DataFetchError::Storage( + "Writer not initialized - call init() first".into(), + )); + } + self.batches.push(batch.clone()); + Ok(()) + } + + fn close(self: Box) -> Result { + let rows = self.row_count(); + Ok(BatchWriteResult { + rows, + bytes_written: None, + }) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use datafusion::arrow::array::Int32Array; + use datafusion::arrow::datatypes::{DataType, Field}; + + #[test] + fn test_collecting_writer() { + let schema = Schema::new(vec![Field::new("id", DataType::Int32, false)]); + + let mut writer = CollectingBatchWriter::new(); + writer.init(&schema).unwrap(); + + let batch1 = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![Arc::new(Int32Array::from(vec![1, 2, 3]))], + ) + .unwrap(); + + let batch2 = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![Arc::new(Int32Array::from(vec![4, 5]))], + ) + .unwrap(); + + writer.write_batch(&batch1).unwrap(); + writer.write_batch(&batch2).unwrap(); + + assert_eq!(writer.row_count(), 5); + + let batches = writer.into_batches(); + assert_eq!(batches.len(), 2); + } +} diff --git a/src/datafetch/index_presets.rs b/src/datafetch/index_presets.rs new file mode 100644 index 0000000..a19aa84 --- /dev/null +++ b/src/datafetch/index_presets.rs @@ -0,0 +1,132 @@ +//! Preset index configuration for creating sorted copies of tables at cache time. +//! +//! Index presets are full copies of tables sorted by different columns, +//! enabling efficient row-group pruning for range queries on those columns. + +use std::collections::HashMap; + +/// Configuration for a single index preset. +#[derive(Debug, Clone)] +pub struct IndexPreset { + /// Name of the preset (used in directory structure) + pub name: String, + /// Columns to sort by (in order) + pub sort_columns: Vec, +} + +impl IndexPreset { + pub fn new(name: impl Into, sort_columns: Vec>) -> Self { + Self { + name: name.into(), + sort_columns: sort_columns.into_iter().map(|s| s.into()).collect(), + } + } +} + +/// Registry of index preset configurations per table. +/// Key format: "schema.table" (lowercase) +#[derive(Debug, Clone, Default)] +pub struct IndexPresetRegistry { + presets: HashMap>, +} + +impl IndexPresetRegistry { + pub fn new() -> Self { + Self::default() + } + + /// Register index presets for a table. + pub fn register(&mut self, schema: &str, table: &str, presets: Vec) { + let key = format!("{}.{}", schema.to_lowercase(), table.to_lowercase()); + self.presets.insert(key, presets); + } + + /// Get index presets for a table. + pub fn get(&self, schema: &str, table: &str) -> Option<&[IndexPreset]> { + let key = format!("{}.{}", schema.to_lowercase(), table.to_lowercase()); + self.presets.get(&key).map(|v| v.as_slice()) + } + + /// Create a registry with TPC-H optimized index presets. + pub fn tpch_optimized() -> Self { + let mut registry = Self::new(); + + // lineitem: sorted by l_shipdate (8 queries filter on this) + registry.register( + "main", + "lineitem", + vec![ + IndexPreset::new("by_shipdate", vec!["l_shipdate"]), + IndexPreset::new("by_partkey", vec!["l_partkey"]), + ], + ); + + // orders: sorted by o_orderdate (5 queries filter on this) + registry.register( + "main", + "orders", + vec![ + IndexPreset::new("by_orderdate", vec!["o_orderdate"]), + IndexPreset::new("by_custkey", vec!["o_custkey"]), + ], + ); + + // customer: sorted by c_nationkey (joins) + registry.register( + "main", + "customer", + vec![IndexPreset::new("by_nationkey", vec!["c_nationkey"])], + ); + + // partsupp: sorted by ps_suppkey (joins) + registry.register( + "main", + "partsupp", + vec![IndexPreset::new("by_suppkey", vec!["ps_suppkey"])], + ); + + registry + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_index_preset_registry() { + let mut registry = IndexPresetRegistry::new(); + registry.register( + "main", + "orders", + vec![IndexPreset::new("by_date", vec!["order_date"])], + ); + + let presets = registry.get("main", "orders").unwrap(); + assert_eq!(presets.len(), 1); + assert_eq!(presets[0].name, "by_date"); + assert_eq!(presets[0].sort_columns, vec!["order_date"]); + } + + #[test] + fn test_case_insensitive_lookup() { + let mut registry = IndexPresetRegistry::new(); + registry.register("Main", "Orders", vec![]); + + assert!(registry.get("main", "orders").is_some()); + assert!(registry.get("MAIN", "ORDERS").is_some()); + } + + #[test] + fn test_tpch_optimized() { + let registry = IndexPresetRegistry::tpch_optimized(); + + let lineitem = registry.get("main", "lineitem").unwrap(); + assert_eq!(lineitem.len(), 2); + assert_eq!(lineitem[0].name, "by_shipdate"); + assert_eq!(lineitem[1].name, "by_partkey"); + + let orders = registry.get("main", "orders").unwrap(); + assert_eq!(orders.len(), 2); + } +} diff --git a/src/datafetch/mod.rs b/src/datafetch/mod.rs index ad335ca..0102ef2 100644 --- a/src/datafetch/mod.rs +++ b/src/datafetch/mod.rs @@ -1,13 +1,17 @@ mod batch_writer; +mod collecting_writer; mod error; mod fetcher; +mod index_presets; pub mod native; mod orchestrator; +mod sorted_parquet; mod types; pub use batch_writer::{BatchWriteResult, BatchWriter}; pub use error::DataFetchError; pub use fetcher::DataFetcher; -pub use native::{NativeFetcher, StreamingParquetWriter}; +pub use index_presets::{IndexPreset, IndexPresetRegistry}; +pub use native::{NativeFetcher, ParquetConfig, StreamingParquetWriter}; pub use orchestrator::FetchOrchestrator; pub use types::{deserialize_arrow_schema, ColumnMetadata, TableMetadata}; diff --git a/src/datafetch/native/mod.rs b/src/datafetch/native/mod.rs index db8b8c6..07b45f6 100644 --- a/src/datafetch/native/mod.rs +++ b/src/datafetch/native/mod.rs @@ -6,8 +6,7 @@ pub mod mysql; mod parquet_writer; pub mod postgres; pub mod snowflake; - -pub use parquet_writer::StreamingParquetWriter; +pub use parquet_writer::{ParquetConfig, StreamingParquetWriter}; use async_trait::async_trait; use datafusion::arrow::datatypes::DataType; diff --git a/src/datafetch/native/parquet_writer.rs b/src/datafetch/native/parquet_writer.rs index eb7a3f7..aa55cb3 100644 --- a/src/datafetch/native/parquet_writer.rs +++ b/src/datafetch/native/parquet_writer.rs @@ -12,15 +12,42 @@ use std::sync::Arc; use crate::datafetch::batch_writer::{BatchWriteResult, BatchWriter}; use crate::datafetch::DataFetchError; -/// Build shared writer properties for parquet files. +/// Configuration for parquet file writing. +#[derive(Debug, Clone)] +pub struct ParquetConfig { + pub max_row_group_size: usize, + pub bloom_filter_enabled: bool, +} + +impl Default for ParquetConfig { + fn default() -> Self { + Self { + max_row_group_size: 100_000, + bloom_filter_enabled: false, + } + } +} + +/// Build shared writer properties for parquet files (default config). fn writer_properties() -> WriterProperties { - WriterProperties::builder() + writer_properties_from_config(&ParquetConfig::default()) +} + +/// Build writer properties from a config. +fn writer_properties_from_config(config: &ParquetConfig) -> WriterProperties { + let mut builder = WriterProperties::builder() .set_writer_version(WriterVersion::PARQUET_2_0) .set_compression(Compression::LZ4) - .set_bloom_filter_enabled(true) - .set_bloom_filter_fpp(0.01) - .set_bloom_filter_position(BloomFilterPosition::End) - .build() + .set_max_row_group_size(config.max_row_group_size); + + if config.bloom_filter_enabled { + builder = builder + .set_bloom_filter_enabled(true) + .set_bloom_filter_fpp(0.01) + .set_bloom_filter_position(BloomFilterPosition::End); + } + + builder.build() } /// Streaming Parquet writer that writes batches incrementally to disk. @@ -28,6 +55,7 @@ fn writer_properties() -> WriterProperties { /// Lifecycle: new(path) -> init(schema) -> write_batch()* -> close() pub struct StreamingParquetWriter { path: PathBuf, + config: Option, writer: Option>, row_count: usize, } @@ -38,6 +66,17 @@ impl StreamingParquetWriter { pub fn new(path: PathBuf) -> Self { Self { path, + config: None, + writer: None, + row_count: 0, + } + } + + /// Create a new writer with a custom parquet config. + pub fn with_config(path: PathBuf, config: ParquetConfig) -> Self { + Self { + path, + config: Some(config), writer: None, row_count: 0, } @@ -61,7 +100,10 @@ impl BatchWriter for StreamingParquetWriter { let file = File::create(&self.path) .map_err(|e| DataFetchError::Storage(format!("Failed to create file: {}", e)))?; - let props = writer_properties(); + let props = match &self.config { + Some(config) => writer_properties_from_config(config), + None => writer_properties(), + }; let writer = ArrowWriter::try_new(file, Arc::new(schema.clone()), Some(props)) .map_err(|e| DataFetchError::Storage(e.to_string()))?; @@ -279,66 +321,4 @@ mod tests { } } } - - #[test] - fn test_bloom_filter_enabled() { - use datafusion::parquet::file::reader::{FileReader, SerializedFileReader}; - use std::fs::File; - - let dir = tempdir().unwrap(); - let path = dir.path().join("bloom_filter.parquet"); - - let schema = Schema::new(vec![ - Field::new("id", DataType::Int32, false), - Field::new("name", DataType::Utf8, true), - ]); - - let mut writer: Box = Box::new(StreamingParquetWriter::new(path.clone())); - writer.init(&schema).unwrap(); - - // Write some data with distinct values to trigger bloom filter creation - let batch = RecordBatch::try_new( - Arc::new(schema), - vec![ - Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10])), - Arc::new(datafusion::arrow::array::StringArray::from(vec![ - Some("Alice"), - Some("Bob"), - Some("Charlie"), - Some("David"), - Some("Eve"), - Some("Frank"), - Some("Grace"), - Some("Henry"), - Some("Ivy"), - Some("Jack"), - ])), - ], - ) - .unwrap(); - - writer.write_batch(&batch).unwrap(); - writer.close().unwrap(); - - // Read back and verify bloom filters exist - let file = File::open(&path).unwrap(); - let reader = SerializedFileReader::new(file).unwrap(); - let metadata = reader.metadata(); - - // Check that bloom filter offsets are set for columns - let mut bloom_filter_found = false; - for row_group in metadata.row_groups() { - for column in row_group.columns() { - if column.bloom_filter_offset().is_some() { - bloom_filter_found = true; - break; - } - } - } - - assert!( - bloom_filter_found, - "Expected bloom filter to be present in parquet metadata" - ); - } } diff --git a/src/datafetch/orchestrator.rs b/src/datafetch/orchestrator.rs index db7bdb8..ad011ea 100644 --- a/src/datafetch/orchestrator.rs +++ b/src/datafetch/orchestrator.rs @@ -2,7 +2,10 @@ use anyhow::Result; use std::sync::Arc; use super::batch_writer::BatchWriter; -use super::native::StreamingParquetWriter; +use super::collecting_writer::CollectingBatchWriter; +use super::index_presets::IndexPresetRegistry; +use super::native::{ParquetConfig, StreamingParquetWriter}; +use super::sorted_parquet::{write_parquet, write_sorted_parquet}; use super::{DataFetchError, DataFetcher, TableMetadata}; use crate::catalog::CatalogManager; use crate::secrets::SecretManager; @@ -16,6 +19,8 @@ pub struct FetchOrchestrator { storage: Arc, catalog: Arc, secret_manager: Arc, + index_preset_registry: Option>, + parquet_config: Option, } impl FetchOrchestrator { @@ -30,9 +35,45 @@ impl FetchOrchestrator { storage, catalog, secret_manager, + index_preset_registry: None, + parquet_config: None, } } + /// Create an orchestrator with index preset support. + pub fn with_index_presets( + fetcher: Arc, + storage: Arc, + catalog: Arc, + secret_manager: Arc, + index_preset_registry: Arc, + ) -> Self { + Self { + fetcher, + storage, + catalog, + secret_manager, + index_preset_registry: Some(index_preset_registry), + parquet_config: None, + } + } + + /// Set a custom parquet configuration for this orchestrator. + pub fn with_parquet_config(mut self, config: ParquetConfig) -> Self { + self.parquet_config = Some(config); + self + } + + /// Get the parquet config if set. + pub fn parquet_config(&self) -> Option<&ParquetConfig> { + self.parquet_config.as_ref() + } + + /// Get the index preset registry if configured. + pub fn index_preset_registry(&self) -> Option<&Arc> { + self.index_preset_registry.as_ref() + } + /// Fetch table data from source, write to cache storage, and update catalog metadata. /// /// Returns the URL of the cached parquet file and the row count. @@ -60,28 +101,21 @@ impl FetchOrchestrator { .storage .prepare_cache_write(connection_id, schema_name, table_name); - // Create writer - let mut writer: Box = - Box::new(StreamingParquetWriter::new(handle.local_path.clone())); - - // Fetch the table data into writer - self.fetcher - .fetch_table( - source, - &self.secret_manager, - None, // catalog - schema_name, - table_name, - writer.as_mut(), - ) - .await - .map_err(|e| anyhow::anyhow!("Failed to fetch table: {}", e))?; - - // Close writer and get row count - let result = writer - .close() - .map_err(|e| anyhow::anyhow!("Failed to close writer: {}", e))?; - let row_count = result.rows; + // Check if index presets are configured for this table + let presets = self + .index_preset_registry + .as_ref() + .and_then(|r| r.get(schema_name, table_name)); + + let row_count = if let Some(presets) = presets { + // Use collecting writer to gather batches for preset index creation + self.cache_table_with_index_presets(source, schema_name, table_name, &handle, presets) + .await? + } else { + // Simple path: stream directly to parquet + self.cache_table_simple(source, schema_name, table_name, &handle) + .await? + }; // Finalize cache write (uploads to S3 if needed, returns URL) let parquet_url = self @@ -111,14 +145,123 @@ impl FetchOrchestrator { let span = tracing::Span::current(); span.record("runtimedb.rows_written", row_count); - if let Some(bytes) = result.bytes_written { - span.record("runtimedb.bytes_written", bytes); - } span.record("runtimedb.cache_url", &parquet_url); Ok((parquet_url, row_count)) } + /// Simple cache path: stream directly to parquet file. + async fn cache_table_simple( + &self, + source: &Source, + schema_name: &str, + table_name: &str, + handle: &crate::storage::CacheWriteHandle, + ) -> Result { + let mut writer: Box = match &self.parquet_config { + Some(config) => Box::new(StreamingParquetWriter::with_config( + handle.local_path.clone(), + config.clone(), + )), + None => Box::new(StreamingParquetWriter::new(handle.local_path.clone())), + }; + + self.fetcher + .fetch_table( + source, + &self.secret_manager, + None, + schema_name, + table_name, + writer.as_mut(), + ) + .await + .map_err(|e| anyhow::anyhow!("Failed to fetch table: {}", e))?; + + let result = writer + .close() + .map_err(|e| anyhow::anyhow!("Failed to close writer: {}", e))?; + + Ok(result.rows) + } + + /// Cache with index presets: collect batches, write main file, then write sorted copies. + async fn cache_table_with_index_presets( + &self, + source: &Source, + schema_name: &str, + table_name: &str, + handle: &crate::storage::CacheWriteHandle, + presets: &[super::index_presets::IndexPreset], + ) -> Result { + // Collect all batches in memory + let mut collector = CollectingBatchWriter::new(); + + self.fetcher + .fetch_table( + source, + &self.secret_manager, + None, + schema_name, + table_name, + &mut collector, + ) + .await + .map_err(|e| anyhow::anyhow!("Failed to fetch table: {}", e))?; + + let schema = collector + .schema() + .ok_or_else(|| anyhow::anyhow!("No schema from fetcher"))? + .as_ref() + .clone(); + let batches = collector.into_batches(); + let row_count: usize = batches.iter().map(|b| b.num_rows()).sum(); + + // Ensure parent directory exists + if let Some(parent) = handle.local_path.parent() { + std::fs::create_dir_all(parent)?; + } + + // Write main parquet file + write_parquet( + &batches, + &schema, + &handle.local_path, + self.parquet_config.as_ref(), + ) + .map_err(|e| anyhow::anyhow!("Failed to write main parquet: {}", e))?; + + // Write preset indexes + let version_dir = handle + .local_path + .parent() + .ok_or_else(|| anyhow::anyhow!("Invalid handle path: no parent directory"))?; + + for preset in presets { + let preset_dir = version_dir.join("presets").join(&preset.name); + std::fs::create_dir_all(&preset_dir)?; + let preset_path = preset_dir.join("data.parquet"); + + tracing::debug!( + preset = %preset.name, + sort_columns = ?preset.sort_columns, + "Writing preset index" + ); + + write_sorted_parquet( + &batches, + &schema, + &preset.sort_columns, + &preset_path, + self.parquet_config.as_ref(), + ) + .await + .map_err(|e| anyhow::anyhow!("Failed to write preset index {}: {}", preset.name, e))?; + } + + Ok(row_count) + } + /// Discover tables from a remote source. /// Delegates to the underlying fetcher. pub async fn discover_tables( @@ -174,35 +317,28 @@ impl FetchOrchestrator { .storage .prepare_cache_write(connection_id, schema_name, table_name); - // 3. Fetch and write to new path - let mut writer: Box = - Box::new(StreamingParquetWriter::new(handle.local_path.clone())); - self.fetcher - .fetch_table( - source, - &self.secret_manager, - None, - schema_name, - table_name, - writer.as_mut(), - ) - .await - .map_err(|e| anyhow::anyhow!("Failed to fetch table: {}", e))?; - - // 4. Close writer and get row count - let result = writer - .close() - .map_err(|e| anyhow::anyhow!("Failed to close writer: {}", e))?; - let row_count = result.rows; + // 3. Check if index presets are configured for this table + let presets = self + .index_preset_registry + .as_ref() + .and_then(|r| r.get(schema_name, table_name)); + + let row_count = if let Some(presets) = presets { + self.cache_table_with_index_presets(source, schema_name, table_name, &handle, presets) + .await? + } else { + self.cache_table_simple(source, schema_name, table_name, &handle) + .await? + }; - // 5. Finalize (upload to S3 if needed) + // 4. Finalize (upload to S3 if needed) let new_url = self .storage .finalize_cache_write(&handle) .await .map_err(|e| anyhow::anyhow!("Failed to finalize cache write: {}", e))?; - // 6. Atomic catalog update with cleanup on failure + // 5. Atomic catalog update with cleanup on failure // If this fails after finalize_cache_write succeeds, we have orphaned files. // Clean them up to prevent storage leaks. let catalog_result = self @@ -223,9 +359,6 @@ impl FetchOrchestrator { let span = tracing::Span::current(); span.record("runtimedb.rows_synced", row_count); - if let Some(bytes) = result.bytes_written { - span.record("runtimedb.bytes_written", bytes); - } span.record("runtimedb.cache_url", &new_url); Ok((new_url, old_path, row_count)) @@ -709,4 +842,67 @@ mod tests { deleted_url ); } + + #[tokio::test] + async fn test_cache_table_with_index_presets() { + use crate::datafetch::index_presets::{IndexPreset, IndexPresetRegistry}; + + let temp_dir = tempfile::tempdir().unwrap(); + let cache_path = temp_dir.path().join("cache"); + std::fs::create_dir_all(&cache_path).unwrap(); + + let fetcher = Arc::new(MockFetcher); + let storage = Arc::new(MockStorage::new(cache_path.clone())); + let catalog = Arc::new(MockCatalog::new()); + let secret_manager = Arc::new(create_test_secret_manager(temp_dir.path()).await); + + // Add a table to the mock catalog + catalog.add_table_entry("conn_test123", "test", "orders"); + + // Create index preset registry + let mut registry = IndexPresetRegistry::new(); + registry.register( + "test", + "orders", + vec![IndexPreset::new("by_id", vec!["id"])], + ); + + let orchestrator = FetchOrchestrator::with_index_presets( + fetcher, + storage.clone(), + catalog, + secret_manager, + Arc::new(registry), + ); + + let source = Source::Duckdb { + path: ":memory:".to_string(), + }; + + let result = orchestrator + .cache_table(&source, "conn_test123", "test", "orders") + .await; + + assert!(result.is_ok(), "cache_table should succeed: {:?}", result); + let (url, row_count) = result.unwrap(); + + assert_eq!(row_count, 3, "Should have cached 3 rows from MockFetcher"); + assert!( + url.contains("/conn_test123/test/orders/"), + "URL should be for the test table" + ); + + // Verify main parquet file exists + let version_dir = url.strip_prefix("file://").unwrap(); + let main_file = std::path::Path::new(version_dir).join("data.parquet"); + assert!(main_file.exists(), "Main parquet file should exist"); + + // Verify preset index file exists + let preset_file = std::path::Path::new(version_dir).join("presets/by_id/data.parquet"); + assert!( + preset_file.exists(), + "Preset index parquet file should exist at {:?}", + preset_file + ); + } } diff --git a/src/datafetch/sorted_parquet.rs b/src/datafetch/sorted_parquet.rs new file mode 100644 index 0000000..07b6ea2 --- /dev/null +++ b/src/datafetch/sorted_parquet.rs @@ -0,0 +1,202 @@ +//! Utilities for writing sorted and unsorted parquet files. + +use datafusion::arrow::array::RecordBatch; +use datafusion::arrow::datatypes::Schema; +use datafusion::execution::context::SessionContext; +use datafusion::logical_expr::SortExpr; +use datafusion::prelude::*; +use std::path::Path; +use std::sync::Arc; + +use super::batch_writer::BatchWriter; +use super::native::{ParquetConfig, StreamingParquetWriter}; +use super::DataFetchError; + +/// Create a StreamingParquetWriter with optional config. +fn make_writer(path: std::path::PathBuf, config: Option<&ParquetConfig>) -> Box { + match config { + Some(c) => Box::new(StreamingParquetWriter::with_config(path, c.clone())), + None => Box::new(StreamingParquetWriter::new(path)), + } +} + +/// Write batches to a parquet file, sorted by specified columns. +pub async fn write_sorted_parquet( + batches: &[RecordBatch], + schema: &Schema, + sort_columns: &[String], + output_path: &Path, + parquet_config: Option<&ParquetConfig>, +) -> Result { + if batches.is_empty() { + // Write empty file with schema + let mut writer = make_writer(output_path.to_path_buf(), parquet_config); + writer.init(schema)?; + let empty_batch = RecordBatch::new_empty(Arc::new(schema.clone())); + writer.write_batch(&empty_batch)?; + writer.close()?; + return Ok(0); + } + + // Create a DataFusion context to do the sorting + let ctx = SessionContext::new(); + + // Create a memory table from batches + let schema_ref = Arc::new(schema.clone()); + let mem_table = + datafusion::datasource::MemTable::try_new(schema_ref.clone(), vec![batches.to_vec()]) + .map_err(|e| { + DataFetchError::Storage(format!("Failed to create memory table: {}", e)) + })?; + + ctx.register_table("data", Arc::new(mem_table)) + .map_err(|e| DataFetchError::Storage(format!("Failed to register table: {}", e)))?; + + // Build sort expression + let sort_exprs: Vec = sort_columns + .iter() + .map(|c| col(c).sort(true, true)) // ASC, NULLS FIRST + .collect(); + + // Execute sorted query + let df = ctx + .table("data") + .await + .map_err(|e| DataFetchError::Storage(format!("Failed to get table: {}", e)))? + .sort(sort_exprs) + .map_err(|e| DataFetchError::Storage(format!("Failed to sort: {}", e)))?; + + let sorted_batches = df + .collect() + .await + .map_err(|e| DataFetchError::Storage(format!("Failed to collect sorted data: {}", e)))?; + + // Write sorted batches to parquet + let mut writer = make_writer(output_path.to_path_buf(), parquet_config); + writer.init(schema)?; + + let mut row_count = 0; + for batch in &sorted_batches { + row_count += batch.num_rows(); + writer.write_batch(batch)?; + } + + writer.close()?; + Ok(row_count) +} + +/// Write batches to a parquet file without sorting. +pub fn write_parquet( + batches: &[RecordBatch], + schema: &Schema, + output_path: &Path, + parquet_config: Option<&ParquetConfig>, +) -> Result { + let mut writer = make_writer(output_path.to_path_buf(), parquet_config); + writer.init(schema)?; + + if batches.is_empty() { + let empty_batch = RecordBatch::new_empty(Arc::new(schema.clone())); + writer.write_batch(&empty_batch)?; + writer.close()?; + return Ok(0); + } + + let mut row_count = 0; + for batch in batches { + row_count += batch.num_rows(); + writer.write_batch(batch)?; + } + + writer.close()?; + Ok(row_count) +} + +#[cfg(test)] +mod tests { + use super::*; + use datafusion::arrow::array::{Int32Array, StringArray}; + use datafusion::arrow::datatypes::{DataType, Field}; + use tempfile::tempdir; + + #[tokio::test] + async fn test_write_sorted_parquet() { + let dir = tempdir().unwrap(); + let path = dir.path().join("sorted.parquet"); + + let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ]); + + // Create unsorted data + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![ + Arc::new(Int32Array::from(vec![3, 1, 2])), + Arc::new(StringArray::from(vec!["c", "a", "b"])), + ], + ) + .unwrap(); + + let rows = write_sorted_parquet(&[batch], &schema, &["id".to_string()], &path, None) + .await + .unwrap(); + + assert_eq!(rows, 3); + assert!(path.exists()); + + // Read back and verify sorted order + let ctx = SessionContext::new(); + ctx.register_parquet("test", path.to_str().unwrap(), Default::default()) + .await + .unwrap(); + + let df = ctx.sql("SELECT id FROM test").await.unwrap(); + let batches = df.collect().await.unwrap(); + + let ids: Vec = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap() + .values() + .to_vec(); + + assert_eq!(ids, vec![1, 2, 3], "Data should be sorted by id"); + } + + #[tokio::test] + async fn test_write_sorted_parquet_empty() { + let dir = tempdir().unwrap(); + let path = dir.path().join("empty.parquet"); + + let schema = Schema::new(vec![Field::new("id", DataType::Int32, false)]); + + let rows = write_sorted_parquet(&[], &schema, &["id".to_string()], &path, None) + .await + .unwrap(); + + assert_eq!(rows, 0); + assert!(path.exists()); + } + + #[test] + fn test_write_parquet() { + let dir = tempdir().unwrap(); + let path = dir.path().join("unsorted.parquet"); + + let schema = Schema::new(vec![Field::new("id", DataType::Int32, false)]); + + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![Arc::new(Int32Array::from(vec![3, 1, 2]))], + ) + .unwrap(); + + let rows = write_parquet(&[batch], &schema, &path, None).unwrap(); + + assert_eq!(rows, 3); + assert!(path.exists()); + } +} diff --git a/src/datafusion/async_connection_schema.rs b/src/datafusion/async_connection_schema.rs index b79846c..c1921e7 100644 --- a/src/datafusion/async_connection_schema.rs +++ b/src/datafusion/async_connection_schema.rs @@ -79,15 +79,30 @@ impl AsyncSchemaProvider for AsyncConnectionSchema { })?; // Create LazyTableProvider for on-demand data fetching - let provider = Arc::new(LazyTableProvider::new( - schema, - self.source.clone(), - self.catalog.clone(), - self.orchestrator.clone(), - self.connection_id.clone(), - self.schema_name.clone(), - name.to_string(), - )) as Arc; + // Use projections if the orchestrator has them configured + let provider: Arc = + if let Some(registry) = self.orchestrator.index_preset_registry() { + Arc::new(LazyTableProvider::with_index_presets( + schema, + self.source.clone(), + self.catalog.clone(), + self.orchestrator.clone(), + self.connection_id.clone(), + self.schema_name.clone(), + name.to_string(), + Arc::clone(registry), + )) + } else { + Arc::new(LazyTableProvider::new( + schema, + self.source.clone(), + self.catalog.clone(), + self.orchestrator.clone(), + self.connection_id.clone(), + self.schema_name.clone(), + name.to_string(), + )) + }; Ok(Some(provider)) } diff --git a/src/datafusion/lazy_table_provider.rs b/src/datafusion/lazy_table_provider.rs index 200561d..5f7bb70 100644 --- a/src/datafusion/lazy_table_provider.rs +++ b/src/datafusion/lazy_table_provider.rs @@ -3,13 +3,13 @@ use datafusion::arrow::datatypes::SchemaRef; use datafusion::catalog::Session; use datafusion::datasource::{TableProvider, TableType}; use datafusion::error::DataFusionError; -use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; +use datafusion::logical_expr::{Expr, Operator, TableProviderFilterPushDown}; use datafusion::physical_plan::ExecutionPlan; use std::any::Any; use std::sync::Arc; use crate::catalog::CatalogManager; -use crate::datafetch::FetchOrchestrator; +use crate::datafetch::{FetchOrchestrator, IndexPresetRegistry}; use crate::source::Source; /// A lazy table provider that defers data fetching until scan() is called. @@ -17,6 +17,9 @@ use crate::source::Source; /// This provider stores the table schema from catalog metadata and only triggers /// the actual data fetch when a query scan is executed. This enables efficient /// query planning without unnecessary I/O operations. +/// +/// Supports index preset selection: if index presets are configured, the provider +/// analyzes query filters and selects the best preset index based on range filters. #[derive(Debug)] pub struct LazyTableProvider { schema: SchemaRef, @@ -26,6 +29,7 @@ pub struct LazyTableProvider { connection_id: String, schema_name: String, table_name: String, + index_preset_registry: Option>, } impl LazyTableProvider { @@ -46,6 +50,219 @@ impl LazyTableProvider { connection_id, schema_name, table_name, + index_preset_registry: None, + } + } + + /// Create a provider with index preset support. + #[allow(clippy::too_many_arguments)] + pub fn with_index_presets( + schema: SchemaRef, + source: Arc, + catalog: Arc, + orchestrator: Arc, + connection_id: String, + schema_name: String, + table_name: String, + index_preset_registry: Arc, + ) -> Self { + Self { + schema, + source, + catalog, + orchestrator, + connection_id, + schema_name, + table_name, + index_preset_registry: Some(index_preset_registry), + } + } + + /// Select the best preset index for the given filters. + /// Returns the preset name and its sort columns (with direction) if a beneficial preset is found. + /// + /// Index presets help with range/equality filters (>, <, >=, <=, =, BETWEEN) — + /// sorted data has tight row group statistics enabling effective pruning. + fn select_preset_index(&self, filters: &[Expr]) -> Option<(String, Vec<(String, bool)>)> { + let registry = self.index_preset_registry.as_ref()?; + let presets = registry.get(&self.schema_name, &self.table_name)?; + + // Extract column names that have range or equality filters + let filter_columns = Self::extract_filter_columns(filters); + + // First try: match a preset index to a filter column + for preset in presets { + if let Some(first_col) = preset.sort_columns.first() { + let first_col_lower: String = first_col.to_lowercase(); + + if filter_columns + .iter() + .any(|c: &String| c.to_lowercase() == first_col_lower) + { + tracing::debug!( + preset = %preset.name, + column = %first_col, + "Selected preset index for filter match" + ); + // Preset indexes default to ASC + let cols_with_dir: Vec<(String, bool)> = preset + .sort_columns + .iter() + .map(|c| (c.clone(), false)) + .collect(); + return Some((preset.name.clone(), cols_with_dir)); + } + } + } + + None + } + + /// Select the best catalog index for the given filters. + /// Returns the index parquet URL and sort columns (with direction) if a beneficial index is found. + /// + /// This checks indexes created via CREATE INDEX SQL commands. + /// Selection priority: + /// 1. Index whose sort column matches a filter column (best for range pruning) + /// 2. Any available index when no filters exist (enables sort elimination for ORDER BY queries) + async fn select_catalog_index( + &self, + filters: &[Expr], + ) -> Option<(String, Vec<(String, bool)>)> { + // Query catalog for indexes on this table + let indexes = match self + .catalog + .list_indexes(&self.connection_id, &self.schema_name, &self.table_name) + .await + { + Ok(idx) => idx, + Err(e) => { + tracing::warn!("Failed to list indexes: {}", e); + return None; + } + }; + + tracing::debug!( + index_count = indexes.len(), + filter_count = filters.len(), + "select_catalog_index: checking indexes" + ); + + if indexes.is_empty() { + return None; + } + + // Extract column names that have range or equality filters + let filter_columns = Self::extract_filter_columns(filters); + tracing::debug!( + ?filter_columns, + "select_catalog_index: extracted filter columns" + ); + + // First try: match an index to a filter column + for index in &indexes { + // Skip indexes without a parquet path (not yet built) + let Some(parquet_path) = index.parquet_path.as_ref() else { + continue; + }; + let sort_columns = index.sort_columns_with_direction(); + + if let Some((first_col, _)) = sort_columns.first() { + let first_col_lower = first_col.to_lowercase(); + + if filter_columns + .iter() + .any(|c| c.to_lowercase() == first_col_lower) + { + tracing::debug!( + index = %index.index_name, + column = %first_col, + "Selected catalog index for filter match" + ); + return Some((parquet_path.clone(), sort_columns)); + } + } + } + + // Fallback: when no filters are pushed down at all, use any available index. + // The index parquet has the same data as the base, just pre-sorted. + // Declaring the output ordering enables DataFusion to eliminate sorts + // for ORDER BY queries on the index column, with no downside since + // there are no filters that would benefit from base parquet statistics. + // We check filters.is_empty() (not filter_columns) to avoid using the + // index when ANY filter exists — even types we don't extract (IN, LIKE, etc.) + // — since the base parquet may have better row group statistics for those. + if filters.is_empty() { + for index in &indexes { + if let Some(parquet_path) = &index.parquet_path { + let sort_columns = index.sort_columns_with_direction(); + if !sort_columns.is_empty() { + tracing::debug!( + index = %index.index_name, + "Selected catalog index for sort elimination" + ); + return Some((parquet_path.clone(), sort_columns)); + } + } + } + } + + None + } + + /// Extract column names that have range or equality filters applied. + fn extract_filter_columns(filters: &[Expr]) -> Vec { + let mut columns = Vec::new(); + + for filter in filters { + Self::collect_filter_columns(filter, &mut columns); + } + + columns + } + + /// Recursively collect column names from range and equality filter expressions. + fn collect_filter_columns(expr: &Expr, columns: &mut Vec) { + match expr { + Expr::BinaryExpr(binary) => { + // Check if this is a range or equality comparison + let is_filter_op = matches!( + binary.op, + Operator::Lt | Operator::LtEq | Operator::Gt | Operator::GtEq | Operator::Eq + ); + + if is_filter_op { + // Try to extract column name from left side + if let Some(col_name) = Self::extract_column_name(&binary.left) { + columns.push(col_name); + } + // Also check right side (for cases like "10 < col") + if let Some(col_name) = Self::extract_column_name(&binary.right) { + columns.push(col_name); + } + } + + // Recurse into AND expressions + if matches!(binary.op, Operator::And) { + Self::collect_filter_columns(&binary.left, columns); + Self::collect_filter_columns(&binary.right, columns); + } + } + Expr::Between(between) => { + if let Some(col_name) = Self::extract_column_name(&between.expr) { + columns.push(col_name); + } + } + _ => {} + } + } + + /// Extract column name from an expression if it's a column reference. + fn extract_column_name(expr: &Expr) -> Option { + match expr { + Expr::Column(col) => Some(col.name.clone()), + Expr::Cast(cast) => Self::extract_column_name(&cast.expr), + _ => None, } } @@ -59,7 +276,7 @@ impl LazyTableProvider { /// optimizer on DataSourceExec. #[tracing::instrument( name = "load_parquet_exec", - skip(self, state, projection, limit), + skip(self, state, projection, limit, sort_columns), fields(runtimedb.parquet_url = %parquet_url) )] async fn load_parquet_exec( @@ -68,6 +285,7 @@ impl LazyTableProvider { state: &dyn Session, projection: Option<&Vec>, limit: Option, + sort_columns: Option<&Vec<(String, bool)>>, ) -> datafusion::common::Result> { // parquet_url for cache tables is a directory URL (e.g., s3://bucket/.../version/) // Append data.parquet to get the actual file URL. @@ -79,6 +297,7 @@ impl LazyTableProvider { state, projection, limit, + sort_columns, ) .await } @@ -118,19 +337,20 @@ impl TableProvider for LazyTableProvider { #[tracing::instrument( name = "lazy_table_scan", - skip(self, state, projection, _filters, limit), + skip(self, state, projection, filters, limit), fields( runtimedb.connection_id = %self.connection_id, runtimedb.schema = %self.schema_name, runtimedb.table = %self.table_name, runtimedb.cache_hit = tracing::field::Empty, + runtimedb.index_used = tracing::field::Empty, ) )] async fn scan( &self, state: &dyn Session, projection: Option<&Vec>, - _filters: &[Expr], + filters: &[Expr], limit: Option, ) -> datafusion::common::Result> { // Check if table is already cached @@ -159,11 +379,40 @@ impl TableProvider for LazyTableProvider { (url, false) }; - tracing::Span::current().record("runtimedb.cache_hit", cache_hit); + let span = tracing::Span::current(); + span.record("runtimedb.cache_hit", cache_hit); + + // Check if we should use a sorted index for better row group pruning or ORDER BY + // Priority: 1. Catalog indexes (user-created via SQL), 2. Index presets (hardcoded) + let (final_url, sort_columns) = + if let Some((index_url, sort_cols)) = self.select_catalog_index(filters).await { + // Use catalog index - index_url is the full path to the index directory + span.record( + "runtimedb.index_used", + format!("catalog:{}", index_url).as_str(), + ); + (index_url, Some(sort_cols)) + } else if let Some((preset_name, sort_cols)) = self.select_preset_index(filters) { + // Use preset index path: {base_url}/presets/{name} + let preset_url = format!( + "{}/presets/{}", + parquet_url.trim_end_matches('/'), + preset_name + ); + span.record( + "runtimedb.index_used", + format!("preset:{}", preset_name).as_str(), + ); + (preset_url, Some(sort_cols)) + } else { + span.record("runtimedb.index_used", "none"); + (parquet_url, None) + }; // Load the parquet file and create execution plan with projection and limit pushdown. // Filter pushdown is handled by the physical optimizer on DataSourceExec. - self.load_parquet_exec(&parquet_url, state, projection, limit) + // If using a sorted index, pass sort columns so output ordering can be declared. + self.load_parquet_exec(&final_url, state, projection, limit, sort_columns.as_ref()) .await } @@ -175,3 +424,174 @@ impl TableProvider for LazyTableProvider { Ok(vec![TableProviderFilterPushDown::Inexact; filters.len()]) } } + +#[cfg(test)] +mod tests { + use super::*; + use crate::datafetch::IndexPreset; + use datafusion::common::Column; + use datafusion::logical_expr::{lit, BinaryExpr}; + + fn make_provider_with_presets( + schema_name: &str, + table_name: &str, + presets: Vec, + ) -> LazyTableProvider { + use crate::catalog::MockCatalog; + use crate::datafetch::NativeFetcher; + use crate::secrets::SecretManager; + use crate::storage::FilesystemStorage; + use datafusion::arrow::datatypes::{DataType, Field, Schema}; + + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("ship_date", DataType::Date32, true), + ])); + + let source = Arc::new(Source::Duckdb { + path: ":memory:".to_string(), + }); + + // Create minimal mocks + let storage = Arc::new(FilesystemStorage::new("/tmp")); + let catalog: Arc = Arc::new(MockCatalog::new()); + let fetcher = Arc::new(NativeFetcher::new()); + + // Create a dummy SecretManager - just for the test, won't be used + let secret_manager = { + use crate::secrets::{EncryptedCatalogBackend, ENCRYPTED_PROVIDER_TYPE}; + let key = [0x42u8; 32]; + let backend = Arc::new(EncryptedCatalogBackend::new(key, catalog.clone())); + Arc::new(SecretManager::new( + backend, + catalog.clone(), + ENCRYPTED_PROVIDER_TYPE, + )) + }; + + let orchestrator = Arc::new(crate::datafetch::FetchOrchestrator::new( + fetcher, + storage, + catalog.clone(), + secret_manager, + )); + + let mut registry = IndexPresetRegistry::new(); + registry.register(schema_name, table_name, presets); + + LazyTableProvider::with_index_presets( + schema, + source, + catalog, + orchestrator, + "conn".to_string(), + schema_name.to_string(), + table_name.to_string(), + Arc::new(registry), + ) + } + + #[test] + fn test_extract_filter_columns_gt() { + let filter = Expr::BinaryExpr(BinaryExpr::new( + Box::new(Expr::Column(Column::from_name("ship_date"))), + Operator::Gt, + Box::new(lit("2024-01-01")), + )); + + let columns = LazyTableProvider::extract_filter_columns(&[filter]); + assert_eq!(columns, vec!["ship_date"]); + } + + #[test] + fn test_extract_filter_columns_between() { + let filter = Expr::Between(datafusion::logical_expr::Between::new( + Box::new(Expr::Column(Column::from_name("order_date"))), + false, + Box::new(lit("2024-01-01")), + Box::new(lit("2024-12-31")), + )); + + let columns = LazyTableProvider::extract_filter_columns(&[filter]); + assert_eq!(columns, vec!["order_date"]); + } + + #[test] + fn test_extract_filter_columns_equality_included() { + let filter = Expr::BinaryExpr(BinaryExpr::new( + Box::new(Expr::Column(Column::from_name("id"))), + Operator::Eq, + Box::new(lit(42)), + )); + + let columns = LazyTableProvider::extract_filter_columns(&[filter]); + assert_eq!(columns, vec!["id"]); + } + + #[test] + fn test_select_preset_index_matches() { + let provider = make_provider_with_presets( + "main", + "lineitem", + vec![IndexPreset::new("by_shipdate", vec!["ship_date"])], + ); + + let filter = Expr::BinaryExpr(BinaryExpr::new( + Box::new(Expr::Column(Column::from_name("ship_date"))), + Operator::GtEq, + Box::new(lit("2024-01-01")), + )); + + let result = provider.select_preset_index(&[filter]); + assert_eq!( + result, + Some(( + "by_shipdate".to_string(), + vec![("ship_date".to_string(), false)] + )) + ); + } + + #[test] + fn test_select_preset_index_no_match() { + let provider = make_provider_with_presets( + "main", + "lineitem", + vec![IndexPreset::new("by_shipdate", vec!["ship_date"])], + ); + + // Filter on a column that doesn't have a preset index + let filter = Expr::BinaryExpr(BinaryExpr::new( + Box::new(Expr::Column(Column::from_name("id"))), + Operator::Gt, + Box::new(lit(100)), + )); + + let result = provider.select_preset_index(&[filter]); + assert_eq!(result, None); + } + + #[test] + fn test_select_preset_index_case_insensitive() { + let provider = make_provider_with_presets( + "MAIN", + "LINEITEM", + vec![IndexPreset::new("by_shipdate", vec!["SHIP_DATE"])], + ); + + let filter = Expr::BinaryExpr(BinaryExpr::new( + Box::new(Expr::Column(Column::from_name("ship_date"))), + Operator::Lt, + Box::new(lit("2024-12-31")), + )); + + let result = provider.select_preset_index(&[filter]); + assert_eq!( + result, + Some(( + "by_shipdate".to_string(), + vec![("SHIP_DATE".to_string(), false)] + )) + ); + } +} diff --git a/src/datafusion/parquet_exec.rs b/src/datafusion/parquet_exec.rs index 92421e4..90ae581 100644 --- a/src/datafusion/parquet_exec.rs +++ b/src/datafusion/parquet_exec.rs @@ -14,6 +14,7 @@ use datafusion::datasource::source::DataSourceExec; use datafusion::datasource::{TableProvider, TableType}; use datafusion::error::DataFusionError; use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; +use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; use datafusion::physical_plan::ExecutionPlan; use std::any::Any; use std::sync::Arc; @@ -24,12 +25,17 @@ use std::sync::Arc; /// /// `file_url` must be the full URL to the parquet file (e.g., /// `s3://bucket/.../data.parquet` or `file:///tmp/.../data.parquet`). +/// +/// If `sort_columns` is provided, the execution plan will declare that its output +/// is sorted by those columns, allowing DataFusion to skip redundant sorts. +/// Each tuple is (column_name, is_descending). pub async fn build_parquet_exec( file_url: &str, file_schema: SchemaRef, state: &dyn Session, projection: Option<&Vec>, limit: Option, + sort_columns: Option<&Vec<(String, bool)>>, ) -> datafusion::common::Result> { // Parse URL to extract ObjectStoreUrl and object-store-relative path. // ListingTableUrl handles both file:// and s3:// schemes. @@ -47,15 +53,41 @@ pub async fn build_parquet_exec( ) })?; + // In DataFusion 52+, ParquetSource takes the schema, and FileScanConfigBuilder takes 2 args let parquet_source: Arc = - Arc::new(ParquetSource::default()); + Arc::new(ParquetSource::new(file_schema.clone())); - let config = FileScanConfigBuilder::new(object_store_url, file_schema, parquet_source) + let mut config_builder = FileScanConfigBuilder::new(object_store_url, parquet_source) .with_file(object_meta.into()) - .with_projection_indices(projection.cloned()) - .with_limit(limit) - .build(); + .with_projection_indices(projection.cloned())? + .with_limit(limit); + + // If sort columns are provided, declare output ordering so DataFusion can skip sorts + if let Some(cols) = sort_columns { + let sort_exprs: Vec = cols + .iter() + .filter_map(|(col_name, is_descending)| { + // Find the column in the schema and create a sort expression + datafusion::physical_expr::expressions::col(col_name, &file_schema) + .ok() + .map(|expr| PhysicalSortExpr { + expr, + options: datafusion::arrow::compute::SortOptions { + descending: *is_descending, + nulls_first: true, + }, + }) + }) + .collect(); + + if !sort_exprs.is_empty() { + if let Some(ordering) = LexOrdering::new(sort_exprs) { + config_builder = config_builder.with_output_ordering(vec![ordering]); + } + } + } + let config = config_builder.build(); Ok(DataSourceExec::from_data_source(config)) } @@ -103,6 +135,7 @@ impl TableProvider for SingleFileParquetProvider { state, projection, limit, + None, // No sort ordering for SingleFileParquetProvider ) .await } @@ -160,7 +193,7 @@ mod tests { let ctx = SessionContext::new(); let state = ctx.state(); - let plan = build_parquet_exec(&file_url, schema, &state, None, None) + let plan = build_parquet_exec(&file_url, schema, &state, None, None, None) .await .unwrap(); @@ -182,7 +215,7 @@ mod tests { // Project only "value" column (index 1) let projection = vec![1]; - let plan = build_parquet_exec(&file_url, schema, &state, Some(&projection), None) + let plan = build_parquet_exec(&file_url, schema, &state, Some(&projection), None, None) .await .unwrap(); @@ -202,7 +235,7 @@ mod tests { let ctx = SessionContext::new(); let state = ctx.state(); - let plan = build_parquet_exec(&file_url, schema, &state, None, Some(1)) + let plan = build_parquet_exec(&file_url, schema, &state, None, Some(1), None) .await .unwrap(); @@ -226,6 +259,7 @@ mod tests { &state, None, None, + None, ) .await; diff --git a/src/engine.rs b/src/engine.rs index 7c3e561..ce00167 100644 --- a/src/engine.rs +++ b/src/engine.rs @@ -1,6 +1,6 @@ use crate::catalog::{ is_dataset_table_name_conflict, CachingCatalogManager, CatalogManager, ConnectionInfo, - ResultStatus, ResultUpdate, SqliteCatalogManager, TableInfo, + IndexInfo, ResultStatus, ResultUpdate, SqliteCatalogManager, TableInfo, }; use crate::datafetch::native::StreamingParquetWriter; use crate::datafetch::{BatchWriter, FetchOrchestrator, NativeFetcher}; @@ -23,7 +23,8 @@ use datafusion::execution::SessionStateBuilder; use datafusion::prelude::*; use datafusion_tracing::{instrument_with_info_spans, InstrumentationOptions}; use instrumented_object_store::instrument_object_store; -use liquid_cache_client::LiquidCacheClientBuilder; +// Temporarily disabled - waiting for DataFusion 52 support +// use liquid_cache_client::LiquidCacheClientBuilder; use object_store::ObjectStore; use std::collections::{HashMap, HashSet}; use std::path::{Path, PathBuf}; @@ -70,6 +71,162 @@ const INTERNAL_CONNECTION_ID: &str = "_runtimedb_internal"; /// Schema name for storing query results. const RESULTS_SCHEMA_NAME: &str = "runtimedb_results"; +/// A column for CREATE INDEX (always sorted ASC). +#[derive(Debug, Clone)] +struct SortColumn { + name: String, +} + +/// Parsed CREATE INDEX statement. +#[derive(Debug)] +struct CreateIndexStatement { + /// Index name + index_name: String, + /// Full table reference: catalog.schema.table + catalog: String, + schema: String, + table: String, + /// Column(s) to sort by with direction + columns: Vec, +} + +/// Parse a CREATE INDEX statement. +/// Supports: CREATE INDEX index_name ON catalog.schema.table (column1, column2, ...) +fn parse_create_index(sql: &str) -> Option { + let sql_upper = sql.to_uppercase(); + let sql_trimmed = sql.trim(); + + // Check if it starts with CREATE INDEX + if !sql_upper.starts_with("CREATE INDEX") { + return None; + } + + // Use regex-free parsing for simplicity + // Format: CREATE INDEX ON () + let after_create_index = sql_trimmed[12..].trim(); // Skip "CREATE INDEX" + + // Find "ON" keyword + let on_pos = after_create_index.to_uppercase().find(" ON ")?; + let index_name = after_create_index[..on_pos].trim().to_string(); + + let after_on = after_create_index[on_pos + 4..].trim(); // Skip " ON " + + // Find opening parenthesis for columns + let paren_pos = after_on.find('(')?; + let table_ref = after_on[..paren_pos].trim(); + + // Parse table reference (catalog.schema.table) + let parts: Vec<&str> = table_ref.split('.').collect(); + let (catalog, schema, table) = match parts.len() { + 3 => ( + parts[0].trim().to_string(), + parts[1].trim().to_string(), + parts[2].trim().to_string(), + ), + 2 => ( + // Assume first part is catalog (connection name), second is table + // In our case: connection.schema.table, but if only 2 parts, assume it's schema.table + // and we'll need to infer the catalog later + String::new(), + parts[0].trim().to_string(), + parts[1].trim().to_string(), + ), + 1 => (String::new(), String::new(), parts[0].trim().to_string()), + _ => return None, + }; + + // Parse columns from parentheses + // Supports: (col1), (col1, col2), (col1 DESC), (col1 ASC, col2 DESC) + // ASC/DESC keywords are stripped — indexes are always sorted ASC. + let after_paren = &after_on[paren_pos + 1..]; + let close_paren = after_paren.find(')')?; + let columns_str = &after_paren[..close_paren]; + + let columns: Vec = columns_str + .split(',') + .filter_map(|c| { + let c = c.trim(); + if c.is_empty() { + return None; + } + let c_upper = c.to_uppercase(); + let name = if c_upper.ends_with(" DESC") { + c[..c.len() - 5].trim().to_string() + } else if c_upper.ends_with(" ASC") { + c[..c.len() - 4].trim().to_string() + } else { + c.to_string() + }; + Some(SortColumn { name }) + }) + .collect(); + + if columns.is_empty() { + return None; + } + + Some(CreateIndexStatement { + index_name, + catalog, + schema, + table, + columns, + }) +} + +/// Parsed DROP INDEX statement. +#[derive(Debug)] +struct DropIndexStatement { + index_name: String, + catalog: String, + schema: String, + table: String, +} + +/// Parse a DROP INDEX statement. +/// Supports: DROP INDEX index_name ON catalog.schema.table +fn parse_drop_index(sql: &str) -> Option { + let sql_upper = sql.to_uppercase(); + let sql_trimmed = sql.trim(); + + if !sql_upper.starts_with("DROP INDEX") { + return None; + } + + let after_drop_index = sql_trimmed[10..].trim(); // Skip "DROP INDEX" + + // Find "ON" keyword + let on_pos = after_drop_index.to_uppercase().find(" ON ")?; + let index_name = after_drop_index[..on_pos].trim().to_string(); + + let after_on = after_drop_index[on_pos + 4..].trim(); + + // Parse table reference + let table_ref = after_on.trim_end_matches(';').trim(); + let parts: Vec<&str> = table_ref.split('.').collect(); + let (catalog, schema, table) = match parts.len() { + 3 => ( + parts[0].trim().to_string(), + parts[1].trim().to_string(), + parts[2].trim().to_string(), + ), + 2 => ( + String::new(), + parts[0].trim().to_string(), + parts[1].trim().to_string(), + ), + 1 => (String::new(), String::new(), parts[0].trim().to_string()), + _ => return None, + }; + + Some(DropIndexStatement { + index_name, + catalog, + schema, + table, + }) +} + /// Result of a query execution with optional persistence. pub struct QueryResponse { pub schema: Arc, @@ -123,6 +280,8 @@ pub struct RuntimeEngine { persistence_tasks: Mutex>, /// Handle for the stale result cleanup worker task. stale_result_cleanup_handle: Mutex>>, + /// Optional parquet configuration for controlling row group size and bloom filters. + parquet_config: Option, } impl RuntimeEngine { @@ -431,6 +590,21 @@ impl RuntimeEngine { tracing::Span::current().record("runtimedb.sql", sql); } + // Intercept CREATE INDEX statements + if let Some(create_index) = parse_create_index(sql) { + return self.execute_create_index_query(create_index, start).await; + } + + // Intercept DROP INDEX statements + if let Some(drop_index) = parse_drop_index(sql) { + return self.execute_drop_index_query(drop_index, start).await; + } + + // Intercept SHOW INDEXES statements + if sql.trim().to_uppercase().starts_with("SHOW INDEXES") { + return self.execute_show_indexes_query(sql, start).await; + } + let session_state = self.df_ctx.state(); // Step 1: Parse SQL into a statement @@ -510,6 +684,218 @@ impl RuntimeEngine { }) } + /// Execute a CREATE INDEX statement and return results. + async fn execute_create_index_query( + &self, + stmt: CreateIndexStatement, + start: Instant, + ) -> Result { + use datafusion::arrow::array::StringArray; + use datafusion::arrow::datatypes::{DataType, Field}; + + // For CREATE INDEX, we need to resolve the connection_id from the catalog name + // The catalog name in SQL corresponds to the connection name + let connection_id = if stmt.catalog.is_empty() { + // No catalog specified - this is an error for index creation + return Err(anyhow::anyhow!( + "CREATE INDEX requires fully qualified table name: catalog.schema.table" + )); + } else { + // Look up connection by name (catalog = connection name) + let conn = self + .catalog + .get_connection_by_name(&stmt.catalog) + .await? + .ok_or_else(|| anyhow::anyhow!("Connection '{}' not found", stmt.catalog))?; + conn.id + }; + + // Create the index + let col_names: Vec = stmt.columns.iter().map(|c| c.name.clone()).collect(); + let index_info = self + .create_index( + &connection_id, + &stmt.schema, + &stmt.table, + &stmt.index_name, + &col_names, + ) + .await?; + + // Return a result set with the created index info + let schema = Arc::new(Schema::new(vec![ + Field::new("status", DataType::Utf8, false), + Field::new("index_name", DataType::Utf8, false), + Field::new("table", DataType::Utf8, false), + Field::new("columns", DataType::Utf8, false), + ])); + + let status = StringArray::from(vec!["INDEX CREATED"]); + let name = StringArray::from(vec![index_info.index_name.as_str()]); + let table = StringArray::from(vec![format!( + "{}.{}.{}", + stmt.catalog, stmt.schema, stmt.table + )]); + let columns_str: Vec = stmt.columns.iter().map(|c| c.name.clone()).collect(); + let columns = StringArray::from(vec![columns_str.join(", ")]); + + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(status), + Arc::new(name), + Arc::new(table), + Arc::new(columns), + ], + )?; + + Ok(QueryResponse { + schema, + execution_time: start.elapsed(), + results: vec![batch], + }) + } + + /// Execute a DROP INDEX statement and return results. + async fn execute_drop_index_query( + &self, + stmt: DropIndexStatement, + start: Instant, + ) -> Result { + use datafusion::arrow::array::StringArray; + use datafusion::arrow::datatypes::{DataType, Field}; + + // Resolve connection_id from catalog name + let connection_id = if stmt.catalog.is_empty() { + return Err(anyhow::anyhow!( + "DROP INDEX requires fully qualified table name: catalog.schema.table" + )); + } else { + let conn = self + .catalog + .get_connection_by_name(&stmt.catalog) + .await? + .ok_or_else(|| anyhow::anyhow!("Connection '{}' not found", stmt.catalog))?; + conn.id + }; + + // Drop the index + let dropped = self + .drop_index(&connection_id, &stmt.schema, &stmt.table, &stmt.index_name) + .await?; + + // Return result + let schema = Arc::new(Schema::new(vec![ + Field::new("status", DataType::Utf8, false), + Field::new("index_name", DataType::Utf8, false), + ])); + + let (status, name) = if dropped.is_some() { + ("INDEX DROPPED", stmt.index_name.as_str()) + } else { + ("INDEX NOT FOUND", stmt.index_name.as_str()) + }; + + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(StringArray::from(vec![status])), + Arc::new(StringArray::from(vec![name])), + ], + )?; + + Ok(QueryResponse { + schema, + execution_time: start.elapsed(), + results: vec![batch], + }) + } + + /// Execute a SHOW INDEXES statement and return results. + /// Format: SHOW INDEXES ON catalog.schema.table + async fn execute_show_indexes_query(&self, sql: &str, start: Instant) -> Result { + use datafusion::arrow::array::{StringArray, TimestampMicrosecondArray}; + use datafusion::arrow::datatypes::{DataType, Field, TimeUnit}; + + // Parse SHOW INDEXES ON catalog.schema.table + let sql_upper = sql.to_uppercase(); + let on_pos = sql_upper.find(" ON ").ok_or_else(|| { + anyhow::anyhow!("SHOW INDEXES requires ON clause: SHOW INDEXES ON catalog.schema.table") + })?; + + let table_ref = sql[on_pos + 4..].trim().trim_end_matches(';').trim(); + let parts: Vec<&str> = table_ref.split('.').collect(); + + let (catalog, schema_name, table_name) = match parts.len() { + 3 => (parts[0].trim(), parts[1].trim(), parts[2].trim()), + _ => { + return Err(anyhow::anyhow!( + "SHOW INDEXES requires fully qualified table name: catalog.schema.table" + )) + } + }; + + // Resolve connection_id + let conn = self + .catalog + .get_connection_by_name(catalog) + .await? + .ok_or_else(|| anyhow::anyhow!("Connection '{}' not found", catalog))?; + + // Get indexes + let indexes = self.list_indexes(&conn.id, schema_name, table_name).await?; + + // Build result schema + let result_schema = Arc::new(Schema::new(vec![ + Field::new("index_name", DataType::Utf8, false), + Field::new("columns", DataType::Utf8, false), + Field::new("parquet_path", DataType::Utf8, true), + Field::new( + "created_at", + DataType::Timestamp(TimeUnit::Microsecond, Some("UTC".into())), + false, + ), + ])); + + if indexes.is_empty() { + let batch = RecordBatch::new_empty(result_schema.clone()); + return Ok(QueryResponse { + schema: result_schema, + execution_time: start.elapsed(), + results: vec![batch], + }); + } + + let names: Vec<&str> = indexes.iter().map(|i| i.index_name.as_str()).collect(); + let columns: Vec = indexes + .iter() + .map(|i| i.sort_columns_vec().join(", ")) + .collect(); + let paths: Vec> = indexes.iter().map(|i| i.parquet_path.as_deref()).collect(); + let created_ats: Vec = indexes + .iter() + .map(|i| i.created_at.timestamp_micros()) + .collect(); + + let batch = RecordBatch::try_new( + result_schema.clone(), + vec![ + Arc::new(StringArray::from(names)), + Arc::new(StringArray::from( + columns.iter().map(|s| s.as_str()).collect::>(), + )), + Arc::new(StringArray::from(paths)), + Arc::new(TimestampMicrosecondArray::from(created_ats).with_timezone("UTC")), + ], + )?; + + Ok(QueryResponse { + schema: result_schema, + execution_time: start.elapsed(), + results: vec![batch], + }) + } + /// Execute a SQL query and persist results asynchronously. /// /// This method: @@ -1170,6 +1556,37 @@ impl RuntimeEngine { } } + // Invalidate indexes built from old data + match self + .invalidate_table_indexes(connection_id, schema_name, table_name) + .await + { + Ok(count) if count > 0 => { + warnings.push(RefreshWarning { + schema_name: Some(schema_name.to_string()), + table_name: Some(table_name.to_string()), + message: format!( + "{} index(es) were dropped because the underlying data changed. Recreate them with CREATE INDEX.", + count + ), + }); + } + Err(e) => { + tracing::warn!( + schema = schema_name, + table = table_name, + error = %e, + "Failed to invalidate indexes after resync" + ); + warnings.push(RefreshWarning { + schema_name: Some(schema_name.to_string()), + table_name: Some(table_name.to_string()), + message: format!("Failed to invalidate indexes after resync: {}", e), + }); + } + _ => {} + } + tracing::Span::current() .record("runtimedb.rows_synced", rows_synced) .record("runtimedb.warnings_count", warnings.len()); @@ -1304,6 +1721,330 @@ impl RuntimeEngine { Ok(result) } + // ========================================================================= + // Index Management + // ========================================================================= + + /// Execute a CREATE INDEX statement. + /// + /// Creates a sorted copy of the table's cached data, sorted by the specified columns. + /// The index is stored in the catalog and can be used for efficient range queries. + #[tracing::instrument( + name = "create_index", + skip(self), + fields( + runtimedb.index_name = %index_name, + runtimedb.connection_id = %connection_id, + runtimedb.schema = %schema_name, + runtimedb.table = %table_name, + runtimedb.rows_written = tracing::field::Empty, + ) + )] + pub async fn create_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + sort_columns: &[String], + ) -> Result { + use crate::datafetch::native::StreamingParquetWriter; + use crate::datafetch::BatchWriter; + + info!( + "Creating index {} on {}.{}.{} with columns {:?}", + index_name, connection_id, schema_name, table_name, sort_columns + ); + + // 1. Verify the table exists and has cached data + let table_info = self + .catalog + .get_table(connection_id, schema_name, table_name) + .await? + .ok_or_else(|| { + anyhow::anyhow!( + "Table {}.{} not found in connection {}", + schema_name, + table_name, + connection_id + ) + })?; + + let parquet_path = table_info.parquet_path.as_ref().ok_or_else(|| { + anyhow::anyhow!( + "Table {}.{} has no cached data. Run SYNC TABLE first.", + schema_name, + table_name + ) + })?; + + // 2. Check if index already exists + if self + .catalog + .get_index(connection_id, schema_name, table_name, index_name) + .await? + .is_some() + { + return Err(anyhow::anyhow!( + "Index {} already exists on {}.{}", + index_name, + schema_name, + table_name + )); + } + + // 3. Read the source parquet file + // Indexes write to subdirectories of the table's cache path. For remote storage + // (S3), we'd need to upload the index files — not yet supported. + if !parquet_path.starts_with("file://") { + return Err(anyhow::anyhow!( + "CREATE INDEX is not yet supported for remote-storage tables. \ + Table {}.{} is stored at {}", + schema_name, + table_name, + parquet_path + )); + } + + let (local_path, _is_temp) = self.storage.get_local_path(parquet_path).await?; + + // Find parquet files in the directory + let parquet_files = if local_path.is_dir() { + let mut files = Vec::new(); + for entry in std::fs::read_dir(&local_path)? { + let entry = entry?; + let path = entry.path(); + if path.extension().map(|e| e == "parquet").unwrap_or(false) { + files.push(path); + } + } + files.sort(); + files + } else { + vec![local_path.clone()] + }; + + if parquet_files.is_empty() { + return Err(anyhow::anyhow!( + "No parquet files found at {}", + parquet_path + )); + } + + // Read all batches from the source files + let ctx = datafusion::prelude::SessionContext::new(); + let mut all_batches = Vec::new(); + let mut schema = None; + + for file_path in &parquet_files { + ctx.register_parquet("source", file_path.to_str().unwrap(), Default::default()) + .await?; + + let df = ctx.table("source").await?; + if schema.is_none() { + schema = Some(df.schema().inner().as_ref().clone()); + } + let batches = df.collect().await?; + all_batches.extend(batches); + + ctx.deregister_table("source")?; + } + + let schema = schema.ok_or_else(|| anyhow::anyhow!("Failed to get schema"))?; + + // 4. Validate sort columns exist in schema + let schema_fields: Vec = schema.fields().iter().map(|f| f.name().clone()).collect(); + for col in sort_columns { + if !schema_fields.iter().any(|f| f == col) { + return Err(anyhow::anyhow!( + "Column '{}' not found in table {}.{}. Available columns: {}", + col, + schema_name, + table_name, + schema_fields.join(", ") + )); + } + } + + // 5. Sort and write to index directory + // Use a unique suffix to prevent race conditions with async file deletion + // Write inside the version directory so the URL (derived from parquet_path) matches + let unique_suffix = nanoid::nanoid!(8); + let index_dir = local_path + .join("indexes") + .join(format!("{}_{}", index_name, unique_suffix)); + std::fs::create_dir_all(&index_dir)?; + + let index_parquet_path = index_dir.join("data.parquet"); + + // Sort and write using DataFusion + let schema_ref = Arc::new(schema.clone()); + let mem_table = + datafusion::datasource::MemTable::try_new(schema_ref.clone(), vec![all_batches])?; + + ctx.register_table("data", Arc::new(mem_table))?; + + // Create sort expressions — always ASC, nulls first + let sort_exprs: Vec = sort_columns + .iter() + .map(|c| datafusion::prelude::col(c).sort(true, true)) + .collect(); + + let df = ctx.table("data").await?.sort(sort_exprs)?; + let sorted_batches = df.collect().await?; + + // Write sorted data + let mut writer: Box = match &self.parquet_config { + Some(config) => Box::new(StreamingParquetWriter::with_config( + index_parquet_path.clone(), + config.clone(), + )), + None => Box::new(StreamingParquetWriter::new(index_parquet_path.clone())), + }; + writer.init(&schema)?; + + let mut row_count = 0; + for batch in &sorted_batches { + row_count += batch.num_rows(); + writer.write_batch(batch)?; + } + writer.close()?; + + tracing::Span::current().record("runtimedb.rows_written", row_count); + + // 6. Create catalog entry with parquet path (only after successful write) + // Derive the canonical URL from the table's parquet_path, not from the local filesystem + let index_url = format!( + "{}/indexes/{}_{}", + parquet_path.trim_end_matches('/'), + index_name, + unique_suffix + ); + self.catalog + .create_index( + connection_id, + schema_name, + table_name, + index_name, + sort_columns, + Some(&index_url), + ) + .await?; + + // 7. Return the created index info + let index_info = self + .catalog + .get_index(connection_id, schema_name, table_name, index_name) + .await? + .ok_or_else(|| anyhow::anyhow!("Failed to retrieve created index"))?; + + info!( + "Created index {} with {} rows at {}", + index_name, row_count, index_url + ); + + Ok(index_info) + } + + /// Execute a DROP INDEX statement. + #[tracing::instrument( + name = "drop_index", + skip(self), + fields( + runtimedb.index_name = %index_name, + runtimedb.connection_id = %connection_id, + runtimedb.schema = %schema_name, + runtimedb.table = %table_name, + ) + )] + pub async fn drop_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + ) -> Result> { + info!( + "Dropping index {} on {}.{}.{}", + index_name, connection_id, schema_name, table_name + ); + + // Delete from catalog (returns the old info if it existed) + let deleted_index = self + .catalog + .delete_index(connection_id, schema_name, table_name, index_name) + .await?; + + // If there was an index, schedule deletion of its files + if let Some(ref index_info) = deleted_index { + if let Some(ref parquet_path) = index_info.parquet_path { + if let Err(e) = self.schedule_file_deletion(parquet_path).await { + warn!( + "Failed to schedule deletion of index files at {}: {}", + parquet_path, e + ); + } + } + } + + Ok(deleted_index) + } + + /// List all indexes for a table. + pub async fn list_indexes( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + ) -> Result> { + self.catalog + .list_indexes(connection_id, schema_name, table_name) + .await + } + + /// Invalidate all indexes for a table by removing catalog entries and scheduling file deletion. + /// + /// Called after table resync to ensure stale indexes (built from old data) are cleaned up. + /// Returns the number of indexes invalidated. + async fn invalidate_table_indexes( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + ) -> Result { + let indexes = self + .catalog + .list_indexes(connection_id, schema_name, table_name) + .await?; + + let count = indexes.len(); + for index in &indexes { + // Delete catalog entry + self.catalog + .delete_index(connection_id, schema_name, table_name, &index.index_name) + .await?; + + // Schedule file deletion if the index had a parquet path + if let Some(ref parquet_path) = index.parquet_path { + if let Err(e) = self.schedule_file_deletion(parquet_path).await { + warn!( + "Failed to schedule deletion of index files at {}: {}", + parquet_path, e + ); + } + } + } + + if count > 0 { + info!( + "Invalidated {} index(es) on {}.{} due to table resync", + count, schema_name, table_name + ); + } + + Ok(count) + } + /// Store an uploaded file and create an upload record. pub async fn store_upload( &self, @@ -2286,6 +3027,8 @@ pub struct RuntimeEngineBuilder { cache_config: Option, stale_result_cleanup_interval: Duration, stale_result_timeout: Duration, + index_preset_registry: Option, + parquet_config: Option, } impl Default for RuntimeEngineBuilder { @@ -2315,9 +3058,33 @@ impl RuntimeEngineBuilder { DEFAULT_STALE_RESULT_CLEANUP_INTERVAL_SECS, ), stale_result_timeout: Duration::from_secs(DEFAULT_STALE_RESULT_TIMEOUT_SECS), + index_preset_registry: None, + parquet_config: None, } } + /// Enable TPC-H optimized index presets. + /// Creates sorted copies of large tables for efficient row-group pruning on range queries. + pub fn with_tpch_index_presets(mut self) -> Self { + self.index_preset_registry = Some(crate::datafetch::IndexPresetRegistry::tpch_optimized()); + self + } + + /// Set a custom index preset registry. + pub fn index_preset_registry( + mut self, + registry: crate::datafetch::IndexPresetRegistry, + ) -> Self { + self.index_preset_registry = Some(registry); + self + } + + /// Set a custom parquet configuration for controlling row group size and bloom filters. + pub fn parquet_config(mut self, config: crate::datafetch::ParquetConfig) -> Self { + self.parquet_config = Some(config); + self + } + /// Set the base directory for all RuntimeDB data. /// Defaults to ~/.hotdata/runtimedb if not set. pub fn base_dir(mut self, dir: impl Into) -> Self { @@ -2541,12 +3308,28 @@ impl RuntimeEngineBuilder { // Step 7: Create fetch orchestrator (needs secret_manager) let fetcher = Arc::new(NativeFetcher::new()); - let orchestrator = Arc::new(FetchOrchestrator::new( - fetcher, - storage.clone(), - catalog.clone(), - secret_manager.clone(), - )); + let mut orchestrator = if let Some(registry) = self.index_preset_registry { + FetchOrchestrator::with_index_presets( + fetcher, + storage.clone(), + catalog.clone(), + secret_manager.clone(), + Arc::new(registry), + ) + } else { + FetchOrchestrator::new( + fetcher, + storage.clone(), + catalog.clone(), + secret_manager.clone(), + ) + }; + + if let Some(config) = &self.parquet_config { + orchestrator = orchestrator.with_parquet_config(config.clone()); + } + + let orchestrator = Arc::new(orchestrator); // Create shutdown token for graceful shutdown let shutdown_token = CancellationToken::new(); @@ -2593,6 +3376,7 @@ impl RuntimeEngineBuilder { persistence_semaphore: Arc::new(Semaphore::new(self.max_concurrent_persistence)), persistence_tasks: Mutex::new(JoinSet::new()), stale_result_cleanup_handle: Mutex::new(stale_result_cleanup_handle), + parquet_config: self.parquet_config, }; // Note: All catalogs (connections, datasets, runtimedb) are now resolved on-demand @@ -2743,25 +3527,12 @@ fn build_instrumented_context( object_stores: Vec<(ObjectStoreUrl, Arc)>, liquid_cache_config: Option, ) -> Result { - // When liquid-cache is enabled, use LiquidCacheClientBuilder - // Object stores are registered with the builder for the server, not locally - if let Some((server_address, store_configs)) = liquid_cache_config { - info!(server = %server_address, "Building liquid-cache session context"); - - let mut liquid_cache_builder = LiquidCacheClientBuilder::new(&server_address); - - // Register object stores with liquid-cache builder (server needs these configs) - for (url, options) in store_configs { - info!(url = %url.as_str(), "Registering object store with liquid-cache"); - liquid_cache_builder = liquid_cache_builder.with_object_store(url, Some(options)); - } - - let session_config = SessionConfig::new() - .set_bool("datafusion.execution.parquet.bloom_filter_on_read", true); - return Ok(liquid_cache_builder.build(session_config)?); + // Liquid-cache temporarily disabled - waiting for DataFusion 52 support + if liquid_cache_config.is_some() { + warn!("Liquid-cache is temporarily disabled pending DataFusion 52 upgrade"); } - // Non-liquid-cache path: build with full local instrumentation + // Build with full local instrumentation let runtime_env = Arc::new(RuntimeEnv::default()); // Register instrumented object stores (using actual pre-built stores) @@ -2778,9 +3549,8 @@ fn build_instrumented_context( runtime_env.register_object_store(url.as_ref(), instrumented_store); } - // Configure session — enable bloom filter reads for point lookups - let session_config = - SessionConfig::new().set_bool("datafusion.execution.parquet.bloom_filter_on_read", true); + // Configure session (bloom filter reads disabled - files don't have bloom filters) + let session_config = SessionConfig::new(); // Build SessionState with tracing instrumentation let state_builder = SessionStateBuilder::new() @@ -3204,4 +3974,130 @@ mod tests { .unwrap(); assert_eq!(name_col.value(0), "Alice"); } + + // ========================================================================= + // CREATE INDEX / DROP INDEX Parsing Tests + // ========================================================================= + + #[test] + fn test_parse_create_index_single_column() { + let sql = "CREATE INDEX idx_date ON tpch.main.orders (o_orderdate)"; + let result = parse_create_index(sql).unwrap(); + + assert_eq!(result.index_name, "idx_date"); + assert_eq!(result.catalog, "tpch"); + assert_eq!(result.schema, "main"); + assert_eq!(result.table, "orders"); + assert_eq!(result.columns.len(), 1); + assert_eq!(result.columns[0].name, "o_orderdate"); + } + + #[test] + fn test_parse_create_index_multiple_columns() { + let sql = "CREATE INDEX idx_composite ON mydb.public.lineitem (l_shipdate, l_quantity)"; + let result = parse_create_index(sql).unwrap(); + + assert_eq!(result.index_name, "idx_composite"); + assert_eq!(result.catalog, "mydb"); + assert_eq!(result.schema, "public"); + assert_eq!(result.table, "lineitem"); + assert_eq!(result.columns.len(), 2); + assert_eq!(result.columns[0].name, "l_shipdate"); + assert_eq!(result.columns[1].name, "l_quantity"); + } + + #[test] + fn test_parse_create_index_with_desc() { + let sql = "CREATE INDEX idx ON cat.sch.tbl (col1 DESC, col2 ASC, col3)"; + let result = parse_create_index(sql).unwrap(); + + assert_eq!(result.columns.len(), 3); + assert_eq!(result.columns[0].name, "col1"); + assert_eq!(result.columns[1].name, "col2"); + assert_eq!(result.columns[2].name, "col3"); + } + + #[test] + fn test_parse_create_index_with_spaces() { + let sql = "CREATE INDEX my_idx ON catalog.schema.table ( col1 , col2 )"; + let result = parse_create_index(sql).unwrap(); + + assert_eq!(result.index_name, "my_idx"); + assert_eq!(result.columns.len(), 2); + } + + #[test] + fn test_parse_create_index_lowercase() { + let sql = "create index idx on cat.sch.tbl (col desc)"; + let result = parse_create_index(sql).unwrap(); + + assert_eq!(result.index_name, "idx"); + assert_eq!(result.catalog, "cat"); + assert_eq!(result.schema, "sch"); + assert_eq!(result.table, "tbl"); + assert_eq!(result.columns.len(), 1); + assert_eq!(result.columns[0].name, "col"); + } + + #[test] + fn test_parse_create_index_two_part_table_ref() { + let sql = "CREATE INDEX idx ON schema.table (col)"; + let result = parse_create_index(sql).unwrap(); + + assert_eq!(result.catalog, ""); + assert_eq!(result.schema, "schema"); + assert_eq!(result.table, "table"); + } + + #[test] + fn test_parse_create_index_invalid() { + // Missing ON keyword + assert!(parse_create_index("CREATE INDEX idx table (col)").is_none()); + + // Missing columns + assert!(parse_create_index("CREATE INDEX idx ON table").is_none()); + + // Empty columns + assert!(parse_create_index("CREATE INDEX idx ON table ()").is_none()); + + // Not a CREATE INDEX statement + assert!(parse_create_index("SELECT * FROM table").is_none()); + } + + #[test] + fn test_parse_drop_index_full() { + let sql = "DROP INDEX idx_date ON tpch.main.orders"; + let result = parse_drop_index(sql).unwrap(); + + assert_eq!(result.index_name, "idx_date"); + assert_eq!(result.catalog, "tpch"); + assert_eq!(result.schema, "main"); + assert_eq!(result.table, "orders"); + } + + #[test] + fn test_parse_drop_index_with_semicolon() { + let sql = "DROP INDEX my_idx ON cat.sch.tbl;"; + let result = parse_drop_index(sql).unwrap(); + + assert_eq!(result.index_name, "my_idx"); + assert_eq!(result.table, "tbl"); + } + + #[test] + fn test_parse_drop_index_lowercase() { + let sql = "drop index idx on cat.sch.tbl"; + let result = parse_drop_index(sql).unwrap(); + + assert_eq!(result.index_name, "idx"); + } + + #[test] + fn test_parse_drop_index_invalid() { + // Missing ON keyword + assert!(parse_drop_index("DROP INDEX idx table").is_none()); + + // Not a DROP INDEX statement + assert!(parse_drop_index("DROP TABLE foo").is_none()); + } } diff --git a/tests/result_persistence_tests.rs b/tests/result_persistence_tests.rs index 2f1384d..1b0e9ba 100644 --- a/tests/result_persistence_tests.rs +++ b/tests/result_persistence_tests.rs @@ -9,8 +9,8 @@ use chrono::{DateTime, Utc}; use datafusion::prelude::SessionContext; use rand::RngCore; use runtimedb::catalog::{ - CatalogManager, ConnectionInfo, DatasetInfo, OptimisticLock, PendingDeletion, QueryResult, - ResultStatus, ResultUpdate, SqliteCatalogManager, TableInfo, UploadInfo, + CatalogManager, ConnectionInfo, DatasetInfo, IndexInfo, OptimisticLock, PendingDeletion, + QueryResult, ResultStatus, ResultUpdate, SqliteCatalogManager, TableInfo, UploadInfo, }; use runtimedb::http::app_server::{AppServer, PATH_QUERY, PATH_RESULT, PATH_RESULTS}; use runtimedb::secrets::{SecretMetadata, SecretStatus}; @@ -323,6 +323,64 @@ impl CatalogManager for FailingCatalog { async fn delete_dataset(&self, id: &str) -> Result> { self.inner.delete_dataset(id).await } + + // Index management methods + + async fn create_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + sort_columns: &[String], + parquet_path: Option<&str>, + ) -> Result { + self.inner + .create_index( + connection_id, + schema_name, + table_name, + index_name, + sort_columns, + parquet_path, + ) + .await + } + + async fn get_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + ) -> Result> { + self.inner + .get_index(connection_id, schema_name, table_name, index_name) + .await + } + + async fn list_indexes( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + ) -> Result> { + self.inner + .list_indexes(connection_id, schema_name, table_name) + .await + } + + async fn delete_index( + &self, + connection_id: &str, + schema_name: &str, + table_name: &str, + index_name: &str, + ) -> Result> { + self.inner + .delete_index(connection_id, schema_name, table_name, index_name) + .await + } } async fn setup_test() -> Result<(AppServer, TempDir)> { diff --git a/tests/tpch_benchmark_tests.rs b/tests/tpch_benchmark_tests.rs new file mode 100644 index 0000000..95eb439 --- /dev/null +++ b/tests/tpch_benchmark_tests.rs @@ -0,0 +1,2210 @@ +//! TPC-H Benchmark Tests for RuntimeDB. +//! +//! These tests measure query performance using standard TPC-H benchmark queries +//! to enable optimization comparisons. +//! +//! Usage: +//! ```bash +//! # Quick test (CI, ~10MB data) +//! cargo test tpch_benchmark -- --nocapture +//! +//! # Full benchmark (1GB data) +//! TPCH_SCALE_FACTOR=1.0 cargo test tpch_benchmark --release -- --nocapture +//! +//! # Or use ignored test for full benchmark +//! cargo test tpch_benchmark_full --release -- --ignored --nocapture +//! ``` + +use anyhow::Result; +use axum::{ + body::Body, + http::{Request, StatusCode}, + Router, +}; +use base64::{engine::general_purpose::STANDARD, Engine}; +use rand::RngCore; +use runtimedb::datafetch::ParquetConfig; +use runtimedb::http::app_server::{AppServer, PATH_CONNECTIONS}; +use runtimedb::RuntimeEngine; +use serde_json::json; +use std::sync::Arc; +use std::time::{Duration, Instant}; +use tempfile::TempDir; +use tower::util::ServiceExt; + +/// Generate a test secret key (base64-encoded 32 bytes) +fn generate_test_secret_key() -> String { + let mut key = [0u8; 32]; + rand::thread_rng().fill_bytes(&mut key); + STANDARD.encode(key) +} + +/// TPC-H benchmark query definitions +struct TpchQuery { + name: &'static str, + sql: &'static str, +} + +/// TPC-H Q1: Pricing Summary Report +/// Pattern: Aggregation with GROUP BY over lineitem +const TPCH_Q1: TpchQuery = TpchQuery { + name: "Q1", + sql: r#" +SELECT + l_returnflag, + l_linestatus, + SUM(l_quantity) AS sum_qty, + SUM(l_extendedprice) AS sum_base_price, + SUM(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + SUM(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + AVG(l_quantity) AS avg_qty, + AVG(l_extendedprice) AS avg_price, + AVG(l_discount) AS avg_disc, + COUNT(*) AS count_order +FROM + tpch.main.lineitem +WHERE + l_shipdate <= DATE '1998-12-01' - INTERVAL '90' DAY +GROUP BY + l_returnflag, + l_linestatus +ORDER BY + l_returnflag, + l_linestatus +"#, +}; + +/// TPC-H Q3: Shipping Priority +/// Pattern: 3-way join (customer-orders-lineitem) +const TPCH_Q3: TpchQuery = TpchQuery { + name: "Q3", + sql: r#" +SELECT + l_orderkey, + SUM(l_extendedprice * (1 - l_discount)) AS revenue, + o_orderdate, + o_shippriority +FROM + tpch.main.customer, + tpch.main.orders, + tpch.main.lineitem +WHERE + c_mktsegment = 'BUILDING' + AND c_custkey = o_custkey + AND l_orderkey = o_orderkey + AND o_orderdate < DATE '1995-03-15' + AND l_shipdate > DATE '1995-03-15' +GROUP BY + l_orderkey, + o_orderdate, + o_shippriority +ORDER BY + revenue DESC, + o_orderdate +LIMIT 10 +"#, +}; + +/// TPC-H Q5: Local Supplier Volume +/// Pattern: 6-way join (all dimension tables) +const TPCH_Q5: TpchQuery = TpchQuery { + name: "Q5", + sql: r#" +SELECT + n_name, + SUM(l_extendedprice * (1 - l_discount)) AS revenue +FROM + tpch.main.customer, + tpch.main.orders, + tpch.main.lineitem, + tpch.main.supplier, + tpch.main.nation, + tpch.main.region +WHERE + c_custkey = o_custkey + AND l_orderkey = o_orderkey + AND l_suppkey = s_suppkey + AND c_nationkey = s_nationkey + AND s_nationkey = n_nationkey + AND n_regionkey = r_regionkey + AND r_name = 'ASIA' + AND o_orderdate >= DATE '1994-01-01' + AND o_orderdate < DATE '1995-01-01' +GROUP BY + n_name +ORDER BY + revenue DESC +"#, +}; + +/// TPC-H Q2: Minimum Cost Supplier +/// Pattern: Correlated subquery +const TPCH_Q2: TpchQuery = TpchQuery { + name: "Q2", + sql: r#" +SELECT + s_acctbal, s_name, n_name, p_partkey, p_mfgr, + s_address, s_phone, s_comment +FROM + tpch.main.part, + tpch.main.supplier, + tpch.main.partsupp, + tpch.main.nation, + tpch.main.region +WHERE + p_partkey = ps_partkey + AND s_suppkey = ps_suppkey + AND p_size = 15 + AND p_type LIKE '%BRASS' + AND s_nationkey = n_nationkey + AND n_regionkey = r_regionkey + AND r_name = 'EUROPE' + AND ps_supplycost = ( + SELECT MIN(ps_supplycost) + FROM tpch.main.partsupp, tpch.main.supplier, tpch.main.nation, tpch.main.region + WHERE + p_partkey = ps_partkey + AND s_suppkey = ps_suppkey + AND s_nationkey = n_nationkey + AND n_regionkey = r_regionkey + AND r_name = 'EUROPE' + ) +ORDER BY s_acctbal DESC, n_name, s_name, p_partkey +LIMIT 100 +"#, +}; + +/// TPC-H Q4: Order Priority Checking +/// Pattern: EXISTS subquery +const TPCH_Q4: TpchQuery = TpchQuery { + name: "Q4", + sql: r#" +SELECT + o_orderpriority, + COUNT(*) AS order_count +FROM + tpch.main.orders +WHERE + o_orderdate >= DATE '1993-07-01' + AND o_orderdate < DATE '1993-10-01' + AND EXISTS ( + SELECT * FROM tpch.main.lineitem + WHERE l_orderkey = o_orderkey AND l_commitdate < l_receiptdate + ) +GROUP BY o_orderpriority +ORDER BY o_orderpriority +"#, +}; + +/// TPC-H Q6: Forecasting Revenue Change +/// Pattern: Scan + filter (lineitem with predicates) +const TPCH_Q6: TpchQuery = TpchQuery { + name: "Q6", + sql: r#" +SELECT + SUM(l_extendedprice * l_discount) AS revenue +FROM + tpch.main.lineitem +WHERE + l_shipdate >= DATE '1994-01-01' + AND l_shipdate < DATE '1995-01-01' + AND l_discount BETWEEN 0.05 AND 0.07 + AND l_quantity < 24 +"#, +}; + +/// TPC-H Q7: Volume Shipping +/// Pattern: 6-way join + CASE +const TPCH_Q7: TpchQuery = TpchQuery { + name: "Q7", + sql: r#" +SELECT + supp_nation, cust_nation, l_year, SUM(volume) AS revenue +FROM ( + SELECT + n1.n_name AS supp_nation, + n2.n_name AS cust_nation, + EXTRACT(YEAR FROM l_shipdate) AS l_year, + l_extendedprice * (1 - l_discount) AS volume + FROM + tpch.main.supplier, + tpch.main.lineitem, + tpch.main.orders, + tpch.main.customer, + tpch.main.nation n1, + tpch.main.nation n2 + WHERE + s_suppkey = l_suppkey + AND o_orderkey = l_orderkey + AND c_custkey = o_custkey + AND s_nationkey = n1.n_nationkey + AND c_nationkey = n2.n_nationkey + AND ((n1.n_name = 'FRANCE' AND n2.n_name = 'GERMANY') + OR (n1.n_name = 'GERMANY' AND n2.n_name = 'FRANCE')) + AND l_shipdate BETWEEN DATE '1995-01-01' AND DATE '1996-12-31' +) AS shipping +GROUP BY supp_nation, cust_nation, l_year +ORDER BY supp_nation, cust_nation, l_year +"#, +}; + +/// TPC-H Q8: National Market Share +/// Pattern: 8-way join +const TPCH_Q8: TpchQuery = TpchQuery { + name: "Q8", + sql: r#" +SELECT + o_year, + SUM(CASE WHEN nation = 'BRAZIL' THEN volume ELSE 0 END) / SUM(volume) AS mkt_share +FROM ( + SELECT + EXTRACT(YEAR FROM o_orderdate) AS o_year, + l_extendedprice * (1 - l_discount) AS volume, + n2.n_name AS nation + FROM + tpch.main.part, + tpch.main.supplier, + tpch.main.lineitem, + tpch.main.orders, + tpch.main.customer, + tpch.main.nation n1, + tpch.main.nation n2, + tpch.main.region + WHERE + p_partkey = l_partkey + AND s_suppkey = l_suppkey + AND l_orderkey = o_orderkey + AND o_custkey = c_custkey + AND c_nationkey = n1.n_nationkey + AND n1.n_regionkey = r_regionkey + AND r_name = 'AMERICA' + AND s_nationkey = n2.n_nationkey + AND o_orderdate BETWEEN DATE '1995-01-01' AND DATE '1996-12-31' + AND p_type = 'ECONOMY ANODIZED STEEL' +) AS all_nations +GROUP BY o_year +ORDER BY o_year +"#, +}; + +/// TPC-H Q9: Product Type Profit Measure +/// Pattern: 6-way join + LIKE +const TPCH_Q9: TpchQuery = TpchQuery { + name: "Q9", + sql: r#" +SELECT + nation, o_year, SUM(amount) AS sum_profit +FROM ( + SELECT + n_name AS nation, + EXTRACT(YEAR FROM o_orderdate) AS o_year, + l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity AS amount + FROM + tpch.main.part, + tpch.main.supplier, + tpch.main.lineitem, + tpch.main.partsupp, + tpch.main.orders, + tpch.main.nation + WHERE + s_suppkey = l_suppkey + AND ps_suppkey = l_suppkey + AND ps_partkey = l_partkey + AND p_partkey = l_partkey + AND o_orderkey = l_orderkey + AND s_nationkey = n_nationkey + AND p_name LIKE '%green%' +) AS profit +GROUP BY nation, o_year +ORDER BY nation, o_year DESC +"#, +}; + +/// TPC-H Q10: Returned Item Reporting +/// Pattern: 4-way join +const TPCH_Q10: TpchQuery = TpchQuery { + name: "Q10", + sql: r#" +SELECT + c_custkey, c_name, + SUM(l_extendedprice * (1 - l_discount)) AS revenue, + c_acctbal, n_name, c_address, c_phone, c_comment +FROM + tpch.main.customer, + tpch.main.orders, + tpch.main.lineitem, + tpch.main.nation +WHERE + c_custkey = o_custkey + AND l_orderkey = o_orderkey + AND o_orderdate >= DATE '1993-10-01' + AND o_orderdate < DATE '1994-01-01' + AND l_returnflag = 'R' + AND c_nationkey = n_nationkey +GROUP BY c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment +ORDER BY revenue DESC +LIMIT 20 +"#, +}; + +/// TPC-H Q11: Important Stock Identification +/// Pattern: Nested subquery +const TPCH_Q11: TpchQuery = TpchQuery { + name: "Q11", + sql: r#" +SELECT + ps_partkey, SUM(ps_supplycost * ps_availqty) AS value +FROM + tpch.main.partsupp, + tpch.main.supplier, + tpch.main.nation +WHERE + ps_suppkey = s_suppkey + AND s_nationkey = n_nationkey + AND n_name = 'GERMANY' +GROUP BY ps_partkey +HAVING SUM(ps_supplycost * ps_availqty) > ( + SELECT SUM(ps_supplycost * ps_availqty) * 0.0001 + FROM tpch.main.partsupp, tpch.main.supplier, tpch.main.nation + WHERE ps_suppkey = s_suppkey AND s_nationkey = n_nationkey AND n_name = 'GERMANY' +) +ORDER BY value DESC +"#, +}; + +/// TPC-H Q12: Shipping Modes and Order Priority +/// Pattern: 3-way join + CASE +const TPCH_Q12: TpchQuery = TpchQuery { + name: "Q12", + sql: r#" +SELECT + l_shipmode, + SUM(CASE WHEN o_orderpriority = '1-URGENT' OR o_orderpriority = '2-HIGH' THEN 1 ELSE 0 END) AS high_line_count, + SUM(CASE WHEN o_orderpriority <> '1-URGENT' AND o_orderpriority <> '2-HIGH' THEN 1 ELSE 0 END) AS low_line_count +FROM + tpch.main.orders, + tpch.main.lineitem +WHERE + o_orderkey = l_orderkey + AND l_shipmode IN ('MAIL', 'SHIP') + AND l_commitdate < l_receiptdate + AND l_shipdate < l_commitdate + AND l_receiptdate >= DATE '1994-01-01' + AND l_receiptdate < DATE '1995-01-01' +GROUP BY l_shipmode +ORDER BY l_shipmode +"#, +}; + +/// TPC-H Q13: Customer Distribution +/// Pattern: LEFT OUTER join +const TPCH_Q13: TpchQuery = TpchQuery { + name: "Q13", + sql: r#" +SELECT + c_count, COUNT(*) AS custdist +FROM ( + SELECT c_custkey, COUNT(o_orderkey) AS c_count + FROM tpch.main.customer + LEFT OUTER JOIN tpch.main.orders ON c_custkey = o_custkey AND o_comment NOT LIKE '%special%requests%' + GROUP BY c_custkey +) AS c_orders +GROUP BY c_count +ORDER BY custdist DESC, c_count DESC +"#, +}; + +/// TPC-H Q14: Promotion Effect +/// Pattern: 2-way join +const TPCH_Q14: TpchQuery = TpchQuery { + name: "Q14", + sql: r#" +SELECT + 100.00 * SUM(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1 - l_discount) ELSE 0 END) / SUM(l_extendedprice * (1 - l_discount)) AS promo_revenue +FROM + tpch.main.lineitem, + tpch.main.part +WHERE + l_partkey = p_partkey + AND l_shipdate >= DATE '1995-09-01' + AND l_shipdate < DATE '1995-10-01' +"#, +}; + +/// TPC-H Q15: Top Supplier +/// Pattern: View + MAX subquery (using CTE instead of view) +const TPCH_Q15: TpchQuery = TpchQuery { + name: "Q15", + sql: r#" +WITH revenue AS ( + SELECT + l_suppkey AS supplier_no, + SUM(l_extendedprice * (1 - l_discount)) AS total_revenue + FROM tpch.main.lineitem + WHERE l_shipdate >= DATE '1996-01-01' AND l_shipdate < DATE '1996-04-01' + GROUP BY l_suppkey +) +SELECT s_suppkey, s_name, s_address, s_phone, total_revenue +FROM tpch.main.supplier, revenue +WHERE s_suppkey = supplier_no + AND total_revenue = (SELECT MAX(total_revenue) FROM revenue) +ORDER BY s_suppkey +"#, +}; + +/// TPC-H Q16: Parts/Supplier Relationship +/// Pattern: NOT IN subquery +const TPCH_Q16: TpchQuery = TpchQuery { + name: "Q16", + sql: r#" +SELECT + p_brand, p_type, p_size, COUNT(DISTINCT ps_suppkey) AS supplier_cnt +FROM + tpch.main.partsupp, + tpch.main.part +WHERE + p_partkey = ps_partkey + AND p_brand <> 'Brand#45' + AND p_type NOT LIKE 'MEDIUM POLISHED%' + AND p_size IN (49, 14, 23, 45, 19, 3, 36, 9) + AND ps_suppkey NOT IN ( + SELECT s_suppkey FROM tpch.main.supplier WHERE s_comment LIKE '%Customer%Complaints%' + ) +GROUP BY p_brand, p_type, p_size +ORDER BY supplier_cnt DESC, p_brand, p_type, p_size +"#, +}; + +/// TPC-H Q17: Small-Quantity-Order Revenue +/// Pattern: AVG subquery +const TPCH_Q17: TpchQuery = TpchQuery { + name: "Q17", + sql: r#" +SELECT + SUM(l_extendedprice) / 7.0 AS avg_yearly +FROM + tpch.main.lineitem, + tpch.main.part +WHERE + p_partkey = l_partkey + AND p_brand = 'Brand#23' + AND p_container = 'MED BOX' + AND l_quantity < ( + SELECT 0.2 * AVG(l_quantity) + FROM tpch.main.lineitem + WHERE l_partkey = p_partkey + ) +"#, +}; + +/// TPC-H Q18: Large Volume Customer +/// Pattern: IN subquery + HAVING +const TPCH_Q18: TpchQuery = TpchQuery { + name: "Q18", + sql: r#" +SELECT + c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, SUM(l_quantity) AS total_qty +FROM + tpch.main.customer, + tpch.main.orders, + tpch.main.lineitem +WHERE + o_orderkey IN ( + SELECT l_orderkey FROM tpch.main.lineitem GROUP BY l_orderkey HAVING SUM(l_quantity) > 300 + ) + AND c_custkey = o_custkey + AND o_orderkey = l_orderkey +GROUP BY c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice +ORDER BY o_totalprice DESC, o_orderdate +LIMIT 100 +"#, +}; + +/// TPC-H Q19: Discounted Revenue +/// Pattern: OR predicates +const TPCH_Q19: TpchQuery = TpchQuery { + name: "Q19", + sql: r#" +SELECT + SUM(l_extendedprice * (1 - l_discount)) AS revenue +FROM + tpch.main.lineitem, + tpch.main.part +WHERE + ( + p_partkey = l_partkey + AND p_brand = 'Brand#12' + AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') + AND l_quantity >= 1 AND l_quantity <= 11 + AND p_size BETWEEN 1 AND 5 + AND l_shipmode IN ('AIR', 'AIR REG') + AND l_shipinstruct = 'DELIVER IN PERSON' + ) + OR ( + p_partkey = l_partkey + AND p_brand = 'Brand#23' + AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') + AND l_quantity >= 10 AND l_quantity <= 20 + AND p_size BETWEEN 1 AND 10 + AND l_shipmode IN ('AIR', 'AIR REG') + AND l_shipinstruct = 'DELIVER IN PERSON' + ) + OR ( + p_partkey = l_partkey + AND p_brand = 'Brand#34' + AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') + AND l_quantity >= 20 AND l_quantity <= 30 + AND p_size BETWEEN 1 AND 15 + AND l_shipmode IN ('AIR', 'AIR REG') + AND l_shipinstruct = 'DELIVER IN PERSON' + ) +"#, +}; + +/// TPC-H Q20: Potential Part Promotion +/// Pattern: IN + EXISTS subquery +const TPCH_Q20: TpchQuery = TpchQuery { + name: "Q20", + sql: r#" +SELECT + s_name, s_address +FROM + tpch.main.supplier, + tpch.main.nation +WHERE + s_suppkey IN ( + SELECT ps_suppkey + FROM tpch.main.partsupp + WHERE ps_partkey IN (SELECT p_partkey FROM tpch.main.part WHERE p_name LIKE 'forest%') + AND ps_availqty > ( + SELECT 0.5 * SUM(l_quantity) + FROM tpch.main.lineitem + WHERE l_partkey = ps_partkey AND l_suppkey = ps_suppkey + AND l_shipdate >= DATE '1994-01-01' AND l_shipdate < DATE '1995-01-01' + ) + ) + AND s_nationkey = n_nationkey + AND n_name = 'CANADA' +ORDER BY s_name +"#, +}; + +/// TPC-H Q21: Suppliers Who Kept Orders Waiting +/// Pattern: EXISTS + NOT EXISTS +const TPCH_Q21: TpchQuery = TpchQuery { + name: "Q21", + sql: r#" +SELECT + s_name, COUNT(*) AS numwait +FROM + tpch.main.supplier, + tpch.main.lineitem l1, + tpch.main.orders, + tpch.main.nation +WHERE + s_suppkey = l1.l_suppkey + AND o_orderkey = l1.l_orderkey + AND o_orderstatus = 'F' + AND l1.l_receiptdate > l1.l_commitdate + AND EXISTS ( + SELECT * FROM tpch.main.lineitem l2 + WHERE l2.l_orderkey = l1.l_orderkey AND l2.l_suppkey <> l1.l_suppkey + ) + AND NOT EXISTS ( + SELECT * FROM tpch.main.lineitem l3 + WHERE l3.l_orderkey = l1.l_orderkey AND l3.l_suppkey <> l1.l_suppkey AND l3.l_receiptdate > l3.l_commitdate + ) + AND s_nationkey = n_nationkey + AND n_name = 'SAUDI ARABIA' +GROUP BY s_name +ORDER BY numwait DESC, s_name +LIMIT 100 +"#, +}; + +/// TPC-H Q22: Global Sales Opportunity +/// Pattern: NOT EXISTS + SUBSTRING +const TPCH_Q22: TpchQuery = TpchQuery { + name: "Q22", + sql: r#" +SELECT + cntrycode, COUNT(*) AS numcust, SUM(c_acctbal) AS totacctbal +FROM ( + SELECT + SUBSTRING(c_phone FROM 1 FOR 2) AS cntrycode, + c_acctbal + FROM tpch.main.customer + WHERE + SUBSTRING(c_phone FROM 1 FOR 2) IN ('13', '31', '23', '29', '30', '18', '17') + AND c_acctbal > ( + SELECT AVG(c_acctbal) FROM tpch.main.customer + WHERE c_acctbal > 0.00 AND SUBSTRING(c_phone FROM 1 FOR 2) IN ('13', '31', '23', '29', '30', '18', '17') + ) + AND NOT EXISTS ( + SELECT * FROM tpch.main.orders WHERE o_custkey = c_custkey + ) +) AS custsale +GROUP BY cntrycode +ORDER BY cntrycode +"#, +}; + +/// Bloom Filter Test: Point lookup on lineitem by order key +/// Pattern: Equality predicate on high-cardinality column (should benefit from bloom filter) +const BLOOM_Q1: TpchQuery = TpchQuery { + name: "BF1", + sql: r#" +SELECT + l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice +FROM + tpch.main.lineitem +WHERE + l_orderkey = 12345 +"#, +}; + +/// Bloom Filter Test: Point lookup on orders by order key +/// Pattern: Equality predicate on primary key +const BLOOM_Q2: TpchQuery = TpchQuery { + name: "BF2", + sql: r#" +SELECT + o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate +FROM + tpch.main.orders +WHERE + o_orderkey = 54321 +"#, +}; + +/// Bloom Filter Test: Point lookup on customer by customer key +/// Pattern: Equality predicate on primary key +const BLOOM_Q3: TpchQuery = TpchQuery { + name: "BF3", + sql: r#" +SELECT + c_custkey, c_name, c_address, c_nationkey, c_acctbal +FROM + tpch.main.customer +WHERE + c_custkey = 7500 +"#, +}; + +/// Bloom Filter Test: Multiple equality predicates +/// Pattern: Multiple equality conditions that can use bloom filters +const BLOOM_Q4: TpchQuery = TpchQuery { + name: "BF4", + sql: r#" +SELECT + l_orderkey, l_linenumber, l_quantity, l_extendedprice +FROM + tpch.main.lineitem +WHERE + l_orderkey = 99999 + AND l_linenumber = 3 +"#, +}; + +/// Distinct Index Test: Equality on low-cardinality column (l_returnflag has 3 values: A, F, R) +const INDEX_Q1: TpchQuery = TpchQuery { + name: "IX1", + sql: r#" +SELECT * +FROM tpch.main.lineitem +WHERE l_returnflag = 'R' +"#, +}; + +/// Distinct Index Test: Equality on order status (3 values: F, O, P) +const INDEX_Q2: TpchQuery = TpchQuery { + name: "IX2", + sql: r#" +SELECT * +FROM tpch.main.orders +WHERE o_orderstatus = 'F' +"#, +}; + +/// Distinct Index Test: Equality on nation name (25 values) +const INDEX_Q3: TpchQuery = TpchQuery { + name: "IX3", + sql: r#" +SELECT * +FROM tpch.main.nation +WHERE n_name = 'GERMANY' +"#, +}; + +/// Distinct Index Test: IN predicate on ship mode (7 values) +const INDEX_Q4: TpchQuery = TpchQuery { + name: "IX4", + sql: r#" +SELECT * +FROM tpch.main.lineitem +WHERE l_shipmode IN ('AIR', 'MAIL') +"#, +}; + +/// All TPC-H queries to run +const TPCH_QUERIES: &[TpchQuery] = &[ + TPCH_Q1, TPCH_Q2, TPCH_Q3, TPCH_Q4, TPCH_Q5, TPCH_Q6, TPCH_Q7, TPCH_Q8, TPCH_Q9, TPCH_Q10, + TPCH_Q11, TPCH_Q12, TPCH_Q13, TPCH_Q14, TPCH_Q15, TPCH_Q16, TPCH_Q17, TPCH_Q18, TPCH_Q19, + TPCH_Q20, TPCH_Q21, TPCH_Q22, +]; + +/// Bloom filter specific queries (equality predicates - bloom filters should help) +const BLOOM_QUERIES: &[TpchQuery] = &[BLOOM_Q1, BLOOM_Q2, BLOOM_Q3, BLOOM_Q4]; + +/// Distinct index specific queries (equality/IN on low-cardinality columns) +const INDEX_QUERIES: &[TpchQuery] = &[INDEX_Q1, INDEX_Q2, INDEX_Q3, INDEX_Q4]; + +/// Result of a single query benchmark +struct QueryBenchmarkResult { + name: &'static str, + duration: Duration, + row_count: usize, + success: bool, + error: Option, +} + +/// Test harness for TPC-H benchmarks +struct TpchBenchmarkHarness { + engine: Arc, + router: Router, + #[allow(dead_code)] + temp_dir: TempDir, + tpch_db_path: String, + scale_factor: f64, +} + +impl TpchBenchmarkHarness { + /// Create a new harness with TPC-H data at the specified scale factor + async fn new(scale_factor: f64) -> Result { + Self::new_with_index_presets(scale_factor, false).await + } + + /// Create a new harness with TPC-H optimized index presets enabled + async fn new_with_index_presets(scale_factor: f64, enable_presets: bool) -> Result { + let temp_dir = tempfile::tempdir()?; + + let mut builder = RuntimeEngine::builder() + .base_dir(temp_dir.path()) + .secret_key(generate_test_secret_key()); + + if enable_presets { + builder = builder.with_tpch_index_presets(); + } + + let engine = builder.build().await?; + + let app = AppServer::new(engine); + + // Create TPC-H database + let tpch_db_path = Self::create_tpch_duckdb(&temp_dir, scale_factor)?; + + Ok(Self { + engine: app.engine, + router: app.router, + temp_dir, + tpch_db_path, + scale_factor, + }) + } + + /// Create a new harness with a custom parquet config. + async fn new_with_parquet_config(scale_factor: f64, config: ParquetConfig) -> Result { + let temp_dir = tempfile::tempdir()?; + + let engine = RuntimeEngine::builder() + .base_dir(temp_dir.path()) + .secret_key(generate_test_secret_key()) + .parquet_config(config) + .build() + .await?; + + let app = AppServer::new(engine); + + let tpch_db_path = Self::create_tpch_duckdb(&temp_dir, scale_factor)?; + + Ok(Self { + engine: app.engine, + router: app.router, + temp_dir, + tpch_db_path, + scale_factor, + }) + } + + /// Create a DuckDB with TPC-H data using the built-in dbgen + fn create_tpch_duckdb(temp_dir: &TempDir, scale_factor: f64) -> Result { + let db_path = temp_dir.path().join("tpch.duckdb"); + let conn = duckdb::Connection::open(&db_path)?; + + // Install and load TPC-H extension + conn.execute("INSTALL tpch", [])?; + conn.execute("LOAD tpch", [])?; + + // Generate TPC-H data at specified scale factor + conn.execute(&format!("CALL dbgen(sf = {})", scale_factor), [])?; + + Ok(db_path.to_str().unwrap().to_string()) + } + + /// Create a connection to the TPC-H database via API + async fn create_connection(&self) -> Result { + let response = self + .router + .clone() + .oneshot( + Request::builder() + .method("POST") + .uri(PATH_CONNECTIONS) + .header("content-type", "application/json") + .body(Body::from(serde_json::to_string(&json!({ + "name": "tpch", + "source_type": "duckdb", + "config": { + "path": self.tpch_db_path + } + }))?))?, + ) + .await?; + + assert_eq!(response.status(), StatusCode::CREATED); + + let body = axum::body::to_bytes(response.into_body(), usize::MAX).await?; + let json: serde_json::Value = serde_json::from_slice(&body)?; + + Ok(json["id"].as_str().unwrap().to_string()) + } + + /// Get total size of all parquet files in the cache directory + fn get_parquet_cache_size(base_dir: &std::path::Path) -> u64 { + let mut total = 0u64; + fn walk_dir(dir: &std::path::Path, total: &mut u64) { + if let Ok(entries) = std::fs::read_dir(dir) { + for entry in entries.flatten() { + let path = entry.path(); + if path.is_dir() { + walk_dir(&path, total); + } else if path.extension().map_or(false, |ext| ext == "parquet") { + if let Ok(meta) = std::fs::metadata(&path) { + *total += meta.len(); + } + } + } + } + } + walk_dir(base_dir, &mut total); + total + } + + /// Run a single benchmark query + async fn run_query(&self, query: &TpchQuery) -> QueryBenchmarkResult { + let start = Instant::now(); + + match self.engine.execute_query(query.sql).await { + Ok(response) => { + let duration = start.elapsed(); + let row_count: usize = response.results.iter().map(|b| b.num_rows()).sum(); + + QueryBenchmarkResult { + name: query.name, + duration, + row_count, + success: true, + error: None, + } + } + Err(e) => QueryBenchmarkResult { + name: query.name, + duration: start.elapsed(), + row_count: 0, + success: false, + error: Some(e.to_string()), + }, + } + } + + /// Run all benchmark queries and print results + async fn run_benchmark(&self) -> Vec { + println!(); + println!("========================================"); + println!("TPC-H Benchmark Results (SF={})", self.scale_factor); + println!("========================================"); + println!(); + + let mut results = Vec::new(); + + // Run each query + println!("Query Results (includes sync time on first run):"); + println!( + "{:<15} {:>15} {:>10} {}", + "Query", "Time (ms)", "Rows", "Status" + ); + println!("{}", "-".repeat(50)); + + for query in TPCH_QUERIES { + let result = self.run_query(query).await; + + let status = if result.success { + "OK".to_string() + } else { + format!("FAIL: {}", result.error.as_deref().unwrap_or("unknown")) + }; + + println!( + "{:<15} {:>15.2} {:>10} {}", + result.name, + result.duration.as_secs_f64() * 1000.0, + result.row_count, + status + ); + + results.push(result); + } + + println!("{}", "-".repeat(50)); + + // Calculate totals + let total_time: Duration = results.iter().map(|r| r.duration).sum(); + let successful_count = results.iter().filter(|r| r.success).count(); + + println!( + "Total Query Time: {:.3}s ({}/{} queries succeeded)", + total_time.as_secs_f64(), + successful_count, + results.len() + ); + println!(); + + results + } + + /// Run benchmark with warmup - first run syncs data, second run measures cached performance + async fn run_benchmark_with_warmup( + &self, + ) -> (Vec, Vec) { + println!(); + println!("========================================"); + println!("TPC-H Benchmark Results (SF={})", self.scale_factor); + println!("========================================"); + + // First pass: warmup (triggers sync from DuckDB to Parquet cache) + println!(); + println!("--- Warmup Pass (includes sync time) ---"); + println!( + "{:<15} {:>15} {:>10} {}", + "Query", "Time (ms)", "Rows", "Status" + ); + println!("{}", "-".repeat(50)); + + let mut warmup_results = Vec::new(); + for query in TPCH_QUERIES { + let result = self.run_query(query).await; + let status = if result.success { "OK" } else { "FAIL" }; + println!( + "{:<15} {:>15.2} {:>10} {}", + result.name, + result.duration.as_secs_f64() * 1000.0, + result.row_count, + status + ); + warmup_results.push(result); + } + + let warmup_total: Duration = warmup_results.iter().map(|r| r.duration).sum(); + println!("{}", "-".repeat(50)); + println!("Warmup Total: {:.3}s", warmup_total.as_secs_f64()); + + // Print cache size + let cache_size = Self::get_parquet_cache_size(self.temp_dir.path()); + println!( + "Parquet Cache Size: {:.2} MB", + cache_size as f64 / 1024.0 / 1024.0 + ); + + // Second pass: cached performance (reads from Parquet cache) + println!(); + println!("--- Cached Pass (reads from Parquet) ---"); + println!( + "{:<15} {:>15} {:>10} {}", + "Query", "Time (ms)", "Rows", "Status" + ); + println!("{}", "-".repeat(50)); + + let mut cached_results = Vec::new(); + for query in TPCH_QUERIES { + let result = self.run_query(query).await; + let status = if result.success { "OK" } else { "FAIL" }; + println!( + "{:<15} {:>15.2} {:>10} {}", + result.name, + result.duration.as_secs_f64() * 1000.0, + result.row_count, + status + ); + cached_results.push(result); + } + + let cached_total: Duration = cached_results.iter().map(|r| r.duration).sum(); + let successful_count = cached_results.iter().filter(|r| r.success).count(); + println!("{}", "-".repeat(50)); + println!( + "Cached Total: {:.3}s ({}/{} queries succeeded)", + cached_total.as_secs_f64(), + successful_count, + cached_results.len() + ); + println!(); + + (warmup_results, cached_results) + } + + /// Run bloom filter benchmark - tests equality predicates where bloom filters help + async fn run_bloom_filter_benchmark(&self) -> Vec { + println!(); + println!("========================================"); + println!("Bloom Filter Benchmark (SF={})", self.scale_factor); + println!("========================================"); + println!(); + + // First, warmup TPC-H queries to sync the main tables + println!("--- Syncing tables (warmup) ---"); + for query in TPCH_QUERIES { + let _ = self.run_query(query).await; + } + println!("Tables synced."); + + // Now run bloom filter specific queries (equality predicates) + println!(); + println!("--- Bloom Filter Queries (equality predicates) ---"); + println!( + "{:<15} {:>15} {:>10} {}", + "Query", "Time (ms)", "Rows", "Status" + ); + println!("{}", "-".repeat(50)); + + let mut results = Vec::new(); + for query in BLOOM_QUERIES { + let result = self.run_query(query).await; + let status = if result.success { "OK" } else { "FAIL" }; + println!( + "{:<15} {:>15.2} {:>10} {}", + result.name, + result.duration.as_secs_f64() * 1000.0, + result.row_count, + status + ); + results.push(result); + } + + let total: Duration = results.iter().map(|r| r.duration).sum(); + let successful_count = results.iter().filter(|r| r.success).count(); + println!("{}", "-".repeat(50)); + println!( + "Total: {:.3}s ({}/{} queries succeeded)", + total.as_secs_f64(), + successful_count, + results.len() + ); + println!(); + + results + } + + /// Run distinct index benchmark - tests equality/IN predicates on low-cardinality columns + async fn run_distinct_index_benchmark(&self) -> Vec { + println!(); + println!("========================================"); + println!("Distinct Index Benchmark (SF={})", self.scale_factor); + println!("========================================"); + println!(); + + // First, warmup TPC-H queries to sync the main tables + println!("--- Syncing tables (warmup) ---"); + for query in TPCH_QUERIES { + let _ = self.run_query(query).await; + } + println!("Tables synced."); + + // Now run distinct index specific queries + println!(); + println!("--- Distinct Index Queries (equality/IN on low-cardinality columns) ---"); + println!( + "{:<15} {:>15} {:>10} {}", + "Query", "Time (ms)", "Rows", "Status" + ); + println!("{}", "-".repeat(50)); + + let mut results = Vec::new(); + for query in INDEX_QUERIES { + let result = self.run_query(query).await; + let status = if result.success { "OK" } else { "FAIL" }; + println!( + "{:<15} {:>15.2} {:>10} {}", + result.name, + result.duration.as_secs_f64() * 1000.0, + result.row_count, + status + ); + results.push(result); + } + + let total: Duration = results.iter().map(|r| r.duration).sum(); + let successful_count = results.iter().filter(|r| r.success).count(); + println!("{}", "-".repeat(50)); + println!( + "Total: {:.3}s ({}/{} queries succeeded)", + total.as_secs_f64(), + successful_count, + results.len() + ); + println!(); + + results + } +} + +/// Get scale factor from environment variable or use default +fn get_scale_factor(default: f64) -> f64 { + std::env::var("TPCH_SCALE_FACTOR") + .ok() + .and_then(|s| s.parse().ok()) + .unwrap_or(default) +} + +/// Quick TPC-H benchmark test (default SF=0.01 for CI) +#[tokio::test(flavor = "multi_thread")] +async fn test_tpch_benchmark() -> Result<()> { + let scale_factor = get_scale_factor(0.01); + + println!("Setting up TPC-H benchmark (SF={})...", scale_factor); + let setup_start = Instant::now(); + + let harness = TpchBenchmarkHarness::new(scale_factor).await?; + + let data_gen_time = setup_start.elapsed(); + println!(" Data Generation: {:.3}s", data_gen_time.as_secs_f64()); + + let conn_start = Instant::now(); + let _connection_id = harness.create_connection().await?; + let conn_time = conn_start.elapsed(); + println!(" Connection Setup: {:.3}s", conn_time.as_secs_f64()); + + // Run benchmark with warmup to separate sync time from cached query time + let (_warmup_results, cached_results) = harness.run_benchmark_with_warmup().await; + + // Verify all queries succeeded + let failed: Vec<_> = cached_results.iter().filter(|r| !r.success).collect(); + assert!( + failed.is_empty(), + "Some queries failed: {:?}", + failed + .iter() + .map(|r| format!("{}: {}", r.name, r.error.as_deref().unwrap_or("unknown"))) + .collect::>() + ); + + Ok(()) +} + +/// TPC-H benchmark with index presets enabled +/// Run with: TPCH_SCALE_FACTOR=1.0 cargo test tpch_benchmark_presets --release -- --nocapture --test-threads=1 +#[tokio::test(flavor = "multi_thread")] +async fn test_tpch_benchmark_presets() -> Result<()> { + let scale_factor = get_scale_factor(0.01); + + println!( + "Setting up TPC-H benchmark WITH INDEX PRESETS (SF={})...", + scale_factor + ); + let setup_start = Instant::now(); + + // Enable TPC-H optimized index presets + let harness = TpchBenchmarkHarness::new_with_index_presets(scale_factor, true).await?; + + let data_gen_time = setup_start.elapsed(); + println!(" Data Generation: {:.3}s", data_gen_time.as_secs_f64()); + + let conn_start = Instant::now(); + let _connection_id = harness.create_connection().await?; + let conn_time = conn_start.elapsed(); + println!(" Connection Setup: {:.3}s", conn_time.as_secs_f64()); + + // Run benchmark with warmup to separate sync time from cached query time + let (_warmup_results, cached_results) = harness.run_benchmark_with_warmup().await; + + // Verify all queries succeeded + let failed: Vec<_> = cached_results.iter().filter(|r| !r.success).collect(); + assert!( + failed.is_empty(), + "Some queries failed: {:?}", + failed + .iter() + .map(|r| format!("{}: {}", r.name, r.error.as_deref().unwrap_or("unknown"))) + .collect::>() + ); + + Ok(()) +} + +/// Full TPC-H benchmark test with SF=1.0 (~1GB data) +/// Run with: cargo test tpch_benchmark_full --release -- --ignored --nocapture +#[tokio::test(flavor = "multi_thread")] +#[ignore] +async fn test_tpch_benchmark_full() -> Result<()> { + let scale_factor = 1.0; + + println!("Setting up TPC-H benchmark (SF={})...", scale_factor); + println!("Note: This may take a while to generate ~1GB of data..."); + let setup_start = Instant::now(); + + let harness = TpchBenchmarkHarness::new(scale_factor).await?; + + let data_gen_time = setup_start.elapsed(); + println!(" Data Generation: {:.3}s", data_gen_time.as_secs_f64()); + + let conn_start = Instant::now(); + let _connection_id = harness.create_connection().await?; + let conn_time = conn_start.elapsed(); + println!(" Connection Setup: {:.3}s", conn_time.as_secs_f64()); + + // Run benchmark + let results = harness.run_benchmark().await; + + // Verify all queries succeeded + let failed: Vec<_> = results.iter().filter(|r| !r.success).collect(); + assert!( + failed.is_empty(), + "Some queries failed: {:?}", + failed + .iter() + .map(|r| format!("{}: {}", r.name, r.error.as_deref().unwrap_or("unknown"))) + .collect::>() + ); + + Ok(()) +} + +/// Bloom filter benchmark - tests equality predicates where bloom filters should help +/// Run with: TPCH_SCALE_FACTOR=0.1 cargo test bloom_filter_benchmark --release -- --nocapture +#[tokio::test(flavor = "multi_thread")] +async fn test_bloom_filter_benchmark() -> Result<()> { + let scale_factor = get_scale_factor(0.1); + + println!("Setting up Bloom Filter benchmark (SF={})...", scale_factor); + let setup_start = Instant::now(); + + let harness = TpchBenchmarkHarness::new(scale_factor).await?; + + let data_gen_time = setup_start.elapsed(); + println!(" Data Generation: {:.3}s", data_gen_time.as_secs_f64()); + + let conn_start = Instant::now(); + let _connection_id = harness.create_connection().await?; + let conn_time = conn_start.elapsed(); + println!(" Connection Setup: {:.3}s", conn_time.as_secs_f64()); + + // Run bloom filter benchmark (equality predicates) + let results = harness.run_bloom_filter_benchmark().await; + + // Verify all queries succeeded + let failed: Vec<_> = results.iter().filter(|r| !r.success).collect(); + assert!( + failed.is_empty(), + "Some queries failed: {:?}", + failed + .iter() + .map(|r| format!("{}: {}", r.name, r.error.as_deref().unwrap_or("unknown"))) + .collect::>() + ); + + Ok(()) +} + +/// Distinct index benchmark - tests equality/IN predicates on low-cardinality columns +/// Run with: TPCH_SCALE_FACTOR=0.1 cargo test distinct_index_benchmark --release -- --nocapture +#[tokio::test(flavor = "multi_thread")] +async fn test_distinct_index_benchmark() -> Result<()> { + let scale_factor = get_scale_factor(0.1); + + println!( + "Setting up Distinct Index benchmark (SF={})...", + scale_factor + ); + let setup_start = Instant::now(); + + let harness = TpchBenchmarkHarness::new(scale_factor).await?; + + let data_gen_time = setup_start.elapsed(); + println!(" Data Generation: {:.3}s", data_gen_time.as_secs_f64()); + + let conn_start = Instant::now(); + let _connection_id = harness.create_connection().await?; + let conn_time = conn_start.elapsed(); + println!(" Connection Setup: {:.3}s", conn_time.as_secs_f64()); + + // Run distinct index benchmark + let results = harness.run_distinct_index_benchmark().await; + + // Verify all queries succeeded + let failed: Vec<_> = results.iter().filter(|r| !r.success).collect(); + assert!( + failed.is_empty(), + "Some queries failed: {:?}", + failed + .iter() + .map(|r| format!("{}: {}", r.name, r.error.as_deref().unwrap_or("unknown"))) + .collect::>() + ); + + Ok(()) +} + +/// Custom query test - compares ORDER BY performance with and without index +/// Run with: TPCH_SCALE_FACTOR=0.1 cargo test test_custom_query --release -- --nocapture +#[tokio::test(flavor = "multi_thread")] +async fn test_custom_query() -> Result<()> { + let scale_factor = get_scale_factor(0.1); + let query = "SELECT * FROM tpch.main.orders ORDER BY o_orderdate DESC LIMIT 5"; + let filter_query = "SELECT * FROM tpch.main.orders WHERE o_orderdate >= DATE '1998-07-01' ORDER BY o_orderdate LIMIT 5"; + let iterations = 5; + + let harness = TpchBenchmarkHarness::new(scale_factor).await?; + let _conn = harness.create_connection().await?; + + // Sync + let _ = harness + .engine + .execute_query("SELECT COUNT(*) FROM tpch.main.orders") + .await?; + + // Warmup + let _ = harness.engine.execute_query(query).await?; + let _ = harness.engine.execute_query(filter_query).await?; + + // === No index: timed runs === + println!("=== WITHOUT Index ==="); + let mut times = Vec::new(); + for _ in 0..iterations { + let start = Instant::now(); + let r = harness.engine.execute_query(query).await?; + let elapsed = start.elapsed(); + let rows: usize = r.results.iter().map(|b| b.num_rows()).sum(); + println!( + " ORDER BY DESC LIMIT 5: {:.2}ms, {} rows", + elapsed.as_secs_f64() * 1000.0, + rows + ); + times.push(elapsed); + } + times.sort(); + let median_no_idx = times[times.len() / 2]; + + let mut times = Vec::new(); + for _ in 0..iterations { + let start = Instant::now(); + let r = harness.engine.execute_query(filter_query).await?; + let elapsed = start.elapsed(); + let rows: usize = r.results.iter().map(|b| b.num_rows()).sum(); + println!( + " Filter+ORDER BY: {:.2}ms, {} rows", + elapsed.as_secs_f64() * 1000.0, + rows + ); + times.push(elapsed); + } + times.sort(); + let median_filter_no_idx = times[times.len() / 2]; + + // === Create index === + println!("\nCreating index on o_orderdate..."); + let idx_start = Instant::now(); + harness + .engine + .execute_query("CREATE INDEX idx_orderdate ON tpch.main.orders (o_orderdate)") + .await?; + println!( + " Index created in {:.2}ms\n", + idx_start.elapsed().as_secs_f64() * 1000.0 + ); + + // Warmup after index creation (stabilize system) + for _ in 0..3 { + let _ = harness.engine.execute_query(query).await?; + let _ = harness.engine.execute_query(filter_query).await?; + } + + // === With index: timed runs === + println!("=== WITH Index ==="); + let mut times = Vec::new(); + for _ in 0..iterations { + let start = Instant::now(); + let r = harness.engine.execute_query(query).await?; + let elapsed = start.elapsed(); + let rows: usize = r.results.iter().map(|b| b.num_rows()).sum(); + println!( + " ORDER BY DESC LIMIT 5: {:.2}ms, {} rows", + elapsed.as_secs_f64() * 1000.0, + rows + ); + times.push(elapsed); + } + times.sort(); + let median_idx = times[times.len() / 2]; + + let mut times = Vec::new(); + for _ in 0..iterations { + let start = Instant::now(); + let r = harness.engine.execute_query(filter_query).await?; + let elapsed = start.elapsed(); + let rows: usize = r.results.iter().map(|b| b.num_rows()).sum(); + println!( + " Filter+ORDER BY: {:.2}ms, {} rows", + elapsed.as_secs_f64() * 1000.0, + rows + ); + times.push(elapsed); + } + times.sort(); + let median_filter_idx = times[times.len() / 2]; + + // === EXPLAIN === + println!("\n=== EXPLAIN (with index) ==="); + for q in [query, filter_query] { + println!("Query: {}", q); + if let Ok(r) = harness + .engine + .execute_query(&format!("EXPLAIN {}", q)) + .await + { + for batch in &r.results { + use datafusion::arrow::array::StringArray; + if let Some(col) = batch.column(1).as_any().downcast_ref::() { + for i in 0..batch.num_rows() { + println!(" {}", col.value(i)); + } + } + } + } + println!(); + } + + // === Summary === + println!("=== Results ==="); + println!( + "{:<25} {:>12} {:>12} {:>10}", + "Query", "No Index", "With Index", "Speedup" + ); + println!("{}", "-".repeat(62)); + println!( + "{:<25} {:>10.2}ms {:>10.2}ms {:>9.2}x", + "ORDER BY DESC LIMIT 5", + median_no_idx.as_secs_f64() * 1000.0, + median_idx.as_secs_f64() * 1000.0, + median_no_idx.as_secs_f64() / median_idx.as_secs_f64(), + ); + println!( + "{:<25} {:>10.2}ms {:>10.2}ms {:>9.2}x", + "Filter+ORDER BY", + median_filter_no_idx.as_secs_f64() * 1000.0, + median_filter_idx.as_secs_f64() * 1000.0, + median_filter_no_idx.as_secs_f64() / median_filter_idx.as_secs_f64(), + ); + + Ok(()) +} + +/// Test CREATE INDEX SQL command +/// Run with: cargo test test_create_index_sql --release -- --nocapture --test-threads=1 +#[tokio::test(flavor = "multi_thread")] +async fn test_create_index_sql() -> Result<()> { + let scale_factor = get_scale_factor(0.01); + + println!("=== CREATE INDEX SQL Test (SF={}) ===", scale_factor); + + let harness = TpchBenchmarkHarness::new(scale_factor).await?; + let _connection_id = harness.create_connection().await?; + + // Step 1: First sync a table (run a query to trigger caching) + println!("\nStep 1: Syncing orders table..."); + let sync_start = Instant::now(); + let result = harness + .engine + .execute_query("SELECT COUNT(*) FROM tpch.main.orders") + .await; + match &result { + Ok(r) => { + let row_count: usize = r.results.iter().map(|b| b.num_rows()).sum(); + println!( + " Sync complete: {:.2}ms, {} result rows", + sync_start.elapsed().as_secs_f64() * 1000.0, + row_count + ); + } + Err(e) => { + println!(" Sync failed: {}", e); + return Err(anyhow::anyhow!("Sync failed: {}", e)); + } + } + + // Step 2: Create an index on o_orderdate + println!("\nStep 2: Creating index on o_orderdate..."); + let create_start = Instant::now(); + let result = harness + .engine + .execute_query("CREATE INDEX idx_orderdate ON tpch.main.orders (o_orderdate)") + .await; + match &result { + Ok(r) => { + let row_count: usize = r.results.iter().map(|b| b.num_rows()).sum(); + println!( + " CREATE INDEX: {:.2}ms, {} result rows", + create_start.elapsed().as_secs_f64() * 1000.0, + row_count + ); + // Print the result + if let Some(batch) = r.results.first() { + use datafusion::arrow::array::StringArray; + if let Some(status_col) = batch.column(0).as_any().downcast_ref::() { + println!(" Status: {}", status_col.value(0)); + } + } + } + Err(e) => { + println!(" CREATE INDEX failed: {}", e); + return Err(anyhow::anyhow!("CREATE INDEX failed: {}", e)); + } + } + + // Step 3: List indexes using SHOW INDEXES + println!("\nStep 3: Listing indexes..."); + let result = harness + .engine + .execute_query("SHOW INDEXES ON tpch.main.orders") + .await; + match &result { + Ok(r) => { + println!( + " SHOW INDEXES returned {} rows:", + r.results.iter().map(|b| b.num_rows()).sum::() + ); + if let Some(batch) = r.results.first() { + use datafusion::arrow::array::StringArray; + for i in 0..batch.num_rows() { + let name = batch + .column(0) + .as_any() + .downcast_ref::() + .map(|a| a.value(i)) + .unwrap_or(""); + let cols = batch + .column(1) + .as_any() + .downcast_ref::() + .map(|a| a.value(i)) + .unwrap_or(""); + println!(" - {} (columns: {})", name, cols); + } + } + } + Err(e) => { + println!(" SHOW INDEXES failed: {}", e); + return Err(anyhow::anyhow!("SHOW INDEXES failed: {}", e)); + } + } + + // Step 4: Test query performance with the index + println!("\nStep 4: Testing query with date range filter..."); + let query = "SELECT * FROM tpch.main.orders WHERE o_orderdate >= DATE '1995-01-01' AND o_orderdate < DATE '1995-02-01' LIMIT 10"; + let query_start = Instant::now(); + let result = harness.engine.execute_query(query).await; + match &result { + Ok(r) => { + let row_count: usize = r.results.iter().map(|b| b.num_rows()).sum(); + println!( + " Query: {:.2}ms, {} rows", + query_start.elapsed().as_secs_f64() * 1000.0, + row_count + ); + } + Err(e) => println!(" Query failed: {}", e), + } + + // Step 5: Drop the index + println!("\nStep 5: Dropping index..."); + let result = harness + .engine + .execute_query("DROP INDEX idx_orderdate ON tpch.main.orders") + .await; + match &result { + Ok(r) => { + if let Some(batch) = r.results.first() { + use datafusion::arrow::array::StringArray; + if let Some(status_col) = batch.column(0).as_any().downcast_ref::() { + println!(" Status: {}", status_col.value(0)); + } + } + } + Err(e) => { + println!(" DROP INDEX failed: {}", e); + return Err(anyhow::anyhow!("DROP INDEX failed: {}", e)); + } + } + + // Step 6: Verify index was dropped + println!("\nStep 6: Verifying index was dropped..."); + let result = harness + .engine + .execute_query("SHOW INDEXES ON tpch.main.orders") + .await; + match &result { + Ok(r) => { + let row_count: usize = r.results.iter().map(|b| b.num_rows()).sum(); + println!(" Indexes remaining: {}", row_count); + assert_eq!(row_count, 0, "Index should have been dropped"); + } + Err(e) => { + println!(" SHOW INDEXES failed: {}", e); + return Err(anyhow::anyhow!("SHOW INDEXES failed: {}", e)); + } + } + + println!("\n=== All steps completed successfully! ==="); + Ok(()) +} + +/// Index performance benchmark — compares TPC-H Q1-Q22 with and without indexes. +/// +/// Creates 6 indexes matching the tpch_optimized index preset configuration: +/// 1. lineitem(l_shipdate) — range filters in Q1, Q3, Q6, Q7, Q14, Q15, Q20 +/// 2. orders(o_orderdate) — range filters in Q3, Q4, Q5, Q8, Q10 +/// 3. lineitem(l_partkey) — join key in Q2, Q9, Q14, Q17, Q19, Q20 +/// 4. orders(o_custkey) — join key in Q3, Q5, Q8, Q10, Q13, Q18, Q22 +/// 5. customer(c_nationkey) — join key in Q5, Q7, Q8, Q10 +/// 6. partsupp(ps_suppkey) — join key in Q2, Q9, Q11, Q16, Q20 +/// +/// Run with: TPCH_SCALE_FACTOR=1.0 cargo test test_index_performance --release -- --nocapture --test-threads=1 +#[tokio::test(flavor = "multi_thread")] +async fn test_index_performance() -> Result<()> { + let scale_factor = get_scale_factor(0.1); + + println!("=== Index Performance Benchmark (SF={}) ===", scale_factor); + let setup_start = Instant::now(); + + let harness = TpchBenchmarkHarness::new(scale_factor).await?; + + let data_gen_time = setup_start.elapsed(); + println!(" Data Generation: {:.3}s", data_gen_time.as_secs_f64()); + + let _connection_id = harness.create_connection().await?; + + // ── Warmup: sync all tables ────────────────────────────── + println!("\n--- Syncing tables (warmup) ---"); + for query in TPCH_QUERIES { + let _ = harness.run_query(query).await; + } + let cache_size = TpchBenchmarkHarness::get_parquet_cache_size(harness.temp_dir.path()); + println!( + " Parquet Cache Size: {:.2} MB", + cache_size as f64 / 1024.0 / 1024.0 + ); + + // ── Baseline: cached read WITHOUT indexes ──────────────── + println!("\n--- Baseline (no indexes) ---"); + println!( + "{:<15} {:>15} {:>10} {}", + "Query", "Time (ms)", "Rows", "Status" + ); + println!("{}", "-".repeat(55)); + + let mut baseline_results = Vec::new(); + for query in TPCH_QUERIES { + let result = harness.run_query(query).await; + let status = if result.success { "OK" } else { "FAIL" }; + println!( + "{:<15} {:>15.2} {:>10} {}", + result.name, + result.duration.as_secs_f64() * 1000.0, + result.row_count, + status + ); + baseline_results.push(result); + } + let baseline_total: Duration = baseline_results.iter().map(|r| r.duration).sum(); + println!("{}", "-".repeat(55)); + println!("Baseline Total: {:.3}s", baseline_total.as_secs_f64()); + + // ── Create 6 indexes (matches tpch_optimized index presets) ── + println!("\n--- Creating indexes ---"); + + let indexes = [ + ("idx_shipdate", "tpch.main.lineitem", "(l_shipdate)"), + ("idx_orderdate", "tpch.main.orders", "(o_orderdate)"), + ("idx_partkey", "tpch.main.lineitem", "(l_partkey)"), + ("idx_custkey", "tpch.main.orders", "(o_custkey)"), + ("idx_nationkey", "tpch.main.customer", "(c_nationkey)"), + ("idx_suppkey", "tpch.main.partsupp", "(ps_suppkey)"), + ]; + + for (name, table, cols) in &indexes { + let sql = format!("CREATE INDEX {} ON {} {}", name, table, cols); + let start = Instant::now(); + match harness.engine.execute_query(&sql).await { + Ok(_) => println!( + " {} on {} {}: {:.2}ms", + name, + table, + cols, + start.elapsed().as_secs_f64() * 1000.0 + ), + Err(e) => println!(" FAILED {}: {}", name, e), + } + } + + // Show total cache size with indexes + let cache_size_with_idx = TpchBenchmarkHarness::get_parquet_cache_size(harness.temp_dir.path()); + println!( + " Cache Size with indexes: {:.2} MB (+{:.2} MB)", + cache_size_with_idx as f64 / 1024.0 / 1024.0, + (cache_size_with_idx - cache_size) as f64 / 1024.0 / 1024.0 + ); + + // ── Indexed: cached read WITH indexes ──────────────────── + println!("\n--- With Indexes ---"); + println!( + "{:<15} {:>15} {:>10} {:>12} {}", + "Query", "Time (ms)", "Rows", "vs Baseline", "Status" + ); + println!("{}", "-".repeat(70)); + + let mut indexed_results = Vec::new(); + for (i, query) in TPCH_QUERIES.iter().enumerate() { + let result = harness.run_query(query).await; + let status = if result.success { "OK" } else { "FAIL" }; + + let baseline_ms = baseline_results[i].duration.as_secs_f64() * 1000.0; + let indexed_ms = result.duration.as_secs_f64() * 1000.0; + let delta = if baseline_ms > 0.0 { + let pct = ((indexed_ms - baseline_ms) / baseline_ms) * 100.0; + if pct < -1.0 { + format!("{:.0}%", pct) + } else if pct > 1.0 { + format!("+{:.0}%", pct) + } else { + "~same".to_string() + } + } else { + "N/A".to_string() + }; + + println!( + "{:<15} {:>15.2} {:>10} {:>12} {}", + result.name, indexed_ms, result.row_count, delta, status + ); + indexed_results.push(result); + } + + let indexed_total: Duration = indexed_results.iter().map(|r| r.duration).sum(); + let total_speedup = baseline_total.as_secs_f64() / indexed_total.as_secs_f64(); + println!("{}", "-".repeat(70)); + println!( + "Indexed Total: {:.3}s (baseline {:.3}s, {:.2}x)", + indexed_total.as_secs_f64(), + baseline_total.as_secs_f64(), + total_speedup, + ); + + // ── Summary ────────────────────────────────────────────── + println!("\n--- Per-Query Comparison ---"); + println!( + "{:<8} {:>12} {:>12} {:>10}", + "Query", "Baseline", "Indexed", "Change" + ); + println!("{}", "-".repeat(45)); + + for (i, query) in TPCH_QUERIES.iter().enumerate() { + let b = baseline_results[i].duration.as_secs_f64() * 1000.0; + let idx = indexed_results[i].duration.as_secs_f64() * 1000.0; + let diff = idx - b; + let sign = if diff < 0.0 { "" } else { "+" }; + println!( + "{:<8} {:>10.2}ms {:>10.2}ms {:>8}{:.1}ms", + query.name, b, idx, sign, diff + ); + } + println!("{}", "-".repeat(45)); + println!( + "{:<8} {:>10.0}ms {:>10.0}ms {:>8}{:.0}ms", + "TOTAL", + baseline_total.as_secs_f64() * 1000.0, + indexed_total.as_secs_f64() * 1000.0, + if indexed_total < baseline_total { + "" + } else { + "+" + }, + (indexed_total.as_secs_f64() - baseline_total.as_secs_f64()) * 1000.0, + ); + + // Verify correctness: row counts must match + for (i, _query) in TPCH_QUERIES.iter().enumerate() { + assert_eq!( + baseline_results[i].row_count, indexed_results[i].row_count, + "Row count mismatch for {}: baseline={} vs indexed={}", + baseline_results[i].name, baseline_results[i].row_count, indexed_results[i].row_count + ); + } + + Ok(()) +} + +/// Old config (1M row groups + bloom filters) vs Current config (100K row groups + 6 indexes). +/// +/// Run with: TPCH_SCALE_FACTOR=1.0 cargo test test_old_vs_current_config --release -- --nocapture --test-threads=1 +#[tokio::test(flavor = "multi_thread")] +async fn test_old_vs_current_config() -> Result<()> { + let scale_factor = get_scale_factor(0.1); + + println!( + "=== Old (1M+bloom) vs Current (100K+indexes) Benchmark (SF={}) ===", + scale_factor + ); + + // ── OLD CONFIG: 1M row groups + bloom filters ────────────── + println!("\n--- Setting up OLD config (1M row groups + bloom filters) ---"); + let old_setup_start = Instant::now(); + + let old_config = ParquetConfig { + max_row_group_size: 1_000_000, + bloom_filter_enabled: true, + }; + let old_harness = + TpchBenchmarkHarness::new_with_parquet_config(scale_factor, old_config).await?; + println!( + " Engine setup: {:.3}s", + old_setup_start.elapsed().as_secs_f64() + ); + + let _old_conn = old_harness.create_connection().await?; + + // Warmup: sync all tables + println!(" Syncing tables..."); + for query in TPCH_QUERIES { + let _ = old_harness.run_query(query).await; + } + let old_cache_size = TpchBenchmarkHarness::get_parquet_cache_size(old_harness.temp_dir.path()); + println!( + " Cache size: {:.2} MB", + old_cache_size as f64 / 1024.0 / 1024.0 + ); + + // Cached run + println!(" Running queries (cached)..."); + let mut old_results = Vec::new(); + for query in TPCH_QUERIES { + old_results.push(old_harness.run_query(query).await); + } + + // ── CURRENT CONFIG: 100K row groups + 6 indexes ──────────── + println!("\n--- Setting up CURRENT config (100K row groups + indexes) ---"); + let new_setup_start = Instant::now(); + + // Use default config (100K, no bloom) + let new_harness = TpchBenchmarkHarness::new(scale_factor).await?; + println!( + " Engine setup: {:.3}s", + new_setup_start.elapsed().as_secs_f64() + ); + + let _new_conn = new_harness.create_connection().await?; + + // Warmup: sync all tables + println!(" Syncing tables..."); + for query in TPCH_QUERIES { + let _ = new_harness.run_query(query).await; + } + let new_cache_size = TpchBenchmarkHarness::get_parquet_cache_size(new_harness.temp_dir.path()); + println!( + " Cache size (before indexes): {:.2} MB", + new_cache_size as f64 / 1024.0 / 1024.0 + ); + + // Create 6 indexes + println!(" Creating 6 indexes..."); + let indexes = [ + ("idx_shipdate", "tpch.main.lineitem", "(l_shipdate)"), + ("idx_orderdate", "tpch.main.orders", "(o_orderdate)"), + ("idx_partkey", "tpch.main.lineitem", "(l_partkey)"), + ("idx_custkey", "tpch.main.orders", "(o_custkey)"), + ("idx_nationkey", "tpch.main.customer", "(c_nationkey)"), + ("idx_suppkey", "tpch.main.partsupp", "(ps_suppkey)"), + ]; + + for (name, table, cols) in &indexes { + let sql = format!("CREATE INDEX {} ON {} {}", name, table, cols); + let start = Instant::now(); + match new_harness.engine.execute_query(&sql).await { + Ok(_) => println!( + " {} on {} {}: {:.2}ms", + name, + table, + cols, + start.elapsed().as_secs_f64() * 1000.0 + ), + Err(e) => println!(" FAILED {}: {}", name, e), + } + } + + let new_cache_with_idx = + TpchBenchmarkHarness::get_parquet_cache_size(new_harness.temp_dir.path()); + println!( + " Cache size (with indexes): {:.2} MB", + new_cache_with_idx as f64 / 1024.0 / 1024.0 + ); + + // Cached run + println!(" Running queries (cached)..."); + let mut new_results = Vec::new(); + for query in TPCH_QUERIES { + new_results.push(new_harness.run_query(query).await); + } + + // ── COMPARISON TABLE ─────────────────────────────────────── + println!(); + println!("========================================================================"); + println!( + " Old (1M+bloom) vs Current (100K+indexes) SF={}", + scale_factor + ); + println!("========================================================================"); + println!( + "{:<8} {:>12} {:>12} {:>10} {:>8} {:>8}", + "Query", "Old (ms)", "New (ms)", "Change", "Old Row", "New Row" + ); + println!("{}", "-".repeat(70)); + + let mut old_total = Duration::ZERO; + let mut new_total = Duration::ZERO; + let mut old_success = 0; + let mut new_success = 0; + + for (i, query) in TPCH_QUERIES.iter().enumerate() { + let old_ms = old_results[i].duration.as_secs_f64() * 1000.0; + let new_ms = new_results[i].duration.as_secs_f64() * 1000.0; + old_total += old_results[i].duration; + new_total += new_results[i].duration; + if old_results[i].success { + old_success += 1; + } + if new_results[i].success { + new_success += 1; + } + + let delta = if old_ms > 0.0 { + let pct = ((new_ms - old_ms) / old_ms) * 100.0; + if pct < -1.0 { + format!("{:.0}%", pct) + } else if pct > 1.0 { + format!("+{:.0}%", pct) + } else { + "~same".to_string() + } + } else { + "N/A".to_string() + }; + + let old_flag = if old_results[i].success { "" } else { "*" }; + let new_flag = if new_results[i].success { "" } else { "*" }; + + println!( + "{:<8} {:>11.2}{} {:>11.2}{} {:>10} {:>8} {:>8}", + query.name, + old_ms, + old_flag, + new_ms, + new_flag, + delta, + old_results[i].row_count, + new_results[i].row_count, + ); + } + + println!("{}", "-".repeat(70)); + let old_total_ms = old_total.as_secs_f64() * 1000.0; + let new_total_ms = new_total.as_secs_f64() * 1000.0; + let overall_speedup = old_total.as_secs_f64() / new_total.as_secs_f64(); + let overall_pct = ((new_total_ms - old_total_ms) / old_total_ms) * 100.0; + println!( + "{:<8} {:>12.0} {:>12.0} {:>9.0}% {}/22 {}/22", + "TOTAL", old_total_ms, new_total_ms, overall_pct, old_success, new_success, + ); + println!( + "Overall speedup: {:.2}x Cache: {:.1}MB (old) vs {:.1}MB (new+idx)", + overall_speedup, + old_cache_size as f64 / 1024.0 / 1024.0, + new_cache_with_idx as f64 / 1024.0 / 1024.0, + ); + println!("========================================================================"); + + // Verify correctness: row counts must match between configs + for (i, _query) in TPCH_QUERIES.iter().enumerate() { + if old_results[i].success && new_results[i].success { + assert_eq!( + old_results[i].row_count, new_results[i].row_count, + "Row count mismatch for {}: old={} vs new={}", + old_results[i].name, old_results[i].row_count, new_results[i].row_count + ); + } + } + + Ok(()) +} + +/// Pure row group size comparison: 1M (no bloom) vs 100K (no bloom), no indexes. +/// +/// Run with: TPCH_SCALE_FACTOR=1.0 cargo test test_rowgroup_size_comparison --release -- --nocapture --test-threads=1 +#[tokio::test(flavor = "multi_thread")] +async fn test_rowgroup_size_comparison() -> Result<()> { + let scale_factor = get_scale_factor(0.1); + + println!( + "=== Row Group Size: 1M vs 100K (no bloom, no indexes) SF={} ===", + scale_factor + ); + + // ── 1M row groups, no bloom ────────────── + println!("\n--- Setting up 1M row groups (no bloom) ---"); + let old_start = Instant::now(); + + let old_config = ParquetConfig { + max_row_group_size: 1_000_000, + bloom_filter_enabled: false, + }; + let old_harness = + TpchBenchmarkHarness::new_with_parquet_config(scale_factor, old_config).await?; + println!(" Engine setup: {:.3}s", old_start.elapsed().as_secs_f64()); + + let _old_conn = old_harness.create_connection().await?; + + println!(" Syncing tables..."); + for query in TPCH_QUERIES { + let _ = old_harness.run_query(query).await; + } + let old_cache_size = TpchBenchmarkHarness::get_parquet_cache_size(old_harness.temp_dir.path()); + println!( + " Cache size: {:.2} MB", + old_cache_size as f64 / 1024.0 / 1024.0 + ); + + println!(" Running queries (cached)..."); + let mut old_results = Vec::new(); + for query in TPCH_QUERIES { + old_results.push(old_harness.run_query(query).await); + } + + // ── 100K row groups, no bloom, NO indexes ────────────── + println!("\n--- Setting up 100K row groups (no bloom, no indexes) ---"); + let new_start = Instant::now(); + + let new_harness = TpchBenchmarkHarness::new(scale_factor).await?; + println!(" Engine setup: {:.3}s", new_start.elapsed().as_secs_f64()); + + let _new_conn = new_harness.create_connection().await?; + + println!(" Syncing tables..."); + for query in TPCH_QUERIES { + let _ = new_harness.run_query(query).await; + } + let new_cache_size = TpchBenchmarkHarness::get_parquet_cache_size(new_harness.temp_dir.path()); + println!( + " Cache size: {:.2} MB", + new_cache_size as f64 / 1024.0 / 1024.0 + ); + + println!(" Running queries (cached)..."); + let mut new_results = Vec::new(); + for query in TPCH_QUERIES { + new_results.push(new_harness.run_query(query).await); + } + + // ── COMPARISON ─────────────────────────────────────── + println!(); + println!("========================================================================"); + println!( + " 1M row groups vs 100K row groups (no bloom, no indexes) SF={}", + scale_factor + ); + println!("========================================================================"); + println!( + "{:<8} {:>12} {:>12} {:>14} {:>8} {:>8}", + "Query", "1M (ms)", "100K (ms)", "Improvement", "1M Row", "100K Row" + ); + println!("{}", "-".repeat(72)); + + let mut old_total = Duration::ZERO; + let mut new_total = Duration::ZERO; + + for (i, query) in TPCH_QUERIES.iter().enumerate() { + let old_ms = old_results[i].duration.as_secs_f64() * 1000.0; + let new_ms = new_results[i].duration.as_secs_f64() * 1000.0; + old_total += old_results[i].duration; + new_total += new_results[i].duration; + + let improvement = if old_ms > 0.0 { + let pct = ((old_ms - new_ms) / old_ms) * 100.0; + if pct > 1.0 { + format!("{:.0}% faster", pct) + } else if pct < -1.0 { + format!("{:.0}% slower", -pct) + } else { + "~same".to_string() + } + } else { + "N/A".to_string() + }; + + println!( + "{:<8} {:>10.2}ms {:>10.2}ms {:>14} {:>8} {:>8}", + query.name, + old_ms, + new_ms, + improvement, + old_results[i].row_count, + new_results[i].row_count, + ); + } + + println!("{}", "-".repeat(72)); + let old_total_ms = old_total.as_secs_f64() * 1000.0; + let new_total_ms = new_total.as_secs_f64() * 1000.0; + let speedup = old_total.as_secs_f64() / new_total.as_secs_f64(); + let pct = ((old_total_ms - new_total_ms) / old_total_ms) * 100.0; + println!( + "{:<8} {:>10.0}ms {:>10.0}ms {:>11.0}% faster", + "TOTAL", old_total_ms, new_total_ms, pct, + ); + println!( + "Overall speedup: {:.2}x Cache: {:.1}MB (1M) vs {:.1}MB (100K) = +{:.0}%", + speedup, + old_cache_size as f64 / 1024.0 / 1024.0, + new_cache_size as f64 / 1024.0 / 1024.0, + ((new_cache_size as f64 - old_cache_size as f64) / old_cache_size as f64) * 100.0, + ); + println!("========================================================================"); + + // Verify correctness + for (i, _) in TPCH_QUERIES.iter().enumerate() { + if old_results[i].success && new_results[i].success { + assert_eq!( + old_results[i].row_count, new_results[i].row_count, + "Row count mismatch for {}", + old_results[i].name + ); + } + } + + Ok(()) +}