From 7362d0eb3b4d0871ab4d7e70522fd2bb2ea7f39c Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Thu, 23 Feb 2023 15:33:02 +0100 Subject: [PATCH 01/21] Initial migration of create and insert table paths to delta-rs --- Cargo.lock | 4 +- Cargo.toml | 9 +- ...0221081920_delta_lake_integration.down.sql | 2 + ...230221081920_delta_lake_integration.up.sql | 4 + ...0221081928_delta_lake_integration.down.sql | 2 + ...230221081928_delta_lake_integration.up.sql | 4 + src/catalog.rs | 64 +++++++++- src/config/context.rs | 69 +++++++++-- src/context.rs | 110 +++++++++++------- src/datafusion/utils.rs | 2 +- src/nodes.rs | 3 +- src/provider.rs | 100 ++++++++++++---- src/repository/default.rs | 7 +- src/repository/interface.rs | 24 +++- tests/statements/query.rs | 1 - 15 files changed, 315 insertions(+), 90 deletions(-) create mode 100644 migrations/postgres/20230221081920_delta_lake_integration.down.sql create mode 100644 migrations/postgres/20230221081920_delta_lake_integration.up.sql create mode 100644 migrations/sqlite/20230221081928_delta_lake_integration.down.sql create mode 100644 migrations/sqlite/20230221081928_delta_lake_integration.up.sql diff --git a/Cargo.lock b/Cargo.lock index e611c73e..33693e34 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1878,7 +1878,6 @@ checksum = "eaa37046cc0f6c3cc6090fbdbf73ef0b8ef4cfcc37f6befc0020f63e8cf121e1" [[package]] name = "deltalake" version = "0.7.0" -source = "git+https://github.com/splitgraph/delta-rs?branch=datafusion-18-upgrade#f1e972fbafb8d6ec48d83cfec81f57e1f09e40dd" dependencies = [ "arrow", "async-trait", @@ -2032,7 +2031,6 @@ checksum = "1435fa1053d8b2fbbe9be7e97eca7f33d37b28409959813daefc1446a14247f1" [[package]] name = "dynamodb_lock" version = "0.4.2" -source = "git+https://github.com/splitgraph/delta-rs?branch=datafusion-18-upgrade#f1e972fbafb8d6ec48d83cfec81f57e1f09e40dd" dependencies = [ "async-trait", "log", @@ -2571,7 +2569,6 @@ dependencies = [ [[package]] name = "glibc_version" version = "0.1.2" -source = "git+https://github.com/splitgraph/delta-rs?branch=datafusion-18-upgrade#f1e972fbafb8d6ec48d83cfec81f57e1f09e40dd" dependencies = [ "regex", ] @@ -3883,6 +3880,7 @@ dependencies = [ "lz4", "num", "num-bigint", + "object_store", "paste", "seq-macro", "snap", diff --git a/Cargo.toml b/Cargo.toml index 39223baf..1362a8ae 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -22,8 +22,7 @@ include = [ [features] catalog-postgres = ["sqlx/postgres"] -default = ["catalog-postgres", "delta-tables", "frontend-postgres", "object-store-s3", "remote-tables"] -delta-tables = ["dep:deltalake", "dep:dynamodb_lock"] +default = ["catalog-postgres", "frontend-postgres", "object-store-s3", "remote-tables"] frontend-postgres = ["convergence", "convergence-arrow"] object-store-s3 = ["object_store/aws"] remote-tables = ["dep:datafusion-remote-tables"] @@ -55,8 +54,8 @@ datafusion-proto = "18.0.0" datafusion-remote-tables = { path = "./datafusion_remote_tables", optional = true } -deltalake = { git = "https://github.com/splitgraph/delta-rs", branch = "datafusion-18-upgrade", features = ["s3-native-tls", "datafusion-ext"], optional = true } -dynamodb_lock = { git = "https://github.com/splitgraph/delta-rs", branch = "datafusion-18-upgrade", package = "dynamodb_lock", default_features = false, features = ["native-tls"], optional = true } +deltalake = { path = "../delta-rs/rust", features = ["s3-native-tls", "datafusion-ext"] } +dynamodb_lock = { path = "../delta-rs/dynamodb_lock", default_features = false, features = ["native-tls"] } futures = "0.3" hex = ">=0.4.0" @@ -65,7 +64,7 @@ lazy_static = ">=1.4.0" log = "0.4" moka = { version = "0.10.0", default_features = false, features = ["future", "atomic64", "quanta"] } object_store = "0.5.2" -parking_lot = "0.12.1" +parking_lot = { version = "0.12.1", features = ["send_guard"] } percent-encoding = "2.2.0" pretty_env_logger = "0.4" prost = "0.11" diff --git a/migrations/postgres/20230221081920_delta_lake_integration.down.sql b/migrations/postgres/20230221081920_delta_lake_integration.down.sql new file mode 100644 index 00000000..6577cf3a --- /dev/null +++ b/migrations/postgres/20230221081920_delta_lake_integration.down.sql @@ -0,0 +1,2 @@ +-- Add down migration script here +ALTER TABLE "table" DROP COLUMN legacy; diff --git a/migrations/postgres/20230221081920_delta_lake_integration.up.sql b/migrations/postgres/20230221081920_delta_lake_integration.up.sql new file mode 100644 index 00000000..a2b6031b --- /dev/null +++ b/migrations/postgres/20230221081920_delta_lake_integration.up.sql @@ -0,0 +1,4 @@ +-- Add up migration script here +ALTER TABLE "table" ADD COLUMN legacy BOOLEAN DEFAULT FALSE; + +UPDATE "table" SET legacy = TRUE; diff --git a/migrations/sqlite/20230221081928_delta_lake_integration.down.sql b/migrations/sqlite/20230221081928_delta_lake_integration.down.sql new file mode 100644 index 00000000..6577cf3a --- /dev/null +++ b/migrations/sqlite/20230221081928_delta_lake_integration.down.sql @@ -0,0 +1,2 @@ +-- Add down migration script here +ALTER TABLE "table" DROP COLUMN legacy; diff --git a/migrations/sqlite/20230221081928_delta_lake_integration.up.sql b/migrations/sqlite/20230221081928_delta_lake_integration.up.sql new file mode 100644 index 00000000..a2b6031b --- /dev/null +++ b/migrations/sqlite/20230221081928_delta_lake_integration.up.sql @@ -0,0 +1,4 @@ +-- Add up migration script here +ALTER TABLE "table" ADD COLUMN legacy BOOLEAN DEFAULT FALSE; + +UPDATE "table" SET legacy = TRUE; diff --git a/src/catalog.rs b/src/catalog.rs index a10f1a93..e1bca2f3 100644 --- a/src/catalog.rs +++ b/src/catalog.rs @@ -4,6 +4,7 @@ use std::{collections::HashMap, fmt::Debug, sync::Arc}; use async_trait::async_trait; use datafusion::catalog::schema::MemorySchemaProvider; use datafusion::error::DataFusionError; +use deltalake::{DeltaTable, DeltaTableBuilder}; use itertools::Itertools; #[cfg(test)] use mockall::automock; @@ -276,14 +277,22 @@ pub struct DefaultCatalog { // DataFusion's in-memory schema provider for staging external tables staging_schema: Arc, + storage_options: HashMap, + store_uri: String, } impl DefaultCatalog { - pub fn new(repository: Arc) -> Self { + pub fn new( + repository: Arc, + storage_options: HashMap, + store_uri: String, + ) -> Self { let staging_schema = Arc::new(MemorySchemaProvider::new()); Self { repository, staging_schema, + storage_options, + store_uri, } } @@ -320,7 +329,7 @@ impl DefaultCatalog { } } - fn build_table<'a, I>( + fn build_legacy_table<'a, I>( &self, table_name: &str, table_columns: I, @@ -358,6 +367,38 @@ impl DefaultCatalog { (Arc::from(table_name.to_string()), Arc::new(table)) } + fn build_table<'a, I>( + &self, + table_name: &str, + table_columns: I, + ) -> (Arc, Arc) + where + I: Iterator, + { + let table_columns_vec = table_columns.collect_vec(); + + // Build a delta table but don't load it yet; we'll do that only for tables that are + // actually referenced in a statement, via the async `table` method of the schema provider. + + // TODO: if the table has no columns, the result set will be empty, so we use the default DB and schema names. + let (database_name, collection_name) = table_columns_vec.get(0).map_or_else( + || (DEFAULT_DB, DEFAULT_SCHEMA), + |v| (&v.database_name, &v.collection_name), + ); + + let table_uri = format!( + "{}/{}/{}/{}", + self.store_uri, database_name, collection_name, table_name + ); + + let table = DeltaTableBuilder::from_uri(table_uri) + .with_storage_options(self.storage_options.clone()) + .build() + .unwrap(); + + (Arc::from(table_name.to_string()), Arc::new(table)) + } + fn build_collection<'a, I>( &self, collection_name: &str, @@ -366,7 +407,20 @@ impl DefaultCatalog { where I: Iterator, { - let tables = collection_columns + let collection_columns_vec = collection_columns.collect_vec(); + + let legacy_tables = collection_columns_vec + .clone() + .into_iter() + .filter(|c| c.table_legacy) + .group_by(|col| &col.table_name) + .into_iter() + .map(|(tn, tc)| self.build_legacy_table(tn, tc)) + .collect::>(); + + let tables = collection_columns_vec + .into_iter() + .filter(|c| !c.table_legacy) .group_by(|col| &col.table_name) .into_iter() .map(|(tn, tc)| self.build_table(tn, tc)) @@ -376,6 +430,7 @@ impl DefaultCatalog { Arc::from(collection_name.to_string()), Arc::new(SeafowlCollection { name: Arc::from(collection_name.to_string()), + legacy_tables: RwLock::new(legacy_tables), tables: RwLock::new(tables), }), ) @@ -440,9 +495,10 @@ impl TableCatalog for DefaultCatalog { let tables = table_columns .iter() + .filter(|col| col.table_legacy) .group_by(|col| (&col.table_name, &col.table_version_id)) .into_iter() - .map(|((tn, &tv), tc)| (tv, self.build_table(tn, tc).1)) + .map(|((tn, &tv), tc)| (tv, self.build_legacy_table(tn, tc).1)) .collect(); Ok(tables) diff --git a/src/config/context.rs b/src/config/context.rs index 5a5fa61a..4ae25730 100644 --- a/src/config/context.rs +++ b/src/config/context.rs @@ -15,7 +15,6 @@ use datafusion::{ execution::runtime_env::{RuntimeConfig, RuntimeEnv}, prelude::{SessionConfig, SessionContext}, }; -#[cfg(feature = "delta-tables")] use deltalake::delta_datafusion::DeltaTableFactory; use object_store::{local::LocalFileSystem, memory::InMemory, ObjectStore}; @@ -28,12 +27,15 @@ use crate::object_store::wrapped::InternalObjectStore; use datafusion_remote_tables::factory::RemoteTableFactory; #[cfg(feature = "object-store-s3")] use object_store::aws::AmazonS3Builder; +use object_store::aws::AmazonS3ConfigKey; use parking_lot::lock_api::RwLock; use super::schema::{self, MEBIBYTES, MEMORY_FRACTION, S3}; async fn build_catalog( config: &schema::SeafowlConfig, + storage_options: HashMap, + store_uri: String, ) -> ( Arc, Arc, @@ -67,7 +69,7 @@ async fn build_catalog( ), }; - let catalog = Arc::new(DefaultCatalog::new(repository)); + let catalog = Arc::new(DefaultCatalog::new(repository, storage_options, store_uri)); (catalog.clone(), catalog.clone(), catalog) } @@ -104,6 +106,58 @@ fn build_object_store(cfg: &schema::SeafowlConfig) -> Arc { } } +fn build_storage_options( + cfg: &schema::SeafowlConfig, +) -> (HashMap, String) { + match &cfg.object_store { + schema::ObjectStore::Local(schema::Local { data_dir }) => { + (HashMap::::default(), data_dir.clone()) + } + schema::ObjectStore::InMemory(_) => { + (HashMap::::default(), "/".to_string()) + } + #[cfg(feature = "object-store-s3")] + schema::ObjectStore::S3(S3 { + region, + access_key_id, + secret_access_key, + endpoint, + bucket, + }) => { + let mut storage_options = HashMap::::from([ + ( + AmazonS3ConfigKey::AccessKeyId.as_ref().to_string(), + access_key_id.clone(), + ), + ( + AmazonS3ConfigKey::SecretAccessKey.as_ref().to_string(), + secret_access_key.clone(), + ), + ( + AmazonS3ConfigKey::Bucket.as_ref().to_string(), + bucket.clone(), + ), + ]); + + if let Some(region) = region { + storage_options.insert( + AmazonS3ConfigKey::Region.as_ref().to_string(), + region.clone(), + ); + } + + if let Some(endpoint) = endpoint { + storage_options.insert( + AmazonS3ConfigKey::Endpoint.as_ref().to_string(), + endpoint.clone(), + ); + } + + (storage_options, format!("s3://{bucket}")) + } + } +} + #[allow(unused_mut)] pub async fn build_context( cfg: &schema::SeafowlConfig, @@ -131,14 +185,11 @@ pub async fn build_context( // the default ones for PARQUET, CSV, etc. let mut table_factories: HashMap> = HashMap::new(); + table_factories.insert("DELTATABLE".to_string(), Arc::new(DeltaTableFactory {})); #[cfg(feature = "remote-tables")] { table_factories.insert("TABLE".to_string(), Arc::new(RemoteTableFactory {})); } - #[cfg(feature = "delta-tables")] - { - table_factories.insert("DELTATABLE".to_string(), Arc::new(DeltaTableFactory {})); - } let mut runtime_env = RuntimeEnv::new(runtime_config)?; runtime_env.register_table_factories(table_factories); @@ -151,11 +202,13 @@ pub async fn build_context( "", object_store.clone(), ); + let (storage_options, store_uri) = build_storage_options(cfg); // Register the HTTP object store for external tables add_http_object_store(&context, &cfg.misc.ssl_cert_file); - let (tables, partitions, functions) = build_catalog(cfg).await; + let (tables, partitions, functions) = + build_catalog(cfg, storage_options.clone(), store_uri.clone()).await; // Create default DB/collection let default_db = match tables.get_database_id_by_name(DEFAULT_DB).await? { @@ -188,6 +241,8 @@ pub async fn build_context( inner: object_store, config: cfg.object_store.clone(), }), + storage_options, + store_uri, database: DEFAULT_DB.to_string(), database_id: default_db, all_database_ids: Arc::from(RwLock::new(all_database_ids)), diff --git a/src/context.rs b/src/context.rs index e0a3ae92..03b28a50 100644 --- a/src/context.rs +++ b/src/context.rs @@ -85,6 +85,8 @@ use datafusion_expr::logical_plan::{ DropTable, Extension, LogicalPlan, Projection, }; use datafusion_expr::{DmlStatement, Filter, WriteOp}; +use deltalake::operations::{create::CreateBuilder, write::WriteBuilder}; +use deltalake::{DeltaTable, Schema as DeltaSchema}; use log::{debug, info, warn}; use parking_lot::RwLock; use prost::Message; @@ -92,7 +94,7 @@ use tempfile::TempPath; use tokio::sync::Semaphore; use crate::catalog::{PartitionCatalog, DEFAULT_SCHEMA, STAGING_SCHEMA}; -use crate::data_types::{PhysicalPartitionId, TableId, TableVersionId}; +use crate::data_types::{PhysicalPartitionId, TableVersionId}; use crate::datafusion::visit::VisitorMut; use crate::provider::{ project_expressions, PartitionColumn, SeafowlPartition, SeafowlPruningStatistics, @@ -256,6 +258,8 @@ pub struct DefaultSeafowlContext { pub partition_catalog: Arc, pub function_catalog: Arc, pub internal_object_store: Arc, + pub storage_options: HashMap, + pub store_uri: String, pub database: String, pub database_id: DatabaseId, pub all_database_ids: Arc>>, @@ -597,6 +601,8 @@ impl DefaultSeafowlContext { partition_catalog: self.partition_catalog.clone(), function_catalog: self.function_catalog.clone(), internal_object_store: self.internal_object_store.clone(), + storage_options: self.storage_options.clone(), + store_uri: self.store_uri.clone(), database: name, database_id, all_database_ids: self.all_database_ids.clone(), @@ -604,6 +610,33 @@ impl DefaultSeafowlContext { })) } + /// Construct the full table uri in the underlying object store + pub fn table_to_location<'a>(&self, table: impl Into>) -> String { + let table_ref: TableReference = table.into(); + let resolved_ref = table_ref.resolve(&self.database, DEFAULT_SCHEMA); + + format!( + "{}/{}/{}/{}", + self.store_uri, resolved_ref.catalog, resolved_ref.schema, resolved_ref.table + ) + } + + /// Generate the Delta table builder and execute the write + pub async fn plan_to_delta_table( + &self, + plan: &Arc, + location: impl Into, + ) -> Result { + let table = WriteBuilder::new() + .with_input_execution_plan(plan.clone()) + .with_storage_options(self.storage_options.clone()) + .with_location(location) + .await?; + + debug!("Written table version {} for {table}", table.version()); + Ok(table) + } + pub fn inner(&self) -> &SessionContext { &self.inner } @@ -721,18 +754,14 @@ impl DefaultSeafowlContext { Ok(seafowl_table.clone()) } - async fn exec_create_table( - &self, - name: &str, - schema: &Arc, - ) -> Result<(TableId, TableVersionId)> { + async fn exec_create_table(&self, name: &str, schema: &Schema) -> Result { let table_ref = TableReference::from(name); let resolved_ref = table_ref.resolve(&self.database, DEFAULT_SCHEMA); let schema_name = resolved_ref.schema; let table_name = resolved_ref.table; let sf_schema = SeafowlSchema { - arrow_schema: Arc::new(schema.as_ref().into()), + arrow_schema: Arc::new(schema.clone()), }; let collection_id = self .table_catalog @@ -741,10 +770,30 @@ impl DefaultSeafowlContext { .ok_or_else(|| { Error::Plan(format!("Schema {schema_name:?} does not exist!")) })?; - Ok(self - .table_catalog + + let delta_schema = DeltaSchema::try_from(schema)?; + let location = self.table_to_location(name); + + let table = CreateBuilder::new() + .with_storage_options(self.storage_options.clone()) + .with_location(&location) + .with_table_name(&*table_name) + .with_columns(delta_schema.get_fields().clone()) + .with_comment(format!( + "Created by Seafowl version {}", + env!("CARGO_PKG_VERSION") + )) + .await?; + + // We still persist the table into our own catalog, the principal reason being able to load + // all collections/tables and their schemas in bulk to satisfy information_schema queries. + // We should look into doing this via delta-rs somehow eventually. + self.table_catalog .create_table(collection_id, &table_name, &sf_schema) - .await?) + .await?; + + debug!("Created new table at location {location}"); + Ok(table) } fn register_function( @@ -804,25 +853,17 @@ impl DefaultSeafowlContext { output_schema: Option, name: Option, from_table_version: Option, - ) -> Result { - let partition_ids = self - .execute_plan_to_partitions(physical_plan, output_schema.clone()) - .await?; - + ) -> Result<()> { // Create/Update table metadata - let new_table_version_id; match (name, from_table_version) { (Some(name), _) => { let schema = output_schema.unwrap_or_else(|| physical_plan.schema()); // Create an empty table with an empty version - (_, new_table_version_id) = self - .exec_create_table(&name, &schema.to_dfschema_ref()?) - .await?; + self.exec_create_table(&name, schema.as_ref()).await?; } (_, Some(from_table_version)) => { // Duplicate the table version into a new one - new_table_version_id = self - .table_catalog + self.table_catalog .create_new_table_version(from_table_version, true) .await?; } @@ -833,12 +874,7 @@ impl DefaultSeafowlContext { } } - // Attach the partitions to the table - self.partition_catalog - .append_partitions_to_table(partition_ids.clone(), new_table_version_id) - .await?; - - Ok(new_table_version_id) + Ok(()) } // Generate new physical Parquet partition files from the provided plan, upload to object store @@ -1184,10 +1220,10 @@ impl SeafowlContext for DefaultSeafowlContext { && table_properties.is_empty() && with_options.is_empty() => { - let cols = build_schema(columns)?; + let schema = build_schema(columns)?; Ok(LogicalPlan::Extension(Extension { node: Arc::new(SeafowlExtensionNode::CreateTable(CreateTable { - schema: cols.to_dfschema_ref()?, + schema, name: remove_quotes_from_object_name(&name).to_string(), if_not_exists, output_schema: Arc::new(DFSchema::empty()) @@ -1402,17 +1438,10 @@ impl SeafowlContext for DefaultSeafowlContext { input, .. }) => { - let table = self.try_get_seafowl_table(table_name.to_string()).await?; - let physical = self.create_physical_plan(input).await?; - self.execute_plan_to_table( - &physical, - None, - None, - Some(table.table_version_id), - ) - .await?; + self.plan_to_delta_table(&physical, self.table_to_location(table_name)) + .await?; Ok(make_dummy_exec()) } @@ -1971,7 +2000,8 @@ pub mod test_utils { Arc::from("testcol"), Arc::from(SeafowlCollection { name: Arc::from("testcol"), - tables: RwLock::new(tables), + legacy_tables: RwLock::new(tables), + tables: Default::default(), }), )]); @@ -2022,6 +2052,8 @@ pub mod test_utils { partition_catalog: partition_catalog_ptr, function_catalog: Arc::new(function_catalog), internal_object_store: object_store, + storage_options: Default::default(), + store_uri: "/".to_string(), database: "testdb".to_string(), database_id: 0, all_database_ids: Arc::from(RwLock::new(HashMap::from([( diff --git a/src/datafusion/utils.rs b/src/datafusion/utils.rs index 65e56b04..243339f9 100644 --- a/src/datafusion/utils.rs +++ b/src/datafusion/utils.rs @@ -73,7 +73,7 @@ pub(crate) fn convert_simple_data_type(sql_type: &SQLDataType) -> Result Ok(DataType::Date32), SQLDataType::Time(None, tz_info) => { diff --git a/src/nodes.rs b/src/nodes.rs index da7d3c65..30ad6d4d 100644 --- a/src/nodes.rs +++ b/src/nodes.rs @@ -1,5 +1,6 @@ use datafusion::common::DFSchemaRef; +use arrow_schema::Schema; use std::{any::Any, fmt, sync::Arc, vec}; use datafusion_expr::{Expr, LogicalPlan, UserDefinedLogicalNode}; @@ -10,7 +11,7 @@ use crate::{provider::SeafowlTable, wasm_udf::data_types::CreateFunctionDetails} #[derive(Debug, Clone)] pub struct CreateTable { /// The table schema - pub schema: DFSchemaRef, + pub schema: Schema, /// The table name pub name: String, /// Option to not error if table already exists diff --git a/src/provider.rs b/src/provider.rs index d06f901b..e47d420b 100644 --- a/src/provider.rs +++ b/src/provider.rs @@ -39,6 +39,7 @@ use datafusion::{ use datafusion_common::DFSchema; use datafusion_expr::Expr; use datafusion_proto::protobuf; +use deltalake::{DeltaDataTypeVersion, DeltaTable}; use futures::future; use log::warn; @@ -91,7 +92,9 @@ impl CatalogProvider for SeafowlDatabase { pub struct SeafowlCollection { pub name: Arc, - pub tables: RwLock, Arc>>, + // TODO: consider using DashMap instead of RwLock>: https://github.com/xacrimon/conc-map-bench + pub legacy_tables: RwLock, Arc>>, + pub tables: RwLock, Arc>>, } #[async_trait] @@ -101,21 +104,66 @@ impl SchemaProvider for SeafowlCollection { } fn table_names(&self) -> Vec { - let tables = self.tables.read(); - tables.keys().map(|s| s.to_string()).collect() + let mut table_names: Vec<_> = self + .legacy_tables + .read() + .keys() + .map(|s| s.to_string()) + .collect(); + table_names.extend( + self.tables + .read() + .keys() + .map(|s| s.to_string()) + .collect::>(), + ); + table_names } async fn table(&self, name: &str) -> Option> { - let tables = self.tables.read(); - tables.get(name).map(|c| Arc::clone(c) as _) + { + let legacy_table = self + .legacy_tables + .read() + .get(name) + .map(|c| Arc::clone(c) as _); + + if legacy_table.is_some() { + return legacy_table; + } + } + + // TODO: This is not very nice: cloning into a new table, instead of just calling `load` on + // the existing one (which we can't because it's behind of an Arc, and `load` needs `mut`). + // We may be able to improve it by: + // 1. removing the `Arc` from the value in the map + // 2. enclosing the entire map inside of an `Arc` + // 3. using `entry` for in-place mutation + // Ultimately though, since the map gets re-created for each query the only point in + // updating the existing table is to optimize potential multi-lookups during processing of + // a single query. + let mut delta_table = match self.tables.read().get(name) { + None => return None, + Some(table) => table.as_ref().clone(), + }; + + if delta_table.version() == -1 as DeltaDataTypeVersion { + // A negative table version indicates that the table was never loaded + if let Err(err) = delta_table.load().await { + warn!("Failed to load table {name}: {err}"); + return None; + } + } + + Some(Arc::from(delta_table) as _) } fn table_exist(&self, name: &str) -> bool { - let tables = self.tables.read(); - tables.contains_key(name) + self.legacy_tables.read().contains_key(name) + || self.tables.read().contains_key(name) } - // Used for registering versioned as well as delta/remote tables via `SessionContext::register_table`. + // Used for registering versioned tables via `SessionContext::register_table`. fn register_table( &self, name: String, @@ -126,23 +174,25 @@ impl SchemaProvider for SeafowlCollection { "The table {name} already exists" ))); } - let mut tables = self.tables.write(); - let old_table = tables.insert( - Arc::from(name), - Arc::new( - table - .as_any() - .downcast_ref::() - .ok_or_else(|| { - DataFusionError::Execution(format!( - "Couldn't cast table {:?} to SeafowlTable", - table.schema(), - )) - })? - .clone(), - ), - ); - Ok(old_table.map(|t| t as Arc)) + + if let Some(legacy_table) = table.as_any().downcast_ref::() { + let mut tables = self.legacy_tables.write(); + return Ok(tables + .insert(Arc::from(name), Arc::from(legacy_table.clone())) + .map(|t| t as Arc)); + } + + if let Some(table) = table.as_any().downcast_ref::() { + let mut tables = self.tables.write(); + return Ok(tables + .insert(Arc::from(name), Arc::from(table.clone())) + .map(|t| t as Arc)); + } + + Err(DataFusionError::Execution(format!( + "Couldn't cast table {:?} to SeafowlTable or DeltaTable", + table.schema(), + ))) } } diff --git a/src/repository/default.rs b/src/repository/default.rs index c8ce3a91..5b79c4f5 100644 --- a/src/repository/default.rs +++ b/src/repository/default.rs @@ -109,17 +109,20 @@ impl Repository for $repo { builder.push(r#" SELECT + database.name AS database_name, collection.name AS collection_name, "table".name AS table_name, "table".id AS table_id, + "table".legacy AS table_legacy, desired_table_versions.id AS table_version_id, table_column.name AS column_name, table_column.type AS column_type - FROM collection + FROM database + INNER JOIN collection ON database.id = collection.database_id INNER JOIN "table" ON collection.id = "table".collection_id INNER JOIN desired_table_versions ON "table".id = desired_table_versions.table_id INNER JOIN table_column ON table_column.table_version_id = desired_table_versions.id - WHERE collection.database_id = "#); + WHERE database.id = "#); builder.push_bind(database_id); builder.push(r#" diff --git a/src/repository/interface.rs b/src/repository/interface.rs index e4deb570..f68791c7 100644 --- a/src/repository/interface.rs +++ b/src/repository/interface.rs @@ -14,9 +14,11 @@ use crate::{ #[derive(sqlx::FromRow, Debug, PartialEq, Eq)] pub struct AllDatabaseColumnsResult { + pub database_name: String, pub collection_name: String, pub table_name: String, pub table_id: TableId, + pub table_legacy: bool, pub table_version_id: TableVersionId, pub column_name: String, pub column_type: String, @@ -296,22 +298,27 @@ pub mod tests { fn expected( version: TableVersionId, + database_name: String, collection_name: String, table_name: String, ) -> Vec { vec![ AllDatabaseColumnsResult { + database_name: database_name.clone(), collection_name: collection_name.clone(), table_name: table_name.clone(), table_id: 1, + table_legacy: false, table_version_id: version, column_name: "date".to_string(), column_type: "{\"children\":[],\"name\":\"date\",\"nullable\":false,\"type\":{\"name\":\"date\",\"unit\":\"MILLISECOND\"}}".to_string(), }, AllDatabaseColumnsResult { + database_name, collection_name, table_name, table_id: 1, + table_legacy: false, table_version_id: version, column_name: "value".to_string(), column_type: "{\"children\":[],\"name\":\"value\",\"nullable\":false,\"type\":{\"name\":\"floatingpoint\",\"precision\":\"DOUBLE\"}}" @@ -342,7 +349,12 @@ pub mod tests { assert_eq!( all_columns, - expected(1, "testcol".to_string(), "testtable".to_string()) + expected( + 1, + "testdb".to_string(), + "testcol".to_string(), + "testtable".to_string() + ) ); // Duplicate the table @@ -361,6 +373,7 @@ pub mod tests { all_columns, expected( new_version_id, + "testdb".to_string(), "testcol".to_string(), "testtable".to_string() ) @@ -374,7 +387,12 @@ pub mod tests { assert_eq!( all_columns, - expected(1, "testcol".to_string(), "testtable".to_string()) + expected( + 1, + "testdb".to_string(), + "testcol".to_string(), + "testtable".to_string() + ) ); // Check the existing table versions @@ -537,6 +555,7 @@ pub mod tests { all_columns, expected( table_version_id, + "testdb".to_string(), "testcol".to_string(), "testtable2".to_string() ) @@ -561,6 +580,7 @@ pub mod tests { all_columns, expected( table_version_id, + "testdb".to_string(), "testcol2".to_string(), "testtable2".to_string() ) diff --git a/tests/statements/query.rs b/tests/statements/query.rs index b65fa32a..610bcb12 100644 --- a/tests/statements/query.rs +++ b/tests/statements/query.rs @@ -574,7 +574,6 @@ async fn test_remote_table_querying( }; } -#[cfg(feature = "delta-tables")] #[tokio::test] async fn test_delta_tables() { let context = make_context_with_pg().await; From 88b3e53a27f95bd5fd70680e132464355242f496 Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Fri, 24 Feb 2023 10:53:04 +0100 Subject: [PATCH 02/21] Revise the schema provider implementation to fit with non-cloneable DeltaTable --- Cargo.lock | 37 ++++++++++++++++----------- Cargo.toml | 5 ++++ src/catalog.rs | 7 +++--- src/config/context.rs | 7 +++--- src/context.rs | 4 +-- src/provider.rs | 53 ++++++++++++++++++++++----------------- tests/statements/query.rs | 14 +++++------ 7 files changed, 74 insertions(+), 53 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 33693e34..7947e96f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -125,7 +125,7 @@ dependencies = [ "arrow-ord", "arrow-row", "arrow-schema", - "arrow-select", + "arrow-select 32.0.0 (registry+https://github.com/rust-lang/crates.io-index)", "arrow-string", "bitflags", "comfy-table", @@ -149,8 +149,7 @@ dependencies = [ [[package]] name = "arrow-array" version = "32.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fe66ec388d882a61fff3eb613b5266af133aa08a3318e5e493daf0f5c1696cb" +source = "git+https://github.com/splitgraph/arrow-rs?branch=backport-cast-utf8-to-timestamp-microsecond#d01096c6c9ac845f863d43f326709d3aaa99bdaf" dependencies = [ "ahash 0.8.1", "arrow-buffer", @@ -165,8 +164,7 @@ dependencies = [ [[package]] name = "arrow-buffer" version = "32.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ef967dadbccd4586ec8d7aab27d7033ecb5dfae8a605c839613039eac227bda" +source = "git+https://github.com/splitgraph/arrow-rs?branch=backport-cast-utf8-to-timestamp-microsecond#d01096c6c9ac845f863d43f326709d3aaa99bdaf" dependencies = [ "half", "num", @@ -175,14 +173,13 @@ dependencies = [ [[package]] name = "arrow-cast" version = "32.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "491a7979ea9e76dc218f532896e2d245fde5235e2e6420ce80d27cf6395dda84" +source = "git+https://github.com/splitgraph/arrow-rs?branch=backport-cast-utf8-to-timestamp-microsecond#d01096c6c9ac845f863d43f326709d3aaa99bdaf" dependencies = [ "arrow-array", "arrow-buffer", "arrow-data", "arrow-schema", - "arrow-select", + "arrow-select 32.0.0 (git+https://github.com/splitgraph/arrow-rs?branch=backport-cast-utf8-to-timestamp-microsecond)", "chrono", "lexical-core", "num", @@ -210,8 +207,7 @@ dependencies = [ [[package]] name = "arrow-data" version = "32.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ee0c0e3c5d3b80be8f267f4b2af714c08cad630569be01a8379cfe27b4866495" +source = "git+https://github.com/splitgraph/arrow-rs?branch=backport-cast-utf8-to-timestamp-microsecond#d01096c6c9ac845f863d43f326709d3aaa99bdaf" dependencies = [ "arrow-buffer", "arrow-schema", @@ -276,7 +272,7 @@ dependencies = [ "arrow-buffer", "arrow-data", "arrow-schema", - "arrow-select", + "arrow-select 32.0.0 (registry+https://github.com/rust-lang/crates.io-index)", "num", ] @@ -298,8 +294,7 @@ dependencies = [ [[package]] name = "arrow-schema" version = "32.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ead3f373b9173af52f2fdefcb5a7dd89f453fbc40056f574a8aeb23382a4ef81" +source = "git+https://github.com/splitgraph/arrow-rs?branch=backport-cast-utf8-to-timestamp-microsecond#d01096c6c9ac845f863d43f326709d3aaa99bdaf" [[package]] name = "arrow-select" @@ -314,6 +309,18 @@ dependencies = [ "num", ] +[[package]] +name = "arrow-select" +version = "32.0.0" +source = "git+https://github.com/splitgraph/arrow-rs?branch=backport-cast-utf8-to-timestamp-microsecond#d01096c6c9ac845f863d43f326709d3aaa99bdaf" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "num", +] + [[package]] name = "arrow-string" version = "32.0.0" @@ -324,7 +331,7 @@ dependencies = [ "arrow-buffer", "arrow-data", "arrow-schema", - "arrow-select", + "arrow-select 32.0.0 (registry+https://github.com/rust-lang/crates.io-index)", "regex", "regex-syntax", ] @@ -3869,7 +3876,7 @@ dependencies = [ "arrow-data", "arrow-ipc", "arrow-schema", - "arrow-select", + "arrow-select 32.0.0 (registry+https://github.com/rust-lang/crates.io-index)", "base64 0.21.0", "brotli", "bytes", diff --git a/Cargo.toml b/Cargo.toml index 1362a8ae..86c8b2e6 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -95,6 +95,11 @@ wasmtime = "1.0.1" wasmtime-wasi = "1.0.1" [patch.crates-io] +arrow-data = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-data", branch = "backport-cast-utf8-to-timestamp-microsecond" } +arrow-array = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-array", branch = "backport-cast-utf8-to-timestamp-microsecond" } +arrow-buffer = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-buffer", branch = "backport-cast-utf8-to-timestamp-microsecond" } +arrow-schema = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-schema", branch = "backport-cast-utf8-to-timestamp-microsecond" } +arrow-cast = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-cast", branch = "backport-cast-utf8-to-timestamp-microsecond" } [dev-dependencies] assert_unordered = "0.3" diff --git a/src/catalog.rs b/src/catalog.rs index e1bca2f3..f8098c21 100644 --- a/src/catalog.rs +++ b/src/catalog.rs @@ -3,8 +3,9 @@ use std::{collections::HashMap, fmt::Debug, sync::Arc}; use async_trait::async_trait; use datafusion::catalog::schema::MemorySchemaProvider; +use datafusion::datasource::TableProvider; use datafusion::error::DataFusionError; -use deltalake::{DeltaTable, DeltaTableBuilder}; +use deltalake::DeltaTableBuilder; use itertools::Itertools; #[cfg(test)] use mockall::automock; @@ -371,7 +372,7 @@ impl DefaultCatalog { &self, table_name: &str, table_columns: I, - ) -> (Arc, Arc) + ) -> (Arc, Arc) where I: Iterator, { @@ -396,7 +397,7 @@ impl DefaultCatalog { .build() .unwrap(); - (Arc::from(table_name.to_string()), Arc::new(table)) + (Arc::from(table_name.to_string()), Arc::new(table) as _) } fn build_collection<'a, I>( diff --git a/src/config/context.rs b/src/config/context.rs index 4ae25730..0c899b2d 100644 --- a/src/config/context.rs +++ b/src/config/context.rs @@ -113,9 +113,10 @@ fn build_storage_options( schema::ObjectStore::Local(schema::Local { data_dir }) => { (HashMap::::default(), data_dir.clone()) } - schema::ObjectStore::InMemory(_) => { - (HashMap::::default(), "/".to_string()) - } + schema::ObjectStore::InMemory(_) => ( + HashMap::::default(), + "memory://".to_string(), + ), #[cfg(feature = "object-store-s3")] schema::ObjectStore::S3(S3 { region, diff --git a/src/context.rs b/src/context.rs index 03b28a50..05739955 100644 --- a/src/context.rs +++ b/src/context.rs @@ -785,8 +785,8 @@ impl DefaultSeafowlContext { )) .await?; - // We still persist the table into our own catalog, the principal reason being able to load - // all collections/tables and their schemas in bulk to satisfy information_schema queries. + // We still persist the table into our own catalog, the principal reason is us being able to + // load all tables and their schemas in bulk to satisfy information_schema queries. // We should look into doing this via delta-rs somehow eventually. self.table_catalog .create_table(collection_id, &table_name, &sf_schema) diff --git a/src/provider.rs b/src/provider.rs index e47d420b..b577cb23 100644 --- a/src/provider.rs +++ b/src/provider.rs @@ -39,7 +39,7 @@ use datafusion::{ use datafusion_common::DFSchema; use datafusion_expr::Expr; use datafusion_proto::protobuf; -use deltalake::{DeltaDataTypeVersion, DeltaTable}; +use deltalake::{DeltaTable, DeltaTableBuilder}; use futures::future; use log::warn; @@ -94,7 +94,7 @@ pub struct SeafowlCollection { pub name: Arc, // TODO: consider using DashMap instead of RwLock>: https://github.com/xacrimon/conc-map-bench pub legacy_tables: RwLock, Arc>>, - pub tables: RwLock, Arc>>, + pub tables: RwLock, Arc>>, } #[async_trait] @@ -133,7 +133,7 @@ impl SchemaProvider for SeafowlCollection { } } - // TODO: This is not very nice: cloning into a new table, instead of just calling `load` on + // TODO: This is not very nice: rebuilding the table from scratch, instead of just `load` on // the existing one (which we can't because it's behind of an Arc, and `load` needs `mut`). // We may be able to improve it by: // 1. removing the `Arc` from the value in the map @@ -142,20 +142,37 @@ impl SchemaProvider for SeafowlCollection { // Ultimately though, since the map gets re-created for each query the only point in // updating the existing table is to optimize potential multi-lookups during processing of // a single query. - let mut delta_table = match self.tables.read().get(name) { + let (table_uri, object_store) = match self.tables.read().get(name) { None => return None, - Some(table) => table.as_ref().clone(), + Some(table) => match table.as_any().downcast_ref::() { + // This shouldn't happen since we stsore only DeltaTable's in the map + None => return Some(table.clone()), + Some(delta_table) => { + if delta_table.version() != -1 { + // Table was already loaded. + return Some(table.clone()); + } else { + // A negative table version indicates that the table was never loaded; we need + // to do it before returning it. + (delta_table.table_uri(), delta_table.object_store()) + } + } + }, }; - if delta_table.version() == -1 as DeltaDataTypeVersion { - // A negative table version indicates that the table was never loaded - if let Err(err) = delta_table.load().await { - warn!("Failed to load table {name}: {err}"); - return None; - } + let maybe_table = DeltaTableBuilder::from_uri(table_uri.clone()) + .with_storage_backend(object_store, table_uri.parse().unwrap()) + .load() + .await; + + if let Err(err) = maybe_table { + warn!("Failed to load table {name}: {err}"); + return None; } - Some(Arc::from(delta_table) as _) + let table = Arc::from(maybe_table.unwrap()) as Arc; + self.tables.write().insert(Arc::from(name), table.clone()); + Some(table) } fn table_exist(&self, name: &str) -> bool { @@ -182,17 +199,7 @@ impl SchemaProvider for SeafowlCollection { .map(|t| t as Arc)); } - if let Some(table) = table.as_any().downcast_ref::() { - let mut tables = self.tables.write(); - return Ok(tables - .insert(Arc::from(name), Arc::from(table.clone())) - .map(|t| t as Arc)); - } - - Err(DataFusionError::Execution(format!( - "Couldn't cast table {:?} to SeafowlTable or DeltaTable", - table.schema(), - ))) + Ok(self.tables.write().insert(Arc::from(name), table)) } } diff --git a/tests/statements/query.rs b/tests/statements/query.rs index 610bcb12..3fe5b760 100644 --- a/tests/statements/query.rs +++ b/tests/statements/query.rs @@ -75,13 +75,13 @@ async fn test_create_table_and_insert() { let results = context.collect(plan).await.unwrap(); let expected = vec![ - "+-----------------+----------------+------------------+---------------------+------------+", - "| some_bool_value | some_int_value | some_other_value | some_time | some_value |", - "+-----------------+----------------+------------------+---------------------+------------+", - "| | 1111 | | 2022-01-01T20:01:01 | 42 |", - "| | 2222 | | 2022-01-01T20:02:02 | 43 |", - "| | 3333 | | 2022-01-01T20:03:03 | 44 |", - "+-----------------+----------------+------------------+---------------------+------------+", + "+---------------------+------------+------------------+-----------------+----------------+", + "| some_time | some_value | some_other_value | some_bool_value | some_int_value |", + "+---------------------+------------+------------------+-----------------+----------------+", + "| 2022-01-01T20:01:01 | 42 | | | 1111 |", + "| 2022-01-01T20:02:02 | 43 | | | 2222 |", + "| 2022-01-01T20:03:03 | 44 | | | 3333 |", + "+---------------------+------------+------------------+-----------------+----------------+", ]; assert_batches_eq!(expected, &results); From 9707fdf102fbc0635d55408d4fd810680a76ab73 Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Wed, 1 Mar 2023 11:02:10 +0100 Subject: [PATCH 03/21] Use root internal object_store instead of storage options for managing delta tables --- Cargo.lock | 18 +-- Cargo.toml | 4 + src/catalog.rs | 54 +++++--- src/config/context.rs | 74 ++-------- src/context.rs | 269 ++++++++++++++++++------------------ src/frontend/http.rs | 72 ++++++---- src/object_store/wrapped.rs | 173 +++++++++++++++++++++-- src/provider.rs | 15 +- src/repository/default.rs | 25 ++++ src/repository/interface.rs | 7 + tests/http/upload.rs | 2 +- tests/statements/ddl.rs | 62 ++++----- tests/statements/dml.rs | 20 +-- 13 files changed, 477 insertions(+), 318 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7947e96f..0d72bcea 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1684,8 +1684,6 @@ dependencies = [ [[package]] name = "datafusion" version = "18.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd805bdf93d3137b37fd9966042df0c84ddfca0df5a8d32eaacb16cf6ab0d93d" dependencies = [ "ahash 0.8.1", "arrow", @@ -1731,8 +1729,6 @@ dependencies = [ [[package]] name = "datafusion-common" version = "18.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08c58d6714427f52f9815d19debab7adab5bac5b4d2a99d51c250e606acb6cf5" dependencies = [ "arrow", "chrono", @@ -1745,8 +1741,6 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "18.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a32ee054230dd9a57d0bed587406869c4a7814d90154616aff2cb9991c1756f" dependencies = [ "ahash 0.8.1", "arrow", @@ -1758,8 +1752,6 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "18.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6de4d144924de29a835feeff8313a81fdc2c7190111301508e09ea59a80edbbc" dependencies = [ "arrow", "async-trait", @@ -1775,8 +1767,6 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "18.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "943e42356f0f6f5ac37ceacd412de9c4d7d8eba1e81b6f724f88699540c7f070" dependencies = [ "ahash 0.8.1", "arrow", @@ -1806,8 +1796,6 @@ dependencies = [ [[package]] name = "datafusion-proto" version = "18.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "649aec221737d8fb88956a2e6181297456f83b8560d32d775ad1cd22d67fd598" dependencies = [ "arrow", "chrono", @@ -1841,8 +1829,6 @@ dependencies = [ [[package]] name = "datafusion-row" version = "18.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a506f5924f8af54e0806a995da0897f8c2b548d492793e045a3896d88d6714a" dependencies = [ "arrow", "datafusion-common", @@ -1853,8 +1839,6 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "18.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a3d12047a5847f9667f4e2aa8fa2e7d5a6e1094b8e3546d58de492152a50dc7" dependencies = [ "arrow-schema", "datafusion-common", @@ -2037,7 +2021,7 @@ checksum = "1435fa1053d8b2fbbe9be7e97eca7f33d37b28409959813daefc1446a14247f1" [[package]] name = "dynamodb_lock" -version = "0.4.2" +version = "0.4.3" dependencies = [ "async-trait", "log", diff --git a/Cargo.toml b/Cargo.toml index 86c8b2e6..95a7b2b8 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -100,6 +100,10 @@ arrow-array = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow arrow-buffer = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-buffer", branch = "backport-cast-utf8-to-timestamp-microsecond" } arrow-schema = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-schema", branch = "backport-cast-utf8-to-timestamp-microsecond" } arrow-cast = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-cast", branch = "backport-cast-utf8-to-timestamp-microsecond" } +datafusion = { path = "../arrow-datafusion/datafusion/core" } +datafusion-common = { path = "../arrow-datafusion/datafusion/common" } +datafusion-expr = { path = "../arrow-datafusion/datafusion/expr" } +datafusion-proto = { path = "../arrow-datafusion/datafusion/proto" } [dev-dependencies] assert_unordered = "0.3" diff --git a/src/catalog.rs b/src/catalog.rs index f8098c21..209b78ea 100644 --- a/src/catalog.rs +++ b/src/catalog.rs @@ -5,12 +5,13 @@ use async_trait::async_trait; use datafusion::catalog::schema::MemorySchemaProvider; use datafusion::datasource::TableProvider; use datafusion::error::DataFusionError; -use deltalake::DeltaTableBuilder; +use deltalake::DeltaTable; use itertools::Itertools; #[cfg(test)] use mockall::automock; use parking_lot::RwLock; +use crate::object_store::wrapped::InternalObjectStore; use crate::provider::SeafowlFunction; use crate::repository::interface::TablePartitionsResult; use crate::system_tables::SystemSchemaProvider; @@ -175,6 +176,12 @@ pub trait TableCatalog: Sync + Send { database_name: &str, collection_name: &str, ) -> Result>; + async fn get_table_id_by_name( + &self, + database_name: &str, + collection_name: &str, + table_name: &str, + ) -> Result>; async fn create_database(&self, database_name: &str) -> Result; @@ -278,22 +285,19 @@ pub struct DefaultCatalog { // DataFusion's in-memory schema provider for staging external tables staging_schema: Arc, - storage_options: HashMap, - store_uri: String, + object_store: Arc, } impl DefaultCatalog { pub fn new( repository: Arc, - storage_options: HashMap, - store_uri: String, + object_store: Arc, ) -> Self { let staging_schema = Arc::new(MemorySchemaProvider::new()); Self { repository, staging_schema, - storage_options, - store_uri, + object_store, } } @@ -387,16 +391,11 @@ impl DefaultCatalog { |v| (&v.database_name, &v.collection_name), ); - let table_uri = format!( - "{}/{}/{}/{}", - self.store_uri, database_name, collection_name, table_name - ); - - let table = DeltaTableBuilder::from_uri(table_uri) - .with_storage_options(self.storage_options.clone()) - .build() - .unwrap(); + let table_object_store = + self.object_store + .for_delta_table(database_name, collection_name, table_name); + let table = DeltaTable::new(table_object_store, Default::default()); (Arc::from(table_name.to_string()), Arc::new(table) as _) } @@ -542,7 +541,7 @@ impl TableCatalog for DefaultCatalog { database_name: &str, collection_name: &str, ) -> Result> { - if database_name == DEFAULT_DB && collection_name == STAGING_SCHEMA { + if collection_name == STAGING_SCHEMA { return Err(Error::UsedStagingSchema); } @@ -568,6 +567,27 @@ impl TableCatalog for DefaultCatalog { } } + async fn get_table_id_by_name( + &self, + database_name: &str, + collection_name: &str, + table_name: &str, + ) -> Result> { + if collection_name == STAGING_SCHEMA { + return Err(Error::UsedStagingSchema); + } + + match self + .repository + .get_table_id_by_name(database_name, collection_name, table_name) + .await + { + Ok(id) => Ok(Some(id)), + Err(RepositoryError::SqlxError(sqlx::error::Error::RowNotFound)) => Ok(None), + Err(e) => Err(Self::to_sqlx_error(e)), + } + } + async fn create_database(&self, database_name: &str) -> Result { self.repository .create_database(database_name) diff --git a/src/config/context.rs b/src/config/context.rs index 0c899b2d..203e5c0e 100644 --- a/src/config/context.rs +++ b/src/config/context.rs @@ -27,15 +27,13 @@ use crate::object_store::wrapped::InternalObjectStore; use datafusion_remote_tables::factory::RemoteTableFactory; #[cfg(feature = "object-store-s3")] use object_store::aws::AmazonS3Builder; -use object_store::aws::AmazonS3ConfigKey; use parking_lot::lock_api::RwLock; use super::schema::{self, MEBIBYTES, MEMORY_FRACTION, S3}; async fn build_catalog( config: &schema::SeafowlConfig, - storage_options: HashMap, - store_uri: String, + object_store: Arc, ) -> ( Arc, Arc, @@ -69,7 +67,7 @@ async fn build_catalog( ), }; - let catalog = Arc::new(DefaultCatalog::new(repository, storage_options, store_uri)); + let catalog = Arc::new(DefaultCatalog::new(repository, object_store)); (catalog.clone(), catalog.clone(), catalog) } @@ -106,59 +104,6 @@ fn build_object_store(cfg: &schema::SeafowlConfig) -> Arc { } } -fn build_storage_options( - cfg: &schema::SeafowlConfig, -) -> (HashMap, String) { - match &cfg.object_store { - schema::ObjectStore::Local(schema::Local { data_dir }) => { - (HashMap::::default(), data_dir.clone()) - } - schema::ObjectStore::InMemory(_) => ( - HashMap::::default(), - "memory://".to_string(), - ), - #[cfg(feature = "object-store-s3")] - schema::ObjectStore::S3(S3 { - region, - access_key_id, - secret_access_key, - endpoint, - bucket, - }) => { - let mut storage_options = HashMap::::from([ - ( - AmazonS3ConfigKey::AccessKeyId.as_ref().to_string(), - access_key_id.clone(), - ), - ( - AmazonS3ConfigKey::SecretAccessKey.as_ref().to_string(), - secret_access_key.clone(), - ), - ( - AmazonS3ConfigKey::Bucket.as_ref().to_string(), - bucket.clone(), - ), - ]); - - if let Some(region) = region { - storage_options.insert( - AmazonS3ConfigKey::Region.as_ref().to_string(), - region.clone(), - ); - } - - if let Some(endpoint) = endpoint { - storage_options.insert( - AmazonS3ConfigKey::Endpoint.as_ref().to_string(), - endpoint.clone(), - ); - } - - (storage_options, format!("s3://{bucket}")) - } - } -} - #[allow(unused_mut)] pub async fn build_context( cfg: &schema::SeafowlConfig, @@ -203,13 +148,17 @@ pub async fn build_context( "", object_store.clone(), ); - let (storage_options, store_uri) = build_storage_options(cfg); + + let internal_object_store = Arc::new(InternalObjectStore::new( + object_store.clone(), + cfg.object_store.clone(), + )); // Register the HTTP object store for external tables add_http_object_store(&context, &cfg.misc.ssl_cert_file); let (tables, partitions, functions) = - build_catalog(cfg, storage_options.clone(), store_uri.clone()).await; + build_catalog(cfg, internal_object_store.clone()).await; // Create default DB/collection let default_db = match tables.get_database_id_by_name(DEFAULT_DB).await? { @@ -238,12 +187,7 @@ pub async fn build_context( table_catalog: tables, partition_catalog: partitions, function_catalog: functions, - internal_object_store: Arc::new(InternalObjectStore { - inner: object_store, - config: cfg.object_store.clone(), - }), - storage_options, - store_uri, + internal_object_store, database: DEFAULT_DB.to_string(), database_id: default_db, all_database_ids: Arc::from(RwLock::new(all_database_ids)), diff --git a/src/context.rs b/src/context.rs index 05739955..f9c37be5 100644 --- a/src/context.rs +++ b/src/context.rs @@ -94,7 +94,7 @@ use tempfile::TempPath; use tokio::sync::Semaphore; use crate::catalog::{PartitionCatalog, DEFAULT_SCHEMA, STAGING_SCHEMA}; -use crate::data_types::{PhysicalPartitionId, TableVersionId}; +use crate::data_types::PhysicalPartitionId; use crate::datafusion::visit::VisitorMut; use crate::provider::{ project_expressions, PartitionColumn, SeafowlPartition, SeafowlPruningStatistics, @@ -258,8 +258,6 @@ pub struct DefaultSeafowlContext { pub partition_catalog: Arc, pub function_catalog: Arc, pub internal_object_store: Arc, - pub storage_options: HashMap, - pub store_uri: String, pub database: String, pub database_id: DatabaseId, pub all_database_ids: Arc>>, @@ -519,13 +517,13 @@ pub fn is_statement_read_only(statement: &DFStatement) -> bool { } // The only reason to keep this trait around (instead of migrating all the functions directly into -// DefaultDeafowlContext), is that `create_physical_plan` would then be a recursive async function, +// DefaultSeafowlContext), is that `create_physical_plan` would then be a recursive async function, // which works for traits, but not for structs: https://stackoverflow.com/a/74737853 // // The workaround would be to box a future as the return of such functions, which isn't very // appealing atm (involves heap allocations, and is not very readable). // -// Alternatively, if we know that all recursive calls could be handled by the inner (DataFusion's) +// Alternatively, if we're sure that all recursive calls can be handled by the inner (DataFusion's) // `create_physical_plan` we could also rewrite the calls explicitly like that, and thus break the // recursion. #[cfg_attr(test, automock)] @@ -569,7 +567,7 @@ pub trait SeafowlContext: Send + Sync { plan: Arc, schema_name: String, table_name: String, - ) -> Result; + ) -> Result<()>; } impl DefaultSeafowlContext { @@ -601,8 +599,6 @@ impl DefaultSeafowlContext { partition_catalog: self.partition_catalog.clone(), function_catalog: self.function_catalog.clone(), internal_object_store: self.internal_object_store.clone(), - storage_options: self.storage_options.clone(), - store_uri: self.store_uri.clone(), database: name, database_id, all_database_ids: self.all_database_ids.clone(), @@ -610,33 +606,6 @@ impl DefaultSeafowlContext { })) } - /// Construct the full table uri in the underlying object store - pub fn table_to_location<'a>(&self, table: impl Into>) -> String { - let table_ref: TableReference = table.into(); - let resolved_ref = table_ref.resolve(&self.database, DEFAULT_SCHEMA); - - format!( - "{}/{}/{}/{}", - self.store_uri, resolved_ref.catalog, resolved_ref.schema, resolved_ref.table - ) - } - - /// Generate the Delta table builder and execute the write - pub async fn plan_to_delta_table( - &self, - plan: &Arc, - location: impl Into, - ) -> Result { - let table = WriteBuilder::new() - .with_input_execution_plan(plan.clone()) - .with_storage_options(self.storage_options.clone()) - .with_location(location) - .await?; - - debug!("Written table version {} for {table}", table.version()); - Ok(table) - } - pub fn inner(&self) -> &SessionContext { &self.inner } @@ -754,11 +723,15 @@ impl DefaultSeafowlContext { Ok(seafowl_table.clone()) } - async fn exec_create_table(&self, name: &str, schema: &Schema) -> Result { - let table_ref = TableReference::from(name); + async fn create_delta_table<'a>( + &self, + name: impl Into>, + schema: &Schema, + ) -> Result { + let table_ref: TableReference = name.into(); let resolved_ref = table_ref.resolve(&self.database, DEFAULT_SCHEMA); - let schema_name = resolved_ref.schema; - let table_name = resolved_ref.table; + let schema_name = resolved_ref.schema.clone(); + let table_name = resolved_ref.table.clone(); let sf_schema = SeafowlSchema { arrow_schema: Arc::new(schema.clone()), @@ -772,11 +745,14 @@ impl DefaultSeafowlContext { })?; let delta_schema = DeltaSchema::try_from(schema)?; - let location = self.table_to_location(name); + let table_object_store = self.internal_object_store.for_delta_table( + &resolved_ref.catalog, + &resolved_ref.schema, + &resolved_ref.table, + ); let table = CreateBuilder::new() - .with_storage_options(self.storage_options.clone()) - .with_location(&location) + .with_object_store(table_object_store) .with_table_name(&*table_name) .with_columns(delta_schema.get_fields().clone()) .with_comment(format!( @@ -792,7 +768,32 @@ impl DefaultSeafowlContext { .create_table(collection_id, &table_name, &sf_schema) .await?; - debug!("Created new table at location {location}"); + debug!("Created new table {table}"); + Ok(table) + } + + /// Generate the Delta table builder and execute the write + pub async fn plan_to_delta_table<'a>( + &self, + plan: &Arc, + name: impl Into>, + ) -> Result { + let table_ref: TableReference = name.into(); + let resolved_ref = table_ref.resolve(&self.database, DEFAULT_SCHEMA); + + let table_object_store = self.internal_object_store.for_delta_table( + &resolved_ref.catalog, + &resolved_ref.schema, + &resolved_ref.table, + ); + + let table = WriteBuilder::new() + .with_input_execution_plan(plan.clone()) + .with_input_session_state(self.inner.state()) + .with_object_store(table_object_store) + .await?; + + debug!("Written table version {} for {table}", table.version()); Ok(table) } @@ -845,38 +846,6 @@ impl DefaultSeafowlContext { physical_plan.execute(partition, task_context) } - // Execute the plan, repartition to Parquet files, upload them to object store and add metadata - // records for table/partitions. - async fn execute_plan_to_table( - &self, - physical_plan: &Arc, - output_schema: Option, - name: Option, - from_table_version: Option, - ) -> Result<()> { - // Create/Update table metadata - match (name, from_table_version) { - (Some(name), _) => { - let schema = output_schema.unwrap_or_else(|| physical_plan.schema()); - // Create an empty table with an empty version - self.exec_create_table(&name, schema.as_ref()).await?; - } - (_, Some(from_table_version)) => { - // Duplicate the table version into a new one - self.table_catalog - .create_new_table_version(from_table_version, true) - .await?; - } - _ => { - return Err(Error::Internal( - "Either name or source table version need to be supplied".to_string(), - )); - } - } - - Ok(()) - } - // Generate new physical Parquet partition files from the provided plan, upload to object store // and persist partition metadata. async fn execute_plan_to_partitions( @@ -1427,8 +1396,11 @@ impl SeafowlContext for DefaultSeafowlContext { // This is actually CREATE TABLE AS let physical = self.create_physical_plan(input).await?; - self.execute_plan_to_table(&physical, None, Some(name.to_string()), None) + // First create the table and then insert the data from the subqeury + let _table = self + .create_delta_table(name, physical.schema().as_ref()) .await?; + self.plan_to_delta_table(&physical, name).await?; Ok(make_dummy_exec()) } @@ -1440,8 +1412,7 @@ impl SeafowlContext for DefaultSeafowlContext { }) => { let physical = self.create_physical_plan(input).await?; - self.plan_to_delta_table(&physical, self.table_to_location(table_name)) - .await?; + self.plan_to_delta_table(&physical, table_name).await?; Ok(make_dummy_exec()) } @@ -1685,8 +1656,30 @@ impl SeafowlContext for DefaultSeafowlContext { schema: _, }) => { // DROP TABLE - let table = self.try_get_seafowl_table(name.to_string()).await?; - self.table_catalog.drop_table(table.table_id).await?; + if let Ok(table) = self.try_get_seafowl_table(name.to_string()).await { + // Drop for legacy tables + self.table_catalog.drop_table(table.table_id).await?; + return Ok(make_dummy_exec()); + }; + + let table_ref = TableReference::from(name); + let resolved_ref = table_ref.resolve(&self.database, DEFAULT_SCHEMA); + + let table_id = self + .table_catalog + .get_table_id_by_name( + &resolved_ref.catalog, + &resolved_ref.schema, + &resolved_ref.table, + ) + .await? + .ok_or_else(|| { + DataFusionError::Execution("Table {name} not found".to_string()) + })?; + + self.table_catalog.drop_table(table_id).await?; + // TODO: delay the actual delete and perform it during the subsequent vacuuming + // somehow, or when a new CREATE TABLE targets the same uri simply owerwrite Ok(make_dummy_exec()) } LogicalPlan::CreateView(_) => { @@ -1703,7 +1696,7 @@ impl SeafowlContext for DefaultSeafowlContext { name, .. }) => { - self.exec_create_table(name, schema).await?; + self.create_delta_table(name.as_str(), schema).await?; Ok(make_dummy_exec()) } @@ -1828,55 +1821,67 @@ impl SeafowlContext for DefaultSeafowlContext { plan: Arc, schema_name: String, table_name: String, - ) -> Result { + ) -> Result<()> { // Reload the schema since `try_get_seafowl_table` relies on using DataFusion's // TableProvider interface (which we need to pre-populate with up to date // information on our tables) self.reload_schema().await?; - // Ensure the schema exists prior to creating the table - let (full_table_name, from_table_version) = { - let new_table_name = format!("{schema_name}.{table_name}"); + // Check whether table already exists and ensure that the schema exists + let table_exists = match self + .inner + .catalog(&self.database) + .ok_or_else(|| Error::Plan(format!("Database {} not found!", self.database)))? + .schema(&schema_name) + { + Some(_) => { + if self + .try_get_seafowl_table(format!("{schema_name}.{table_name}")) + .await + .is_ok() + { + return Err(DataFusionError::Execution("Cannot insert into legacy table {table_name}, please use a different name".to_string())); + } - match self - .table_catalog - .get_collection_id_by_name(&self.database, &schema_name) - .await? - { - Some(_) => { - if let Ok(table) = self.try_get_seafowl_table(&new_table_name).await { - // Table exists, see if the schemas match - if table.schema.arrow_schema != plan.schema() { + // Schema exists, check if existing table's schema matches the new one + match self.get_table_provider(&table_name).await { + Ok(table) => { + if table.schema() != plan.schema() { return Err(DataFusionError::Execution( - format!( - "The table {new_table_name} already exists but has a different schema than the one provided.") + format!( + "The table {table_name} already exists but has a different schema than the one provided.") ) - ); + ); } - // Instead of creating a new table, just insert the data into a new version - // of an existing table - (None, Some(table.table_version_id)) - } else { - // Table doesn't exist or isn't a Seafowl table - // We assume it doesn't exist for now - (Some(new_table_name), None) + true } - } - None => { - self.table_catalog - .create_collection(self.database_id, &schema_name) - .await?; - - (Some(new_table_name), None) + Err(_) => false, } } + None => { + // Schema doesn't exist; create one first + self.table_catalog + .create_collection(self.database_id, &schema_name) + .await?; + false + } }; - self.execute_plan_to_table(&plan, None, full_table_name, from_table_version) - .await?; + let table_ref = TableReference::Full { + catalog: Cow::from(self.database.clone()), + schema: Cow::from(schema_name), + table: Cow::from(table_name), + }; + + if !table_exists { + self.create_delta_table(table_ref.clone(), plan.schema().as_ref()) + .await?; + } + + self.plan_to_delta_table(&plan, table_ref).await?; - Ok(true) + Ok(()) } } @@ -2036,10 +2041,10 @@ pub mod test_utils { setup_table_catalog(&mut table_catalog); - let object_store = Arc::new(InternalObjectStore { - inner: Arc::new(InMemory::new()), - config: schema::ObjectStore::InMemory(schema::InMemory {}), - }); + let object_store = Arc::new(InternalObjectStore::new( + Arc::new(InMemory::new()), + schema::ObjectStore::InMemory(schema::InMemory {}), + )); session.runtime_env().register_object_store( INTERNAL_OBJECT_STORE_SCHEME, "", @@ -2052,8 +2057,6 @@ pub mod test_utils { partition_catalog: partition_catalog_ptr, function_catalog: Arc::new(function_catalog), internal_object_store: object_store, - storage_options: Default::default(), - store_uri: "/".to_string(), database: "testdb".to_string(), database_id: 0, all_database_ids: Arc::from(RwLock::new(HashMap::from([( @@ -2147,22 +2150,20 @@ mod tests { let tmp_dir = TempDir::new().unwrap(); ( - Arc::new(InternalObjectStore { - inner: Arc::new( - LocalFileSystem::new_with_prefix(tmp_dir.path()).unwrap(), - ), - config: schema::ObjectStore::Local(schema::Local { + Arc::new(InternalObjectStore::new( + Arc::new(LocalFileSystem::new_with_prefix(tmp_dir.path()).unwrap()), + schema::ObjectStore::Local(schema::Local { data_dir: tmp_dir.path().to_string_lossy().to_string(), }), - }), + )), Some(tmp_dir), ) } else { ( - Arc::new(InternalObjectStore { - inner: Arc::new(InMemory::new()), - config: schema::ObjectStore::InMemory(schema::InMemory {}), - }), + Arc::new(InternalObjectStore::new( + Arc::new(InMemory::new()), + schema::ObjectStore::InMemory(schema::InMemory {}), + )), None, ) }; @@ -2318,10 +2319,10 @@ mod tests { MemoryExec::try_new(df_partitions.as_slice(), schema, None).unwrap(), ); - let object_store = Arc::new(InternalObjectStore { - inner: Arc::new(InMemory::new()), - config: schema::ObjectStore::InMemory(schema::InMemory {}), - }); + let object_store = Arc::new(InternalObjectStore::new( + Arc::new(InMemory::new()), + schema::ObjectStore::InMemory(schema::InMemory {}), + )); let disk_manager = DiskManager::try_new(DiskManagerConfig::new()).unwrap(); let partitions = plan_to_object_store( &sf_context.inner.state(), diff --git a/src/frontend/http.rs b/src/frontend/http.rs index 73fc7cc7..e1fb63ff 100644 --- a/src/frontend/http.rs +++ b/src/frontend/http.rs @@ -17,6 +17,8 @@ use datafusion::datasource::DefaultTableSource; use datafusion::parquet::arrow::arrow_reader::ParquetRecordBatchReader; use datafusion::physical_plan::memory::MemoryExec; use datafusion_expr::logical_plan::{LogicalPlan, PlanVisitor, TableScan}; +use deltalake::parquet::data_type::AsBytes; +use deltalake::DeltaTable; use futures::{future, TryStreamExt}; use hex::encode; use log::{debug, info}; @@ -36,8 +38,6 @@ use crate::{ context::{ is_read_only, is_statement_read_only, DefaultSeafowlContext, SeafowlContext, }, - data_types::TableVersionId, - provider::SeafowlTable, }; use super::http_utils::{handle_rejection, into_response, ApiError}; @@ -54,7 +54,7 @@ const VARY: &str = "Content-Type, Origin, X-Seafowl-Query"; #[derive(Default)] struct ETagBuilderVisitor { - table_versions: Vec, + table_versions: Vec, } impl PlanVisitor for ETagBuilderVisitor { @@ -69,9 +69,14 @@ impl PlanVisitor for ETagBuilderVisitor { if let Some(table) = default_table_source .table_provider .as_any() - .downcast_ref::() + .downcast_ref::() { - self.table_versions.push(table.table_version_id) + println!("uri: {}", table.table_uri()); + println!("version: {}", table.version()); + self.table_versions + .extend(table.table_uri().as_bytes().to_vec()); + self.table_versions + .extend(table.version().as_bytes().to_vec()); } } } @@ -682,10 +687,14 @@ mod tests { const CREATE_QUERY: &str = "CREATE TABLE other_test_table(col_1 INT)"; const SELECT_QUERY_HASH: &str = "7fbbf7dddfd330d03e5e08cc5885ad8ca823e1b56e7cbadd156daa0e21c288f6"; - const V1_ETAG: &str = - "038966de9f6b9a901b20b4c6ca8b2a46009feebe031babc842d43690c0bc222b"; - const V2_ETAG: &str = - "06d033ece6645de592db973644cf7357255f24536ff7b03c3b2ace10736f7636"; + const V1_ETAG_DEFAULT_DB: &str = + "c2d5a6434eb700587e0e95a5d22015e2616873d22353e83ce436075f5e01e740"; + const V1_ETAG_TEST_DB: &str = + "b044e9f114a6627dd2449496514e938234dc1ff6e400cb177fe38b3ed977619b"; + const V2_ETAG_DEFAULT_DB: &str = + "780c8dc5587a5a8cd32a4fabcdf1425e3c6f6c859cd24b7963a3007f450ce55e"; + const V2_ETAG_TEST_DB: &str = + "26e1be6caca4a0fefb798b7d197033f6f181719f59042e87d95754bcf2ebdea0"; #[rstest] #[tokio::test] @@ -759,10 +768,10 @@ mod tests { } #[rstest] + #[case(None, V1_ETAG_DEFAULT_DB)] + #[case(Some("test_db"), V1_ETAG_TEST_DB)] #[tokio::test] - async fn test_get_cached_no_etag( - #[values(None, Some("test_db"))] new_db: Option<&str>, - ) { + async fn test_get_cached_no_etag(#[case] new_db: Option<&str>, #[case] etag: &str) { let context = in_memory_context_with_single_table(new_db).await; let handler = filters(context, http_config_from_access_policy(free_for_all())); @@ -776,7 +785,7 @@ mod tests { assert_eq!(resp.body(), "{\"c\":1}\n"); assert_eq!( resp.headers().get(header::ETAG).unwrap().to_str().unwrap(), - V1_ETAG + etag, ); } @@ -800,9 +809,12 @@ mod tests { } #[rstest] + #[case(None, V1_ETAG_DEFAULT_DB)] + #[case(Some("test_db"), V1_ETAG_TEST_DB)] #[tokio::test] async fn test_get_cached_no_etag_query_in_body( - #[values(None, Some("test_db"))] new_db: Option<&str>, + #[case] new_db: Option<&str>, + #[case] etag: &str, ) { let context = in_memory_context_with_single_table(new_db).await; let handler = filters(context, http_config_from_access_policy(free_for_all())); @@ -817,14 +829,17 @@ mod tests { assert_eq!(resp.body(), "{\"c\":1}\n"); assert_eq!( resp.headers().get(header::ETAG).unwrap().to_str().unwrap(), - V1_ETAG + etag ); } #[rstest] + #[case(None, V1_ETAG_DEFAULT_DB)] + #[case(Some("test_db"), V1_ETAG_TEST_DB)] #[tokio::test] async fn test_get_cached_no_etag_extension( - #[values(None, Some("test_db"))] new_db: Option<&str>, + #[case] new_db: Option<&str>, + #[case] etag: &str, ) { let context = in_memory_context_with_single_table(new_db).await; let handler = filters(context, http_config_from_access_policy(free_for_all())); @@ -839,14 +854,17 @@ mod tests { assert_eq!(resp.body(), "{\"c\":1}\n"); assert_eq!( resp.headers().get(header::ETAG).unwrap().to_str().unwrap(), - V1_ETAG + etag ); } #[rstest] + #[case(None, V1_ETAG_DEFAULT_DB)] + #[case(Some("test_db"), V1_ETAG_TEST_DB)] #[tokio::test] async fn test_get_cached_reuse_etag( - #[values(None, Some("test_db"))] new_db: Option<&str>, + #[case] new_db: Option<&str>, + #[case] etag: &str, ) { // Pass the same ETag as If-None-Match, should return a 301 @@ -857,7 +875,7 @@ mod tests { .method("GET") .path(make_uri(format!("/q/{SELECT_QUERY_HASH}"), new_db).as_str()) .header(QUERY_HEADER, SELECT_QUERY) - .header(IF_NONE_MATCH, V1_ETAG) + .header(IF_NONE_MATCH, etag) .reply(&handler) .await; assert_eq!(resp.status(), StatusCode::NOT_MODIFIED); @@ -865,9 +883,12 @@ mod tests { } #[rstest] + #[case(None, V1_ETAG_DEFAULT_DB)] + #[case(Some("test_db"), V1_ETAG_TEST_DB)] #[tokio::test] async fn test_get_encoded_query_special_chars( - #[values(None, Some("test_db"))] new_db: Option<&str>, + #[case] new_db: Option<&str>, + #[case] etag: &str, ) { let context = in_memory_context_with_single_table(new_db).await; let handler = filters(context, http_config_from_access_policy(free_for_all())); @@ -882,7 +903,7 @@ mod tests { assert_eq!(resp.body(), "{\"c\":1}\n"); assert_eq!( resp.headers().get(header::ETAG).unwrap().to_str().unwrap(), - V1_ETAG + etag ); } @@ -905,9 +926,12 @@ mod tests { } #[rstest] + #[case(None, V2_ETAG_DEFAULT_DB)] + #[case(Some("test_db"), V2_ETAG_TEST_DB)] #[tokio::test] async fn test_get_cached_etag_new_version( - #[values(None, Some("test_db"))] new_db: Option<&str>, + #[case] new_db: Option<&str>, + #[case] etag: &str, ) { // Pass the same ETag as If-None-Match, but the table version changed -> reruns the query @@ -918,14 +942,14 @@ mod tests { .method("GET") .path(make_uri(format!("/q/{SELECT_QUERY_HASH}"), new_db).as_str()) .header(QUERY_HEADER, SELECT_QUERY) - .header(header::ETAG, V1_ETAG) + .header(header::ETAG, V1_ETAG_DEFAULT_DB) .reply(&handler) .await; assert_eq!(resp.status(), StatusCode::OK); assert_eq!(resp.body(), "{\"c\":2}\n"); assert_eq!( resp.headers().get(header::ETAG).unwrap().to_str().unwrap(), - V2_ETAG + etag ); } diff --git a/src/object_store/wrapped.rs b/src/object_store/wrapped.rs index bf0c946f..5d9897d3 100644 --- a/src/object_store/wrapped.rs +++ b/src/object_store/wrapped.rs @@ -1,37 +1,98 @@ use crate::config::schema; -use crate::config::schema::Local; -use futures::TryFutureExt; +use crate::config::schema::{Local, S3}; +use bytes::Bytes; +use futures::{stream::BoxStream, TryFutureExt}; use log::debug; -use object_store::ObjectStore; +use object_store::{ + path::Path, Error, GetResult, ListResult, MultipartId, ObjectMeta, ObjectStore, + Result, +}; +use std::fmt::{Debug, Display, Formatter}; +use std::ops::Range; +use tokio::io::AsyncWrite; use tokio::fs::{copy, remove_file, rename}; -use std::path::Path; +use deltalake::storage::DeltaObjectStore; +use object_store::prefix::PrefixObjectStore; +use std::path::Path as StdPath; +use std::str::FromStr; use std::sync::Arc; +use url::Url; /// Wrapper around the object_store crate that holds on to the original config /// in order to provide a more efficient "upload" for the local object store (since it's /// stored on the local filesystem, we can just move the file to it instead). +#[derive(Debug, Clone)] pub struct InternalObjectStore { pub inner: Arc, + pub root_uri: Url, pub config: schema::ObjectStore, } impl InternalObjectStore { + pub fn new(inner: Arc, config: schema::ObjectStore) -> Self { + let root_uri = match config.clone() { + schema::ObjectStore::Local(Local { data_dir }) => { + let canonical_path = StdPath::new(&data_dir).canonicalize().unwrap(); + Url::from_directory_path(canonical_path).unwrap() + } + schema::ObjectStore::InMemory(_) => Url::from_str("memory://").unwrap(), + schema::ObjectStore::S3(S3 { bucket, .. }) => { + Url::from_str(&format!("s3://{bucket}")).unwrap() + } + }; + + Self { + inner, + root_uri, + config, + } + } + + // Wrap our object store with a prefixed one corresponding to the full path to the actual table + // root, and then wrap that with a delta object store. This is done because: + // 1. `DeltaObjectStore` needs an object store with "/" pointing at delta table root + // (i.e. where `_delta_log` is located), hence the `PrefixObjectStore`. + // 2. We want to re-use the underlying object store that we build initially during startup, + // instead of re-building one from scratch whenever we need it (not necessarily for perf + // reasons, but rather because the memory object store doesn't work otherwise). However, + // `PrefixObjectStore` has a trait bound of `T: ObjectStore`, which isn't satisfied by + // `Arc`, so we need another intermediary, which is where + // `InternalObjectStore` comes in. + // This does mean that we have 3 layers of indirection before we hit the "real" object store + // (`DeltaObjectStore` -> `PrefixObjectStore` -> `InternalObjectStore` -> `inner`). + pub fn for_delta_table( + &self, + database_name: &str, + collection_name: &str, + table_name: &str, + ) -> Arc { + let prefix = format!("{}/{}/{}", database_name, collection_name, table_name); + let prefixed_store: PrefixObjectStore = + PrefixObjectStore::new(self.clone(), prefix.clone()); + + Arc::from(DeltaObjectStore::new( + Arc::from(prefixed_store), + Url::from_str(format!("{}/{}", self.root_uri.as_str(), prefix).as_str()) + .unwrap(), + )) + } + /// For local filesystem object stores, try "uploading" by just moving the file. /// Returns a None if the store isn't local. pub async fn fast_upload( &self, - from: &Path, - to: &object_store::path::Path, - ) -> Option> { + from: &StdPath, + to: &Path, + ) -> Option> { let object_store_path = match &self.config { schema::ObjectStore::Local(Local { data_dir }) => data_dir, _ => return None, }; let target_path = - Path::new(&object_store_path).join(Path::new(to.to_string().as_str())); + StdPath::new(&object_store_path).join(StdPath::new(to.to_string().as_str())); debug!( "Moving temporary partition file from {} to {}", @@ -47,13 +108,13 @@ impl InternalObjectStore { if e.raw_os_error() == Some(18) { copy(from, target_path) .and_then(|_| remove_file(from)) - .map_err(|e| object_store::Error::Generic { + .map_err(|e| Error::Generic { store: "local", source: Box::new(e), }) .await } else { - Err(object_store::Error::Generic { + Err(Error::Generic { store: "local", source: Box::new(e), }) @@ -63,3 +124,95 @@ impl InternalObjectStore { }) } } + +impl Display for InternalObjectStore { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "FileStorageBackend({})", self.root_uri) + } +} + +#[async_trait::async_trait] +impl ObjectStore for InternalObjectStore { + /// Save the provided bytes to the specified location. + async fn put(&self, location: &Path, bytes: Bytes) -> Result<()> { + self.inner.put(location, bytes).await + } + + async fn put_multipart( + &self, + location: &Path, + ) -> Result<(MultipartId, Box)> { + self.inner.put_multipart(location).await + } + + async fn abort_multipart( + &self, + location: &Path, + multipart_id: &MultipartId, + ) -> Result<()> { + self.inner.abort_multipart(location, multipart_id).await + } + + /// Return the bytes that are stored at the specified location. + async fn get(&self, location: &Path) -> Result { + self.inner.get(location).await + } + + /// Return the bytes that are stored at the specified location + /// in the given byte range + async fn get_range(&self, location: &Path, range: Range) -> Result { + self.inner.get_range(location, range).await + } + + /// Return the metadata for the specified location + async fn head(&self, location: &Path) -> Result { + self.inner.head(location).await + } + + /// Delete the object at the specified location. + async fn delete(&self, location: &Path) -> Result<()> { + self.inner.delete(location).await + } + + /// List all the objects with the given prefix. + /// + /// Prefixes are evaluated on a path segment basis, i.e. `foo/bar/` is a prefix of `foo/bar/x` but not of + /// `foo/bar_baz/x`. + async fn list( + &self, + prefix: Option<&Path>, + ) -> Result>> { + self.inner.list(prefix).await + } + + /// List objects with the given prefix and an implementation specific + /// delimiter. Returns common prefixes (directories) in addition to object + /// metadata. + /// + /// Prefixes are evaluated on a path segment basis, i.e. `foo/bar/` is a prefix of `foo/bar/x` but not of + /// `foo/bar_baz/x`. + async fn list_with_delimiter(&self, prefix: Option<&Path>) -> Result { + self.inner.list_with_delimiter(prefix).await + } + + /// Copy an object from one path to another in the same object store. + /// + /// If there exists an object at the destination, it will be overwritten. + async fn copy(&self, from: &Path, to: &Path) -> Result<()> { + self.inner.copy(from, to).await + } + + /// Copy an object from one path to another, only if destination is empty. + /// + /// Will return an error if the destination already has an object. + async fn copy_if_not_exists(&self, from: &Path, to: &Path) -> Result<()> { + self.inner.copy_if_not_exists(from, to).await + } + + /// Move an object from one path to another in the same object store. + /// + /// Will return an error if the destination already has an object. + async fn rename_if_not_exists(&self, from: &Path, to: &Path) -> Result<()> { + self.inner.rename_if_not_exists(from, to).await + } +} diff --git a/src/provider.rs b/src/provider.rs index b577cb23..7bbaaa9e 100644 --- a/src/provider.rs +++ b/src/provider.rs @@ -39,7 +39,7 @@ use datafusion::{ use datafusion_common::DFSchema; use datafusion_expr::Expr; use datafusion_proto::protobuf; -use deltalake::{DeltaTable, DeltaTableBuilder}; +use deltalake::DeltaTable; use futures::future; use log::warn; @@ -142,7 +142,7 @@ impl SchemaProvider for SeafowlCollection { // Ultimately though, since the map gets re-created for each query the only point in // updating the existing table is to optimize potential multi-lookups during processing of // a single query. - let (table_uri, object_store) = match self.tables.read().get(name) { + let table_object_store = match self.tables.read().get(name) { None => return None, Some(table) => match table.as_any().downcast_ref::() { // This shouldn't happen since we stsore only DeltaTable's in the map @@ -154,23 +154,20 @@ impl SchemaProvider for SeafowlCollection { } else { // A negative table version indicates that the table was never loaded; we need // to do it before returning it. - (delta_table.table_uri(), delta_table.object_store()) + delta_table.object_store() } } }, }; - let maybe_table = DeltaTableBuilder::from_uri(table_uri.clone()) - .with_storage_backend(object_store, table_uri.parse().unwrap()) - .load() - .await; + let mut delta_table = DeltaTable::new(table_object_store, Default::default()); - if let Err(err) = maybe_table { + if let Err(err) = delta_table.load().await { warn!("Failed to load table {name}: {err}"); return None; } - let table = Arc::from(maybe_table.unwrap()) as Arc; + let table = Arc::from(delta_table) as Arc; self.tables.write().insert(Arc::from(name), table.clone()); Some(table) } diff --git a/src/repository/default.rs b/src/repository/default.rs index 5b79c4f5..a2f10fb9 100644 --- a/src/repository/default.rs +++ b/src/repository/default.rs @@ -210,6 +210,31 @@ impl Repository for $repo { Ok(id) } + async fn get_table_id_by_name( + &self, + database_name: &str, + collection_name: &str, + table_name: &str, + ) -> Result { + let id = sqlx::query( + r#" + SELECT "table".id + FROM "table" + JOIN collection ON "table".collection_id = collection.id + JOIN database ON collection.database_id = database.id + WHERE database.name = $1 AND collection.name = $2 AND "table".name = $3 + "#, + ) + .bind(database_name) + .bind(collection_name) + .bind(table_name) + .fetch_one(&self.executor) + .await.map_err($repo::interpret_error)? + .try_get("id").map_err($repo::interpret_error)?; + + Ok(id) + } + async fn get_all_database_ids(&self) -> Result> { let all_db_ids = sqlx::query(r#"SELECT name, id FROM database"#) .fetch_all(&self.executor) diff --git a/src/repository/interface.rs b/src/repository/interface.rs index f68791c7..a7c09adc 100644 --- a/src/repository/interface.rs +++ b/src/repository/interface.rs @@ -111,6 +111,13 @@ pub trait Repository: Send + Sync + Debug { database_name: &str, ) -> Result; + async fn get_table_id_by_name( + &self, + database_name: &str, + collection_name: &str, + table_name: &str, + ) -> Result; + async fn get_all_database_ids(&self) -> Result, Error>; async fn create_database(&self, database_name: &str) -> Result; diff --git a/tests/http/upload.rs b/tests/http/upload.rs index a048f0a5..0fe6b1be 100644 --- a/tests/http/upload.rs +++ b/tests/http/upload.rs @@ -275,7 +275,7 @@ async fn test_upload_to_existing_table() { .unwrap(); assert_eq!( - "Execution error: The table public.test_table already exists but has a different schema than the one provided.".to_string(), + "Execution error: The table test_table already exists but has a different schema than the one provided.".to_string(), String::from_utf8(output.stdout).unwrap() ); diff --git a/tests/statements/ddl.rs b/tests/statements/ddl.rs index 751df3de..ef9c0821 100644 --- a/tests/statements/ddl.rs +++ b/tests/statements/ddl.rs @@ -21,15 +21,15 @@ async fn test_create_table() { let results = list_columns_query(&context).await; let expected = vec![ - "+--------------+------------+------------------+-----------------------------+", - "| table_schema | table_name | column_name | data_type |", - "+--------------+------------+------------------+-----------------------------+", - "| public | test_table | some_bool_value | Boolean |", - "| public | test_table | some_int_value | Int64 |", - "| public | test_table | some_other_value | Decimal128(38, 10) |", - "| public | test_table | some_time | Timestamp(Nanosecond, None) |", - "| public | test_table | some_value | Float32 |", - "+--------------+------------+------------------+-----------------------------+", + "+--------------+------------+------------------+------------------------------+", + "| table_schema | table_name | column_name | data_type |", + "+--------------+------------+------------------+------------------------------+", + "| public | test_table | some_time | Timestamp(Microsecond, None) |", + "| public | test_table | some_value | Float32 |", + "| public | test_table | some_other_value | Decimal128(38, 10) |", + "| public | test_table | some_bool_value | Boolean |", + "| public | test_table | some_int_value | Int64 |", + "+--------------+------------+------------------+------------------------------+", ]; assert_batches_eq!(expected, &results); @@ -63,13 +63,13 @@ async fn test_create_table_as() { let results = context.collect(plan).await.unwrap(); let expected = vec![ - "+----------------+-------------+------------+", - "| some_int_value | some_minute | some_value |", - "+----------------+-------------+------------+", - "| 3333 | 3 | 49 |", - "| 2222 | 2 | 48 |", - "| 1111 | 1 | 47 |", - "+----------------+-------------+------------+", + "+------------+----------------+-------------+", + "| some_value | some_int_value | some_minute |", + "+------------+----------------+-------------+", + "| 49 | 3333 | 3 |", + "| 48 | 2222 | 2 |", + "| 47 | 1111 | 1 |", + "+------------+----------------+-------------+", ]; assert_batches_eq!(expected, &results); } @@ -87,20 +87,20 @@ async fn test_create_table_move_and_drop() { let results = list_columns_query(&context).await; let expected = vec![ - "+--------------+--------------+------------------+-----------------------------+", - "| table_schema | table_name | column_name | data_type |", - "+--------------+--------------+------------------+-----------------------------+", - "| public | test_table_1 | some_bool_value | Boolean |", - "| public | test_table_1 | some_int_value | Int64 |", - "| public | test_table_1 | some_other_value | Decimal128(38, 10) |", - "| public | test_table_1 | some_time | Timestamp(Nanosecond, None) |", - "| public | test_table_1 | some_value | Float32 |", - "| public | test_table_2 | some_bool_value | Boolean |", - "| public | test_table_2 | some_int_value | Int64 |", - "| public | test_table_2 | some_other_value | Decimal128(38, 10) |", - "| public | test_table_2 | some_time | Timestamp(Nanosecond, None) |", - "| public | test_table_2 | some_value | Float32 |", - "+--------------+--------------+------------------+-----------------------------+", + "+--------------+--------------+------------------+------------------------------+", + "| table_schema | table_name | column_name | data_type |", + "+--------------+--------------+------------------+------------------------------+", + "| public | test_table_1 | some_time | Timestamp(Microsecond, None) |", + "| public | test_table_1 | some_value | Float32 |", + "| public | test_table_1 | some_other_value | Decimal128(38, 10) |", + "| public | test_table_1 | some_bool_value | Boolean |", + "| public | test_table_1 | some_int_value | Int64 |", + "| public | test_table_2 | some_time | Timestamp(Microsecond, None) |", + "| public | test_table_2 | some_value | Float32 |", + "| public | test_table_2 | some_other_value | Decimal128(38, 10) |", + "| public | test_table_2 | some_bool_value | Boolean |", + "| public | test_table_2 | some_int_value | Int64 |", + "+--------------+--------------+------------------+------------------------------+", ]; assert_batches_eq!(expected, &results); @@ -354,7 +354,7 @@ async fn test_create_table_schema_already_exists() { .unwrap_err(); assert_eq!( err.to_string(), - "Error during planning: Table \"some_table\" already exists" + "External error: Generic error: A Delta Lake table already exists at that location." ); let err = context diff --git a/tests/statements/dml.rs b/tests/statements/dml.rs index 7b58a008..294ae555 100644 --- a/tests/statements/dml.rs +++ b/tests/statements/dml.rs @@ -23,16 +23,16 @@ async fn test_insert_two_different_schemas() { let results = context.collect(plan).await.unwrap(); let expected = vec![ - "+-----------------+----------------+------------------+---------------------+------------+", - "| some_bool_value | some_int_value | some_other_value | some_time | some_value |", - "+-----------------+----------------+------------------+---------------------+------------+", - "| | 1111 | | 2022-01-01T20:01:01 | 42 |", - "| | 2222 | | 2022-01-01T20:02:02 | 43 |", - "| | 3333 | | 2022-01-01T20:03:03 | 44 |", - "| false | | 2.1500000000 | | 41 |", - "| true | | 9.1200000000 | | 45 |", - "| false | | 44.3400000000 | | |", - "+-----------------+----------------+------------------+---------------------+------------+", + "+---------------------+------------+------------------+-----------------+----------------+", + "| some_time | some_value | some_other_value | some_bool_value | some_int_value |", + "+---------------------+------------+------------------+-----------------+----------------+", + "| 2022-01-01T20:01:01 | 42 | | | 1111 |", + "| 2022-01-01T20:02:02 | 43 | | | 2222 |", + "| 2022-01-01T20:03:03 | 44 | | | 3333 |", + "| | 41 | 2.1500000000 | false | |", + "| | 45 | 9.1200000000 | true | |", + "| | | 44.3400000000 | false | |", + "+---------------------+------------+------------------+-----------------+----------------+", ]; assert_batches_eq!(expected, &results); } From 97de330e56a9adcc04bc23078b993b14a462464f Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Wed, 1 Mar 2023 22:00:59 +0100 Subject: [PATCH 04/21] Enable time-travel for Delta tables --- src/context.rs | 40 +++--- src/repository/interface.rs | 1 + src/repository/postgres.rs | 1 + src/repository/sqlite.rs | 1 + src/version.rs | 24 ++-- tests/statements/ddl.rs | 14 +-- tests/statements/dml.rs | 12 +- tests/statements/function.rs | 6 +- tests/statements/mod.rs | 57 ++++++--- tests/statements/query.rs | 232 ++++++++++++++++++----------------- tests/statements/vacuum.rs | 4 +- 11 files changed, 216 insertions(+), 176 deletions(-) diff --git a/src/context.rs b/src/context.rs index f9c37be5..8155ec8d 100644 --- a/src/context.rs +++ b/src/context.rs @@ -43,6 +43,7 @@ use sqlparser::ast::{ use arrow_integration_test::field_to_json; use arrow_schema::DataType; +use chrono::{DateTime, FixedOffset, Utc}; use std::iter::zip; use std::str::FromStr; use std::sync::Arc; @@ -1066,29 +1067,40 @@ impl SeafowlContext for DefaultSeafowlContext { // We now have table_version_ids for each table with version specified; do another // run over the query AST to rewrite the table. version_processor.visit_query(&mut q); - debug!("Time travel query rewritten to: {}", q); + println!("Time travel query rewritten to: {}", q); let tables_by_version = self .table_catalog .load_tables_by_version(self.database_id, Some(version_processor.table_version_ids())).await?; for ((table, version), table_version_id) in &version_processor.table_versions { - if let Some(table_version_id) = table_version_id { - let name_with_version = - version_processor.table_with_version(table, version); + let name_with_version = + version_processor.table_with_version(table, version); - let full_table_name = table.to_string(); - let table_ref = match TableReference::from(full_table_name.as_str()) { - TableReference::Bare {.. } => TableReference::Bare{ table: Cow::Borrowed(name_with_version.as_str()) }, - TableReference::Partial { schema, .. } => TableReference::Partial { schema, table: Cow::Borrowed(name_with_version.as_str()) }, - TableReference::Full { catalog, schema, .. } => TableReference::Full { catalog, schema, table: Cow::Borrowed(name_with_version.as_str()) } - }; + let full_table_name = table.to_string(); + let mut resolved_ref = TableReference::from(full_table_name.as_str()).resolve(&self.database, DEFAULT_SCHEMA); - let table_provider = tables_by_version[table_version_id].clone(); + let table_provider_for_version: Arc = if let Some(table_version_id) = table_version_id { + // Legacy tables + tables_by_version[table_version_id].clone() + } else { + // We only support datetime DeltaTable version specification for start + let table_object_store = + self.internal_object_store.for_delta_table(&resolved_ref.catalog, &resolved_ref.schema, &resolved_ref.table); + let datetime = DateTime::::from(DateTime::::parse_from_rfc3339(version).map_err(|_| DataFusionError::Execution(format!( + "Failed to parse version {version} as RFC3339 timestamp" + )))?); + + // This won't work with `InMemory` object store for now: https://github.com/apache/arrow-rs/issues/3782 + let mut delta_table = DeltaTable::new(table_object_store, Default::default()); + delta_table.load_with_datetime(datetime).await?; + Arc::from(delta_table) + }; - if !session_ctx.table_exist(table_ref.clone())? { - session_ctx.register_table(table_ref, table_provider)?; - } + resolved_ref.table = Cow::Borrowed(name_with_version.as_str()); + + if !session_ctx.table_exist(resolved_ref.clone())? { + session_ctx.register_table(resolved_ref, table_provider_for_version)?; } } diff --git a/src/repository/interface.rs b/src/repository/interface.rs index a7c09adc..91c3e245 100644 --- a/src/repository/interface.rs +++ b/src/repository/interface.rs @@ -30,6 +30,7 @@ pub struct TableVersionsResult { pub collection_name: String, pub table_name: String, pub table_version_id: TableVersionId, + pub table_legacy: bool, pub creation_time: Timestamp, } diff --git a/src/repository/postgres.rs b/src/repository/postgres.rs index a8965e4c..58765510 100644 --- a/src/repository/postgres.rs +++ b/src/repository/postgres.rs @@ -48,6 +48,7 @@ impl PostgresRepository { collection.name AS collection_name, "table".name AS table_name, table_version.id AS table_version_id, + "table".legacy AS table_legacy, CAST(EXTRACT(EPOCH FROM table_version.creation_time) AS INT8) AS creation_time FROM table_version INNER JOIN "table" ON "table".id = table_version.table_id diff --git a/src/repository/sqlite.rs b/src/repository/sqlite.rs index bfbd2b98..26aadd9a 100644 --- a/src/repository/sqlite.rs +++ b/src/repository/sqlite.rs @@ -57,6 +57,7 @@ impl SqliteRepository { collection.name AS collection_name, "table".name AS table_name, table_version.id AS table_version_id, + "table".legacy AS table_legacy, CAST(table_version.creation_time AS INTEGER(8)) AS creation_time FROM table_version INNER JOIN "table" ON "table".id = table_version.table_id diff --git a/src/version.rs b/src/version.rs index 687f765f..d652c40c 100644 --- a/src/version.rs +++ b/src/version.rs @@ -44,9 +44,13 @@ impl TableVersionProcessor { { format!("{}:{}", name.0.last().unwrap().value, version_id) } else { - // The resolved table version id is None; this should only happen when we resolved - // the id to the latest table version, in which case no rewrite is needed. - name.0.last().unwrap().value.clone() + // The resolved table version id is None; this should only happen for delta tables, since + // we'll load those directly using the date string + format!( + "{}:{}", + name.0.last().unwrap().value, + version.to_ascii_lowercase() + ) } } @@ -115,6 +119,7 @@ impl TableVersionProcessor { ) .await? .into_iter() + .filter(|tv| tv.table_legacy) .group_by(|tv| { ObjectName(vec![ Ident::new(&tv.database_name), @@ -138,19 +143,16 @@ impl TableVersionProcessor { for (table_version, table_version_id) in self.table_versions.iter_mut() { let table = &table_version.0; let version = &table_version.1; + if !all_table_versions.contains_key(table) { + // This means this is not the legacy table, no need to triage the version id + continue; + } + let all_versions = all_table_versions.get(table).ok_or_else(|| { DataFusionError::Execution(format!("No versions found for table {table}")) })?; let id = TableVersionProcessor::resolve_version_id(version, all_versions)?; - - if id >= all_versions.last().unwrap().0 { - // The resolved table version id points to the latest table version; skip rewriting - // the table reference to that version, since we already have it loaded in the - // default table map of the schema provider. - continue; - } - *table_version_id = Some(id); } diff --git a/tests/statements/ddl.rs b/tests/statements/ddl.rs index ef9c0821..5e46bd42 100644 --- a/tests/statements/ddl.rs +++ b/tests/statements/ddl.rs @@ -2,7 +2,7 @@ use crate::statements::*; #[tokio::test] async fn test_create_table() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; let plan = context .plan_query( @@ -37,7 +37,7 @@ async fn test_create_table() { #[tokio::test] async fn test_create_table_as() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; create_table_and_insert(&context, "test_table").await; let plan = context @@ -78,7 +78,7 @@ async fn test_create_table_as() { async fn test_create_table_move_and_drop() { // Create two tables, insert some data into them - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; for table_name in ["test_table_1", "test_table_2"] { create_table_and_insert(&context, table_name).await; @@ -219,7 +219,7 @@ async fn test_create_table_move_and_drop() { #[tokio::test] async fn test_create_table_drop_schema() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; for table_name in ["test_table_1", "test_table_2"] { create_table_and_insert(&context, table_name).await; @@ -337,7 +337,7 @@ async fn test_create_table_drop_schema() { #[tokio::test] async fn test_create_table_schema_already_exists() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; context .collect( @@ -369,7 +369,7 @@ async fn test_create_table_schema_already_exists() { #[tokio::test] async fn test_create_table_in_staging_schema() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; context .collect( context @@ -420,7 +420,7 @@ async fn test_create_external_table_http() { &mock_server.uri() ); - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; // Try creating a table in a non-staging schema let err = context diff --git a/tests/statements/dml.rs b/tests/statements/dml.rs index 294ae555..f3933d27 100644 --- a/tests/statements/dml.rs +++ b/tests/statements/dml.rs @@ -2,7 +2,7 @@ use crate::statements::*; #[tokio::test] async fn test_insert_two_different_schemas() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; create_table_and_insert(&context, "test_table").await; let plan = context @@ -39,7 +39,7 @@ async fn test_insert_two_different_schemas() { #[tokio::test] async fn test_table_partitioning_and_rechunking() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; // Make table versions 1 and 2 create_table_and_insert(&context, "test_table").await; @@ -211,7 +211,7 @@ async fn test_table_partitioning_and_rechunking() { #[tokio::test] async fn test_delete_statement() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; create_table_and_some_partitions(&context, "test_table", None).await; @@ -438,7 +438,7 @@ async fn test_delete_statement() { #[tokio::test] async fn test_delete_with_string_filter_exact_match() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; context .collect( @@ -521,7 +521,7 @@ async fn test_delete_with_string_filter_exact_match() { #[tokio::test] async fn test_update_statement() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; create_table_and_some_partitions(&context, "test_table", None).await; @@ -649,7 +649,7 @@ async fn test_update_statement() { #[tokio::test] async fn test_update_statement_errors() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; // Creates table with table_versions 1 (empty) and 2 create_table_and_insert(&context, "test_table").await; diff --git a/tests/statements/function.rs b/tests/statements/function.rs index bb72e95d..013a0114 100644 --- a/tests/statements/function.rs +++ b/tests/statements/function.rs @@ -2,7 +2,7 @@ use crate::statements::*; #[tokio::test] async fn test_create_and_run_function() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; let function_query = r#"CREATE FUNCTION sintau AS ' { @@ -55,7 +55,7 @@ async fn test_create_and_run_function() { #[tokio::test] async fn test_create_and_run_function_legacy_type_names() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; let function_query = r#"CREATE FUNCTION sintau AS ' { @@ -108,7 +108,7 @@ async fn test_create_and_run_function_legacy_type_names() { #[tokio::test] async fn test_create_and_run_function_uppercase_type_names() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; let function_query = r#"CREATE FUNCTION sintau AS ' { diff --git a/tests/statements/mod.rs b/tests/statements/mod.rs index a63fcfb8..07ebfd9b 100644 --- a/tests/statements/mod.rs +++ b/tests/statements/mod.rs @@ -8,6 +8,7 @@ use assert_unordered::assert_eq_unordered_sort; use chrono::{TimeZone, Utc}; use datafusion::assert_batches_eq; use datafusion_common::assert_contains; +use deltalake::DeltaDataTypeVersion; use futures::TryStreamExt; use itertools::{sorted, Itertools}; use object_store::path::Path; @@ -20,6 +21,7 @@ use tokio::time::sleep; #[cfg(feature = "remote-tables")] use rstest::rstest; +use tempfile::TempDir; use seafowl::config::context::build_context; use seafowl::config::schema::load_config_from_string; @@ -48,16 +50,34 @@ const FILENAME_2: &str = const FILENAME_RECHUNKED: &str = "ece2031a3e121f8e4e31b2f2ef632a92b00dc479a6643172da8950def9fdb16e.parquet"; +enum ObjectStoreType { + Local(String), + InMemory, +} + /// Make a SeafowlContext that's connected to a real PostgreSQL database /// (but uses an in-memory object store) -async fn make_context_with_pg() -> DefaultSeafowlContext { +async fn make_context_with_pg( + object_store_type: ObjectStoreType, +) -> DefaultSeafowlContext { let dsn = env::var("DATABASE_URL").unwrap(); let schema = get_random_schema(); + let object_store_section = match object_store_type { + ObjectStoreType::Local(data_dir) => { + format!( + r#"type = "local" +data_dir = "{}""#, + data_dir + ) + } + ObjectStoreType::InMemory => r#"type = "memory""#.to_string(), + }; + let config_text = format!( r#" [object_store] -type = "memory" +{object_store_section} [catalog] type = "postgres" @@ -152,19 +172,19 @@ async fn create_table_and_some_partitions( table_name: &str, delay: Option, ) -> ( - HashMap>, - HashMap, + HashMap>, + HashMap, ) { - let mut version_results = HashMap::>::new(); - let mut version_timestamps = HashMap::::new(); + let mut version_results = HashMap::>::new(); + let mut version_timestamps = HashMap::::new(); async fn record_latest_version_snapshot( context: &DefaultSeafowlContext, - version_id: TableVersionId, + version_id: DeltaDataTypeVersion, table_name: &str, delay: Option, - version_results: &mut HashMap>, - version_timestamps: &mut HashMap, + version_results: &mut HashMap>, + version_timestamps: &mut HashMap, ) { if let Some(delay) = delay { let plan = context @@ -182,11 +202,11 @@ async fn create_table_and_some_partitions( } } - // Creates table with table_versions 1 (empty) and 2 + // Creates table with table versions 0 (empty) and 1 create_table_and_insert(context, table_name).await; record_latest_version_snapshot( context, - 2 as TableVersionId, + 1 as DeltaDataTypeVersion, table_name, delay, &mut version_results, @@ -194,7 +214,7 @@ async fn create_table_and_some_partitions( ) .await; - // Add another partition for table_version 3 + // Add another partition for table version 2 let plan = context .plan_query( format!("INSERT INTO {table_name} (some_value) VALUES (45), (46), (47)") @@ -205,7 +225,7 @@ async fn create_table_and_some_partitions( context.collect(plan).await.unwrap(); record_latest_version_snapshot( context, - 3 as TableVersionId, + 2 as DeltaDataTypeVersion, table_name, delay, &mut version_results, @@ -213,7 +233,7 @@ async fn create_table_and_some_partitions( ) .await; - // Add another partition for table_version 4 + // Add another partition for table_version 3 let plan = context .plan_query( format!("INSERT INTO {table_name} (some_value) VALUES (46), (47), (48)") @@ -224,7 +244,7 @@ async fn create_table_and_some_partitions( context.collect(plan).await.unwrap(); record_latest_version_snapshot( context, - 4 as TableVersionId, + 3 as DeltaDataTypeVersion, table_name, delay, &mut version_results, @@ -232,7 +252,7 @@ async fn create_table_and_some_partitions( ) .await; - // Add another partition for table_version 5 + // Add another partition for table_version 4 let plan = context .plan_query( format!("INSERT INTO {table_name} (some_value) VALUES (42), (41), (40)") @@ -243,7 +263,7 @@ async fn create_table_and_some_partitions( context.collect(plan).await.unwrap(); record_latest_version_snapshot( context, - 5 as TableVersionId, + 4 as DeltaDataTypeVersion, table_name, delay, &mut version_results, @@ -251,9 +271,6 @@ async fn create_table_and_some_partitions( ) .await; - // We have 4 partitions from 4 INSERTS - assert_partition_ids(context, 5, vec![1, 2, 3, 4]).await; - (version_results, version_timestamps) } diff --git a/tests/statements/query.rs b/tests/statements/query.rs index 3fe5b760..5213ee4b 100644 --- a/tests/statements/query.rs +++ b/tests/statements/query.rs @@ -2,7 +2,7 @@ use crate::statements::*; #[tokio::test] async fn test_information_schema() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; let plan = context .plan_query( @@ -62,7 +62,7 @@ async fn test_information_schema() { #[tokio::test] async fn test_create_table_and_insert() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; // TODO: insert into nonexistent table outputs a wrong error (schema "public" does not exist) create_table_and_insert(&context, "test_table").await; @@ -122,7 +122,11 @@ async fn test_create_table_and_insert() { #[tokio::test] async fn test_table_time_travel() { - let context = make_context_with_pg().await; + let data_dir = TempDir::new().unwrap(); + let context = make_context_with_pg(ObjectStoreType::Local( + data_dir.path().display().to_string(), + )) + .await; let (version_results, version_timestamps) = create_table_and_some_partitions( &context, "test_table", @@ -138,82 +142,84 @@ async fn test_table_time_travel() { // Verify that the new table versions are shown in the corresponding system table // - let plan = context - .plan_query("SELECT table_schema, table_name, table_version_id FROM system.table_versions") - .await - .unwrap(); - let results = context.collect(plan).await.unwrap(); - - let expected = vec![ - "+--------------+------------+------------------+", - "| table_schema | table_name | table_version_id |", - "+--------------+------------+------------------+", - "| public | test_table | 1 |", - "| public | test_table | 2 |", - "| public | test_table | 3 |", - "| public | test_table | 4 |", - "| public | test_table | 5 |", - "+--------------+------------+------------------+", - ]; - assert_batches_eq!(expected, &results); - + // TODO: Make `system.table_versions` table work for delta tables + // let plan = context + // .plan_query("SELECT table_schema, table_name, table_version_id FROM system.table_versions") + // .await + // .unwrap(); + // let results = context.collect(plan).await.unwrap(); // - // Test that filtering the system table works, given that we provide all rows to DF and expect - // it to do it. + // let expected = vec![ + // "+--------------+------------+------------------+", + // "| table_schema | table_name | table_version_id |", + // "+--------------+------------+------------------+", + // "| public | test_table | 1 |", + // "| public | test_table | 2 |", + // "| public | test_table | 3 |", + // "| public | test_table | 4 |", + // "| public | test_table | 5 |", + // "+--------------+------------+------------------+", + // ]; + // assert_batches_eq!(expected, &results); + + // // + // // Test that filtering the system table works, given that we provide all rows to DF and expect + // // it to do it. + // // + // let plan = context + // .plan_query( + // format!( + // " + // SELECT table_version_id FROM system.table_versions \ + // WHERE table_version_id < 5 AND creation_time > to_timestamp('{}') + // ", + // timestamp_to_rfc3339(version_timestamps[&2]) + // ) + // .as_str(), + // ) + // .await + // .unwrap(); + // let results = context.collect(plan).await.unwrap(); // - let plan = context - .plan_query( - format!( - " - SELECT table_version_id FROM system.table_versions \ - WHERE table_version_id < 5 AND creation_time > to_timestamp('{}') - ", - timestamp_to_rfc3339(version_timestamps[&2]) - ) - .as_str(), - ) - .await - .unwrap(); - let results = context.collect(plan).await.unwrap(); - - let expected = vec![ - "+------------------+", - "| table_version_id |", - "+------------------+", - "| 3 |", - "| 4 |", - "+------------------+", - ]; - assert_batches_eq!(expected, &results); + // let expected = vec![ + // "+------------------+", + // "| table_version_id |", + // "+------------------+", + // "| 3 |", + // "| 4 |", + // "+------------------+", + // ]; + // assert_batches_eq!(expected, &results); // // Verify that the new table partitions for all versions are shown in the corresponding system table // - let plan = context - .plan_query("SELECT table_schema, table_name, table_version_id, table_partition_id, row_count FROM system.table_partitions") - .await - .unwrap(); - let results = context.collect(plan).await.unwrap(); - - let expected = vec![ - "+--------------+------------+------------------+--------------------+-----------+", - "| table_schema | table_name | table_version_id | table_partition_id | row_count |", - "+--------------+------------+------------------+--------------------+-----------+", - "| public | test_table | 1 | | |", - "| public | test_table | 2 | 1 | 3 |", - "| public | test_table | 3 | 1 | 3 |", - "| public | test_table | 3 | 2 | 3 |", - "| public | test_table | 4 | 1 | 3 |", - "| public | test_table | 4 | 2 | 3 |", - "| public | test_table | 4 | 3 | 3 |", - "| public | test_table | 5 | 1 | 3 |", - "| public | test_table | 5 | 2 | 3 |", - "| public | test_table | 5 | 3 | 3 |", - "| public | test_table | 5 | 4 | 3 |", - "+--------------+------------+------------------+--------------------+-----------+", - ]; - assert_batches_eq!(expected, &results); + // TODO: Make `system.table_partitions` table work for delta tables + // let plan = context + // .plan_query("SELECT table_schema, table_name, table_version_id, table_partition_id, row_count FROM system.table_partitions") + // .await + // .unwrap(); + // let results = context.collect(plan).await.unwrap(); + // + // let expected = vec![ + // "+--------------+------------+------------------+--------------------+-----------+", + // "| table_schema | table_name | table_version_id | table_partition_id | row_count |", + // "+--------------+------------+------------------+--------------------+-----------+", + // "| public | test_table | 1 | | |", + // "| public | test_table | 2 | 1 | 3 |", + // "| public | test_table | 3 | 1 | 3 |", + // "| public | test_table | 3 | 2 | 3 |", + // "| public | test_table | 4 | 1 | 3 |", + // "| public | test_table | 4 | 2 | 3 |", + // "| public | test_table | 4 | 3 | 3 |", + // "| public | test_table | 5 | 1 | 3 |", + // "| public | test_table | 5 | 2 | 3 |", + // "| public | test_table | 5 | 3 | 3 |", + // "| public | test_table | 5 | 4 | 3 |", + // "+--------------+------------+------------------+--------------------+-----------+", + // ]; + // assert_batches_eq!(expected, &results); // // Now use the recorded timestamps to query specific earlier table versions and compare them to @@ -222,9 +228,9 @@ async fn test_table_time_travel() { async fn query_table_version( context: &DefaultSeafowlContext, - version_id: TableVersionId, - version_results: &HashMap>, - version_timestamps: &HashMap, + version_id: DeltaDataTypeVersion, + version_results: &HashMap>, + version_timestamps: &HashMap, timestamp_converter: fn(Timestamp) -> String, ) { let plan = context @@ -242,10 +248,10 @@ async fn test_table_time_travel() { assert_eq!(version_results[&version_id], results); } - for version_id in [2, 3, 4, 5] { + for version_id in [1, 2, 3, 4] { query_table_version( &context, - version_id as TableVersionId, + version_id as DeltaDataTypeVersion, &version_results, &version_timestamps, timestamp_to_rfc3339, @@ -253,66 +259,53 @@ async fn test_table_time_travel() { .await; } - // - // Try to query a non-existent version (timestamp older than the oldest version) - // - - let err = context - .plan_query("SELECT * FROM test_table('2012-12-21 20:12:21 +00:00')") - .await - .unwrap_err(); - - assert!(err - .to_string() - .contains("No recorded table versions for the provided timestamp")); - // // Use multiple different version specifiers in the same complex query (including the latest // version both explicitly and in the default notation). // Ensures row differences between different versions are consistent: - // 5 - ((5 - 4) + (4 - 3) + (3 - 2)) = 2 + // 4 - ((4 - 3) + (3 - 2) + (2 - 1)) = 1 // let plan = context .plan_query( format!( r#" - WITH diff_3_2 AS ( + WITH diff_2_1 AS ( SELECT * FROM test_table('{}') EXCEPT SELECT * FROM test_table('{}') - ), diff_4_3 AS ( + ), diff_3_2 AS ( SELECT * FROM test_table('{}') EXCEPT SELECT * FROM test_table('{}') - ), diff_5_4 AS ( + ), diff_4_3 AS ( SELECT * FROM test_table('{}') EXCEPT SELECT * FROM test_table('{}') ) SELECT * FROM test_table EXCEPT ( - SELECT * FROM diff_5_4 - UNION SELECT * FROM diff_4_3 UNION SELECT * FROM diff_3_2 + UNION + SELECT * FROM diff_2_1 ) ORDER BY some_int_value "#, + timestamp_to_rfc3339(version_timestamps[&2]), + timestamp_to_rfc3339(version_timestamps[&1]), timestamp_to_rfc3339(version_timestamps[&3]), timestamp_to_rfc3339(version_timestamps[&2]), timestamp_to_rfc3339(version_timestamps[&4]), timestamp_to_rfc3339(version_timestamps[&3]), - timestamp_to_rfc3339(version_timestamps[&5]), - timestamp_to_rfc3339(version_timestamps[&4]), ) .as_str(), ) .await .unwrap(); let results = context.collect(plan).await.unwrap(); - assert_eq!(version_results[&2], results); + assert_eq!(version_results[&1], results); // Ensure the context table map contains the versioned + the latest table entries assert_eq!( @@ -330,9 +323,22 @@ async fn test_table_time_travel() { .collect::>(), vec![ "test_table".to_string(), - "test_table:2".to_string(), - "test_table:3".to_string(), - "test_table:4".to_string(), + format!( + "test_table:{}", + timestamp_to_rfc3339(version_timestamps[&1]).to_ascii_lowercase() + ), + format!( + "test_table:{}", + timestamp_to_rfc3339(version_timestamps[&2]).to_ascii_lowercase() + ), + format!( + "test_table:{}", + timestamp_to_rfc3339(version_timestamps[&3]).to_ascii_lowercase() + ), + format!( + "test_table:{}", + timestamp_to_rfc3339(version_timestamps[&4]).to_ascii_lowercase() + ), ], ); @@ -358,15 +364,15 @@ async fn test_table_time_travel() { let results = list_columns_query(&context).await; let expected = vec![ - "+--------------+------------+------------------+-----------------------------+", - "| table_schema | table_name | column_name | data_type |", - "+--------------+------------+------------------+-----------------------------+", - "| public | test_table | some_bool_value | Boolean |", - "| public | test_table | some_int_value | Int64 |", - "| public | test_table | some_other_value | Decimal128(38, 10) |", - "| public | test_table | some_time | Timestamp(Nanosecond, None) |", - "| public | test_table | some_value | Float32 |", - "+--------------+------------+------------------+-----------------------------+", + "+--------------+------------+------------------+------------------------------+", + "| table_schema | table_name | column_name | data_type |", + "+--------------+------------+------------------+------------------------------+", + "| public | test_table | some_time | Timestamp(Microsecond, None) |", + "| public | test_table | some_value | Float32 |", + "| public | test_table | some_other_value | Decimal128(38, 10) |", + "| public | test_table | some_bool_value | Boolean |", + "| public | test_table | some_int_value | Int64 |", + "+--------------+------------+------------------+------------------------------+", ]; assert_batches_eq!(expected, &results); } @@ -382,7 +388,7 @@ async fn test_remote_table_querying( #[case] db_type: &str, #[case] introspect_schema: bool, ) { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; let schema = get_random_schema(); let _temp_path: TempPath; @@ -576,7 +582,7 @@ async fn test_remote_table_querying( #[tokio::test] async fn test_delta_tables() { - let context = make_context_with_pg().await; + let context = make_context_with_pg(ObjectStoreType::InMemory).await; let plan = context .plan_query( diff --git a/tests/statements/vacuum.rs b/tests/statements/vacuum.rs index 10edd4ee..e9f214f2 100644 --- a/tests/statements/vacuum.rs +++ b/tests/statements/vacuum.rs @@ -2,7 +2,7 @@ use crate::statements::*; #[tokio::test] async fn test_vacuum_command() { - let context = Arc::new(make_context_with_pg().await); + let context = Arc::new(make_context_with_pg(ObjectStoreType::InMemory).await); let get_object_metas = || async { context @@ -122,7 +122,7 @@ async fn test_vacuum_command() { #[tokio::test] async fn test_vacuum_with_reused_file() { - let context = Arc::new(make_context_with_pg().await); + let context = Arc::new(make_context_with_pg(ObjectStoreType::InMemory).await); // Creates table_1 (empty v1, v2) and table_2 (empty v3, v4) // V2 and V4 point to a single identical partition From 6dce3541dbe991a581d047476e17eedf27f52a0b Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Wed, 1 Mar 2023 22:35:58 +0100 Subject: [PATCH 05/21] Implement deletion of table objects on DROP TABLE --- src/context.rs | 23 ++++++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) diff --git a/src/context.rs b/src/context.rs index 8155ec8d..44b79e75 100644 --- a/src/context.rs +++ b/src/context.rs @@ -34,7 +34,7 @@ use futures::{StreamExt, TryStreamExt}; #[cfg(test)] use mockall::automock; -use object_store::{path::Path, ObjectStore}; +use object_store::{path::Path, ObjectMeta, ObjectStore}; use sqlparser::ast::{ AlterTableOperation, CreateFunctionBody, FunctionDefinition, Ident, ObjectName, @@ -1689,9 +1689,26 @@ impl SeafowlContext for DefaultSeafowlContext { DataFusionError::Execution("Table {name} not found".to_string()) })?; - self.table_catalog.drop_table(table_id).await?; // TODO: delay the actual delete and perform it during the subsequent vacuuming - // somehow, or when a new CREATE TABLE targets the same uri simply owerwrite + // somehow, or when a new CREATE TABLE targets the same uri simply overwrite + let table_object_store = self.internal_object_store.for_delta_table( + &resolved_ref.catalog, + &resolved_ref.schema, + &resolved_ref.table, + ); + let mut path_stream = + table_object_store.list(None).await.map_err(|err| { + DataFusionError::Execution(format!( + "Cannot list contents of table {name}: {}", + err + )) + })?; + while let Some(maybe_object) = path_stream.next().await { + if let Ok(ObjectMeta { location, .. }) = maybe_object { + table_object_store.delete(&location).await?; + } + } + self.table_catalog.drop_table(table_id).await?; Ok(make_dummy_exec()) } LogicalPlan::CreateView(_) => { From 6ae318f7290b33f31de415e9ddca7a8e2fb81876 Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Thu, 2 Mar 2023 22:31:59 +0100 Subject: [PATCH 06/21] Enable ALTER TABLE on object stores by table object renaming --- src/context.rs | 150 ++++++++++++++++++++++++++---------- src/nodes.rs | 10 +-- src/object_store/wrapped.rs | 56 +++++++++++++- tests/statements/ddl.rs | 24 +++--- 4 files changed, 182 insertions(+), 58 deletions(-) diff --git a/src/context.rs b/src/context.rs index 44b79e75..09af917a 100644 --- a/src/context.rs +++ b/src/context.rs @@ -34,7 +34,7 @@ use futures::{StreamExt, TryStreamExt}; #[cfg(test)] use mockall::automock; -use object_store::{path::Path, ObjectMeta, ObjectStore}; +use object_store::{path::Path, ObjectStore}; use sqlparser::ast::{ AlterTableOperation, CreateFunctionBody, FunctionDefinition, Ident, ObjectName, @@ -1216,9 +1216,12 @@ impl SeafowlContext for DefaultSeafowlContext { Statement::AlterTable { name, operation: AlterTableOperation::RenameTable {table_name: new_name }} => { let old_table_name = remove_quotes_from_object_name(&name).to_string(); let new_table_name = remove_quotes_from_object_name(&new_name).to_string(); - let table = self.try_get_seafowl_table(old_table_name).await?; - if self.get_table_provider(new_table_name.to_owned()).await.is_ok() { + if self.get_table_provider(old_table_name.to_owned()).await.is_err() { + return Err(Error::Plan( + format!("Source table {old_table_name:?} doesn't exist") + )) + } else if self.get_table_provider(new_table_name.to_owned()).await.is_ok() { return Err(Error::Plan( format!("Target table {new_table_name:?} already exists") )) @@ -1226,7 +1229,7 @@ impl SeafowlContext for DefaultSeafowlContext { Ok(LogicalPlan::Extension(Extension { node: Arc::new(SeafowlExtensionNode::RenameTable(RenameTable { - table: Arc::from(table), + old_name: old_table_name, new_name: new_table_name, output_schema: Arc::new(DFSchema::empty()) })), @@ -1691,23 +1694,20 @@ impl SeafowlContext for DefaultSeafowlContext { // TODO: delay the actual delete and perform it during the subsequent vacuuming // somehow, or when a new CREATE TABLE targets the same uri simply overwrite - let table_object_store = self.internal_object_store.for_delta_table( - &resolved_ref.catalog, - &resolved_ref.schema, - &resolved_ref.table, - ); - let mut path_stream = - table_object_store.list(None).await.map_err(|err| { + let table_prefix = Path::from(format!( + "{}/{}/{}", + &resolved_ref.catalog, &resolved_ref.schema, &resolved_ref.table + )); + + self.internal_object_store + .delete_in_prefix(&table_prefix) + .await + .map_err(|err| { DataFusionError::Execution(format!( - "Cannot list contents of table {name}: {}", + "Failed to delete table {name} : {}", err )) })?; - while let Some(maybe_object) = path_stream.next().await { - if let Ok(ObjectMeta { location, .. }) = maybe_object { - table_object_store.delete(&location).await?; - } - } self.table_catalog.drop_table(table_id).await?; Ok(make_dummy_exec()) } @@ -1744,45 +1744,101 @@ impl SeafowlContext for DefaultSeafowlContext { Ok(make_dummy_exec()) } SeafowlExtensionNode::RenameTable(RenameTable { - table, + old_name, new_name, .. }) => { - let table_ref = TableReference::from(new_name.as_str()); + // Resolve new table reference + let new_table_ref = TableReference::from(new_name.as_str()); + let resolved_new_ref = + new_table_ref.resolve(&self.database, DEFAULT_SCHEMA); + if resolved_new_ref.catalog != self.database { + return Err(Error::Plan( + "Changing the table's database is not supported!" + .to_string(), + )); + } + let new_path_prefix = Path::from(format!( + "{}/{}/{}", + &resolved_new_ref.catalog, + &resolved_new_ref.schema, + &resolved_new_ref.table + )); + + // Resolve old table reference and fetch the table id + let old_table_ref = TableReference::from(old_name.as_str()); + let resolved_old_ref = + old_table_ref.resolve(&self.database, DEFAULT_SCHEMA); + let old_path_prefix = Path::from(format!( + "{}/{}/{}", + &resolved_old_ref.catalog, + &resolved_old_ref.schema, + &resolved_old_ref.table + )); + + let table_id = self + .table_catalog + .get_table_id_by_name( + &resolved_old_ref.catalog, + &resolved_old_ref.schema, + &resolved_old_ref.table, + ) + .await? + .ok_or_else(|| { + DataFusionError::Execution( + "Table {old_name} not found".to_string(), + ) + })?; - let (new_table_name, new_schema_id) = match table_ref { - // Rename the table (keep same schema) - TableReference::Bare { table } => (table, None), - // Rename the table / move its schema - TableReference::Partial { schema, table } => { + // If the old and new table schema is different check that the + // corresponding collection already exists + let new_schema_id = + if resolved_new_ref.schema != resolved_old_ref.schema { let collection_id = self .table_catalog .get_collection_id_by_name( &self.database, - &schema, + &resolved_new_ref.schema, ) .await? .ok_or_else(|| { Error::Plan(format!( - "Schema {schema:?} does not exist!" + "Schema \"{}\" does not exist!", + &resolved_new_ref.schema, )) })?; - - (table, Some(collection_id)) - } - // Catalog specified: raise an error - TableReference::Full { .. } => { - return Err(Error::Plan( - "Changing the table's database is not supported!" - .to_string(), - )) - } - }; - + Some(collection_id) + } else { + None + }; + + // If we're using the local FS store ensure the full Delta log folder + // path exists prior to the move + self.internal_object_store + .ensure_path_exists(&new_path_prefix.child("_delta_log")) + .await + .map_err(|err| { + DataFusionError::Execution(format!( + "Failed to ensure path existence for table {new_name} : {}", + err + )) + })?; + // Iterate over all objects in the old table path and rename to new path + self.internal_object_store + .rename_in_prefix(&old_path_prefix, &new_path_prefix) + .await + .map_err(|err| { + DataFusionError::Execution(format!( + "Failed to rename table objects in {old_path_prefix} to {new_path_prefix}: {}", + err + )) + })?; + + // Finally update our catalog entry self.table_catalog .move_table( - table.table_id, - &new_table_name, + table_id, + &resolved_new_ref.table, new_schema_id, ) .await?; @@ -1795,6 +1851,18 @@ impl SeafowlContext for DefaultSeafowlContext { .get_collection_id_by_name(&self.database, name) .await? { + let schema_prefix = + Path::from(format!("{}/{}", &self.database, name)); + // This is very bad. + self.internal_object_store + .delete_in_prefix(&schema_prefix) + .await + .map_err(|err| { + DataFusionError::Execution(format!( + "Failed to delete objects in schema {name}: {}", + err + )) + })?; self.table_catalog.drop_collection(collection_id).await? }; @@ -1997,7 +2065,7 @@ pub mod test_utils { { let session = make_session(); let arrow_schema = ArrowSchema::new(vec![ - ArrowField::new("date", ArrowDataType::Date64, false), + ArrowField::new("date", ArrowDataType::Date32, false), ArrowField::new("value", ArrowDataType::Float64, false), ]); diff --git a/src/nodes.rs b/src/nodes.rs index 30ad6d4d..d2cdee9d 100644 --- a/src/nodes.rs +++ b/src/nodes.rs @@ -6,7 +6,7 @@ use std::{any::Any, fmt, sync::Arc, vec}; use datafusion_expr::{Expr, LogicalPlan, UserDefinedLogicalNode}; use crate::data_types::TableId; -use crate::{provider::SeafowlTable, wasm_udf::data_types::CreateFunctionDetails}; +use crate::wasm_udf::data_types::CreateFunctionDetails; #[derive(Debug, Clone)] pub struct CreateTable { @@ -32,8 +32,8 @@ pub struct CreateFunction { #[derive(Debug, Clone)] pub struct RenameTable { - /// The table to rename - pub table: Arc, + /// Old name + pub old_name: String, /// New name (including the schema name) pub new_name: String, /// Dummy result schema for the plan (empty) @@ -122,9 +122,9 @@ impl UserDefinedLogicalNode for SeafowlExtensionNode { write!(f, "CreateFunction: {name}") } SeafowlExtensionNode::RenameTable(RenameTable { - table, new_name, .. + old_name, new_name, .. }) => { - write!(f, "RenameTable: {} to {}", table.name, new_name) + write!(f, "RenameTable: {} to {}", old_name, new_name) } SeafowlExtensionNode::DropSchema(DropSchema { name, .. }) => { write!(f, "DropSchema: {name}") diff --git a/src/object_store/wrapped.rs b/src/object_store/wrapped.rs index 5d9897d3..400e8593 100644 --- a/src/object_store/wrapped.rs +++ b/src/object_store/wrapped.rs @@ -1,7 +1,7 @@ use crate::config::schema; use crate::config::schema::{Local, S3}; use bytes::Bytes; -use futures::{stream::BoxStream, TryFutureExt}; +use futures::{stream::BoxStream, StreamExt, TryFutureExt}; use log::debug; use object_store::{ path::Path, Error, GetResult, ListResult, MultipartId, ObjectMeta, ObjectStore, @@ -11,9 +11,10 @@ use std::fmt::{Debug, Display, Formatter}; use std::ops::Range; use tokio::io::AsyncWrite; -use tokio::fs::{copy, remove_file, rename}; +use tokio::fs::{copy, create_dir_all, remove_file, rename}; use deltalake::storage::DeltaObjectStore; +use itertools::Itertools; use object_store::prefix::PrefixObjectStore; use std::path::Path as StdPath; use std::str::FromStr; @@ -79,6 +80,57 @@ impl InternalObjectStore { )) } + /// Delete all objects under a given prefix + pub async fn delete_in_prefix(&self, prefix: &Path) -> Result<(), Error> { + let mut path_stream = self.inner.list(Some(prefix)).await?; + while let Some(maybe_object) = path_stream.next().await { + if let Ok(ObjectMeta { location, .. }) = maybe_object { + self.inner.delete(&location).await?; + } + } + Ok(()) + } + + /// Create any missing intermediate directories in the given path, so that further actions don't + /// error out. Applicable only to local FS, since in other stores the directories are virtual. + pub async fn ensure_path_exists(&self, path: &Path) -> Result<(), Error> { + if let schema::ObjectStore::Local(_) = self.config.clone() { + let full_path = StdPath::new(self.root_uri.path()).join(path.to_string()); + create_dir_all(full_path) + .await + .map_err(|e| Error::Generic { + store: "local", + source: Box::new(e), + })?; + } + + Ok(()) + } + + /// Moving all objects in paths with `from_prefix` to paths with `to_prefix`. The main purpose + /// of this is to ensure that `to_prefix` actually exists when using the local FS store, otherwise + /// we get "No such file or directory" on rename. + pub async fn rename_in_prefix( + &self, + from_prefix: &Path, + to_prefix: &Path, + ) -> Result<(), Error> { + // Go over all objects with the `from_prefix` prefix and rename them to be with `to_prefix` + let mut path_stream = self.inner.list(Some(from_prefix)).await?; + + while let Some(maybe_object) = path_stream.next().await { + if let Ok(ObjectMeta { location, .. }) = maybe_object { + // We unwrap since the path must match the from prefix + let mut new_path_parts = to_prefix.parts().collect_vec(); + new_path_parts + .extend(location.prefix_match(from_prefix).unwrap().collect_vec()); + let new_location = Path::from_iter(new_path_parts); + self.inner.rename(&location, &new_location).await?; + } + } + Ok(()) + } + /// For local filesystem object stores, try "uploading" by just moving the file. /// Returns a None if the store isn't local. pub async fn fast_upload( diff --git a/tests/statements/ddl.rs b/tests/statements/ddl.rs index 5e46bd42..eaaa997f 100644 --- a/tests/statements/ddl.rs +++ b/tests/statements/ddl.rs @@ -195,15 +195,15 @@ async fn test_create_table_move_and_drop() { let results = list_columns_query(&context).await; let expected = vec![ - "+--------------+--------------+------------------+-----------------------------+", - "| table_schema | table_name | column_name | data_type |", - "+--------------+--------------+------------------+-----------------------------+", - "| public | test_table_2 | some_bool_value | Boolean |", - "| public | test_table_2 | some_int_value | Int64 |", - "| public | test_table_2 | some_other_value | Decimal128(38, 10) |", - "| public | test_table_2 | some_time | Timestamp(Nanosecond, None) |", - "| public | test_table_2 | some_value | Float32 |", - "+--------------+--------------+------------------+-----------------------------+", + "+--------------+--------------+------------------+------------------------------+", + "| table_schema | table_name | column_name | data_type |", + "+--------------+--------------+------------------+------------------------------+", + "| public | test_table_2 | some_time | Timestamp(Microsecond, None) |", + "| public | test_table_2 | some_value | Float32 |", + "| public | test_table_2 | some_other_value | Decimal128(38, 10) |", + "| public | test_table_2 | some_bool_value | Boolean |", + "| public | test_table_2 | some_int_value | Int64 |", + "+--------------+--------------+------------------+------------------------------+", ]; assert_batches_eq!(expected, &results); @@ -219,7 +219,11 @@ async fn test_create_table_move_and_drop() { #[tokio::test] async fn test_create_table_drop_schema() { - let context = make_context_with_pg(ObjectStoreType::InMemory).await; + let data_dir = TempDir::new().unwrap(); + let context = make_context_with_pg(ObjectStoreType::Local( + data_dir.path().display().to_string(), + )) + .await; for table_name in ["test_table_1", "test_table_2"] { create_table_and_insert(&context, table_name).await; From dc2018f7644d81da00cde036e81319ed9a11ef2c Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Fri, 3 Mar 2023 10:05:20 +0100 Subject: [PATCH 07/21] Fix tests or ignore them if the implementation is still missing --- Cargo.lock | 3 +++ Cargo.toml | 4 ++-- src/context.rs | 7 ++++--- tests/statements/dml.rs | 5 +++++ tests/statements/vacuum.rs | 2 ++ 5 files changed, 16 insertions(+), 5 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 0d72bcea..f0c81762 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1869,6 +1869,7 @@ checksum = "eaa37046cc0f6c3cc6090fbdbf73ef0b8ef4cfcc37f6befc0020f63e8cf121e1" [[package]] name = "deltalake" version = "0.7.0" +source = "git+https://github.com/delta-io/delta-rs#8dcf46e614370b93a6d47402456621b713768674" dependencies = [ "arrow", "async-trait", @@ -2022,6 +2023,7 @@ checksum = "1435fa1053d8b2fbbe9be7e97eca7f33d37b28409959813daefc1446a14247f1" [[package]] name = "dynamodb_lock" version = "0.4.3" +source = "git+https://github.com/delta-io/delta-rs#8dcf46e614370b93a6d47402456621b713768674" dependencies = [ "async-trait", "log", @@ -2560,6 +2562,7 @@ dependencies = [ [[package]] name = "glibc_version" version = "0.1.2" +source = "git+https://github.com/delta-io/delta-rs#8dcf46e614370b93a6d47402456621b713768674" dependencies = [ "regex", ] diff --git a/Cargo.toml b/Cargo.toml index 95a7b2b8..42e36be0 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -54,8 +54,8 @@ datafusion-proto = "18.0.0" datafusion-remote-tables = { path = "./datafusion_remote_tables", optional = true } -deltalake = { path = "../delta-rs/rust", features = ["s3-native-tls", "datafusion-ext"] } -dynamodb_lock = { path = "../delta-rs/dynamodb_lock", default_features = false, features = ["native-tls"] } +deltalake = { git = "https://github.com/delta-io/delta-rs", hash = "8dcf46e614370b93a6d47402456621b713768674", features = ["s3-native-tls", "datafusion-ext"] } +dynamodb_lock = { git = "https://github.com/delta-io/delta-rs", package = "dynamodb_lock", hash = "8dcf46e614370b93a6d47402456621b713768674", default_features = false, features = ["native-tls"] } futures = "0.3" hex = ">=0.4.0" diff --git a/src/context.rs b/src/context.rs index 09af917a..22e24222 100644 --- a/src/context.rs +++ b/src/context.rs @@ -2468,7 +2468,7 @@ mod tests { assert_eq!( format!("{plan:?}"), "Dml: op=[Insert] table=[testcol.some_table]\ - \n Projection: CAST(column1 AS Date64) AS date, CAST(column2 AS Float64) AS value\ + \n Projection: CAST(column1 AS Date32) AS date, CAST(column2 AS Float64) AS value\ \n Values: (Utf8(\"2022-01-01T12:00:00\"), Int64(42))" ); } @@ -2618,7 +2618,7 @@ mod tests { async fn test_plan_rename_table_name_in_quotes() { assert_eq!( get_logical_plan("ALTER TABLE \"testcol\".\"some_table\" RENAME TO \"testcol\".\"some_table_2\"").await, - "RenameTable: some_table to testcol.some_table_2" + "RenameTable: testcol.some_table to testcol.some_table_2" ); } @@ -2644,7 +2644,7 @@ mod tests { assert_eq!( format!("{plan:?}"), "Dml: op=[Insert] table=[testcol.some_table]\ - \n Projection: CAST(column1 AS Date64) AS date, CAST(column2 AS Float64) AS value\ + \n Projection: CAST(column1 AS Date32) AS date, CAST(column2 AS Float64) AS value\ \n Values: (Utf8(\"2022-01-01T12:00:00\"), Int64(42))" ); } @@ -2691,6 +2691,7 @@ mod tests { ); } + #[ignore = "fails since '2022-01-01T12:00:00' can't be cast to Date32 in chrono"] #[tokio::test] async fn test_preexec_insert() { let sf_context = mock_context_with_catalog_assertions( diff --git a/tests/statements/dml.rs b/tests/statements/dml.rs index f3933d27..909ee587 100644 --- a/tests/statements/dml.rs +++ b/tests/statements/dml.rs @@ -37,6 +37,7 @@ async fn test_insert_two_different_schemas() { assert_batches_eq!(expected, &results); } +#[ignore = "not yet implemented"] #[tokio::test] async fn test_table_partitioning_and_rechunking() { let context = make_context_with_pg(ObjectStoreType::InMemory).await; @@ -209,6 +210,7 @@ async fn test_table_partitioning_and_rechunking() { assert_batches_eq!(expected, &results); } +#[ignore = "not yet implemented"] #[tokio::test] async fn test_delete_statement() { let context = make_context_with_pg(ObjectStoreType::InMemory).await; @@ -436,6 +438,7 @@ async fn test_delete_statement() { assert!(results.is_empty()); } +#[ignore = "not yet implemented"] #[tokio::test] async fn test_delete_with_string_filter_exact_match() { let context = make_context_with_pg(ObjectStoreType::InMemory).await; @@ -519,6 +522,7 @@ async fn test_delete_with_string_filter_exact_match() { assert_batches_eq!(expected, &results); } +#[ignore = "not yet implemented"] #[tokio::test] async fn test_update_statement() { let context = make_context_with_pg(ObjectStoreType::InMemory).await; @@ -647,6 +651,7 @@ async fn test_update_statement() { assert_batches_eq!(expected, &results); } +#[ignore = "not yet implemented"] #[tokio::test] async fn test_update_statement_errors() { let context = make_context_with_pg(ObjectStoreType::InMemory).await; diff --git a/tests/statements/vacuum.rs b/tests/statements/vacuum.rs index e9f214f2..0f05749f 100644 --- a/tests/statements/vacuum.rs +++ b/tests/statements/vacuum.rs @@ -1,5 +1,6 @@ use crate::statements::*; +#[ignore = "not yet implemented"] #[tokio::test] async fn test_vacuum_command() { let context = Arc::new(make_context_with_pg(ObjectStoreType::InMemory).await); @@ -120,6 +121,7 @@ async fn test_vacuum_command() { assert_eq!(object_metas.len(), 0); } +#[ignore = "not yet implemented"] #[tokio::test] async fn test_vacuum_with_reused_file() { let context = Arc::new(make_context_with_pg(ObjectStoreType::InMemory).await); From 77179e6ad17d9fd55e1cd0e1cd732ec8fdb26226 Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Fri, 3 Mar 2023 11:57:05 +0100 Subject: [PATCH 08/21] Remove testing crutches used during development --- Cargo.lock | 16 ++++++++++++++++ Cargo.toml | 6 +----- src/frontend/http.rs | 2 -- src/provider.rs | 4 ++-- 4 files changed, 19 insertions(+), 9 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f0c81762..b016128e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1684,6 +1684,8 @@ dependencies = [ [[package]] name = "datafusion" version = "18.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cd805bdf93d3137b37fd9966042df0c84ddfca0df5a8d32eaacb16cf6ab0d93d" dependencies = [ "ahash 0.8.1", "arrow", @@ -1729,6 +1731,8 @@ dependencies = [ [[package]] name = "datafusion-common" version = "18.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "08c58d6714427f52f9815d19debab7adab5bac5b4d2a99d51c250e606acb6cf5" dependencies = [ "arrow", "chrono", @@ -1741,6 +1745,8 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "18.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2a32ee054230dd9a57d0bed587406869c4a7814d90154616aff2cb9991c1756f" dependencies = [ "ahash 0.8.1", "arrow", @@ -1752,6 +1758,8 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "18.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6de4d144924de29a835feeff8313a81fdc2c7190111301508e09ea59a80edbbc" dependencies = [ "arrow", "async-trait", @@ -1767,6 +1775,8 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "18.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "943e42356f0f6f5ac37ceacd412de9c4d7d8eba1e81b6f724f88699540c7f070" dependencies = [ "ahash 0.8.1", "arrow", @@ -1796,6 +1806,8 @@ dependencies = [ [[package]] name = "datafusion-proto" version = "18.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "649aec221737d8fb88956a2e6181297456f83b8560d32d775ad1cd22d67fd598" dependencies = [ "arrow", "chrono", @@ -1829,6 +1841,8 @@ dependencies = [ [[package]] name = "datafusion-row" version = "18.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6a506f5924f8af54e0806a995da0897f8c2b548d492793e045a3896d88d6714a" dependencies = [ "arrow", "datafusion-common", @@ -1839,6 +1853,8 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "18.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4a3d12047a5847f9667f4e2aa8fa2e7d5a6e1094b8e3546d58de492152a50dc7" dependencies = [ "arrow-schema", "datafusion-common", diff --git a/Cargo.toml b/Cargo.toml index 42e36be0..82049ef4 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -64,7 +64,7 @@ lazy_static = ">=1.4.0" log = "0.4" moka = { version = "0.10.0", default_features = false, features = ["future", "atomic64", "quanta"] } object_store = "0.5.2" -parking_lot = { version = "0.12.1", features = ["send_guard"] } +parking_lot = "0.12.1" percent-encoding = "2.2.0" pretty_env_logger = "0.4" prost = "0.11" @@ -100,10 +100,6 @@ arrow-array = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow arrow-buffer = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-buffer", branch = "backport-cast-utf8-to-timestamp-microsecond" } arrow-schema = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-schema", branch = "backport-cast-utf8-to-timestamp-microsecond" } arrow-cast = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-cast", branch = "backport-cast-utf8-to-timestamp-microsecond" } -datafusion = { path = "../arrow-datafusion/datafusion/core" } -datafusion-common = { path = "../arrow-datafusion/datafusion/common" } -datafusion-expr = { path = "../arrow-datafusion/datafusion/expr" } -datafusion-proto = { path = "../arrow-datafusion/datafusion/proto" } [dev-dependencies] assert_unordered = "0.3" diff --git a/src/frontend/http.rs b/src/frontend/http.rs index e1fb63ff..a5bbf45c 100644 --- a/src/frontend/http.rs +++ b/src/frontend/http.rs @@ -71,8 +71,6 @@ impl PlanVisitor for ETagBuilderVisitor { .as_any() .downcast_ref::() { - println!("uri: {}", table.table_uri()); - println!("version: {}", table.version()); self.table_versions .extend(table.table_uri().as_bytes().to_vec()); self.table_versions diff --git a/src/provider.rs b/src/provider.rs index 7bbaaa9e..5bca003f 100644 --- a/src/provider.rs +++ b/src/provider.rs @@ -133,9 +133,9 @@ impl SchemaProvider for SeafowlCollection { } } - // TODO: This is not very nice: rebuilding the table from scratch, instead of just `load` on + // TODO: This is kind of meh: rebuilding the table from scratch and over-writing the existing entry, instead of just `load`-ing // the existing one (which we can't because it's behind of an Arc, and `load` needs `mut`). - // We may be able to improve it by: + // We may be able get away with it by: // 1. removing the `Arc` from the value in the map // 2. enclosing the entire map inside of an `Arc` // 3. using `entry` for in-place mutation From 13ce11c0c762227c42f1b3e97283299fc65e80e4 Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Fri, 3 Mar 2023 12:15:24 +0100 Subject: [PATCH 09/21] Format Cargo.toml as per tomlfmt --- Cargo.toml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 82049ef4..611f86f2 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -95,11 +95,11 @@ wasmtime = "1.0.1" wasmtime-wasi = "1.0.1" [patch.crates-io] -arrow-data = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-data", branch = "backport-cast-utf8-to-timestamp-microsecond" } arrow-array = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-array", branch = "backport-cast-utf8-to-timestamp-microsecond" } arrow-buffer = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-buffer", branch = "backport-cast-utf8-to-timestamp-microsecond" } -arrow-schema = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-schema", branch = "backport-cast-utf8-to-timestamp-microsecond" } arrow-cast = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-cast", branch = "backport-cast-utf8-to-timestamp-microsecond" } +arrow-data = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-data", branch = "backport-cast-utf8-to-timestamp-microsecond" } +arrow-schema = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-schema", branch = "backport-cast-utf8-to-timestamp-microsecond" } [dev-dependencies] assert_unordered = "0.3" From 6838409e37c1b79d085b2e47f8aa4f3626e47026 Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Mon, 6 Mar 2023 22:19:35 +0100 Subject: [PATCH 10/21] Pivot to using a uuid as delta table root directory --- Cargo.lock | 2 + Cargo.toml | 3 +- ...0221081920_delta_lake_integration.down.sql | 1 + ...230221081920_delta_lake_integration.up.sql | 1 + ...0221081928_delta_lake_integration.down.sql | 1 + ...230221081928_delta_lake_integration.up.sql | 1 + src/catalog.rs | 18 +-- src/context.rs | 130 +++++++++--------- src/frontend/http.rs | 75 +++++----- src/object_store/wrapped.rs | 56 +------- src/repository/default.rs | 7 +- src/repository/interface.rs | 13 +- src/repository/postgres.rs | 1 + src/repository/sqlite.rs | 1 + tests/statements/ddl.rs | 2 +- 15 files changed, 136 insertions(+), 176 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b016128e..48996c07 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5134,6 +5134,7 @@ dependencies = [ "tempfile", "tokio", "url", + "uuid 1.2.1", "vergen", "warp", "wasi-common", @@ -5526,6 +5527,7 @@ dependencies = [ "thiserror", "tokio-stream", "url", + "uuid 1.2.1", "webpki-roots", "whoami", ] diff --git a/Cargo.toml b/Cargo.toml index 611f86f2..15bad75c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -81,12 +81,13 @@ serde = "1.0.138" serde_json = "1.0.93" sha2 = ">=0.10.1" sqlparser = "0.30.0" -sqlx = { version = "0.6.2", features = [ "runtime-tokio-rustls", "sqlite", "any" ] } +sqlx = { version = "0.6.2", features = [ "runtime-tokio-rustls", "sqlite", "any", "uuid" ] } strum = ">=0.24" strum_macros = ">=0.24" tempfile = "3" tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "signal", "process"] } url = "2.2" +uuid = "1.2.1" warp = "0.3" wasi-common = "1.0.1" diff --git a/migrations/postgres/20230221081920_delta_lake_integration.down.sql b/migrations/postgres/20230221081920_delta_lake_integration.down.sql index 6577cf3a..eb408f1e 100644 --- a/migrations/postgres/20230221081920_delta_lake_integration.down.sql +++ b/migrations/postgres/20230221081920_delta_lake_integration.down.sql @@ -1,2 +1,3 @@ -- Add down migration script here ALTER TABLE "table" DROP COLUMN legacy; +ALTER TABLE "table" DROP COLUMN uuid; diff --git a/migrations/postgres/20230221081920_delta_lake_integration.up.sql b/migrations/postgres/20230221081920_delta_lake_integration.up.sql index a2b6031b..ee8247cb 100644 --- a/migrations/postgres/20230221081920_delta_lake_integration.up.sql +++ b/migrations/postgres/20230221081920_delta_lake_integration.up.sql @@ -1,4 +1,5 @@ -- Add up migration script here ALTER TABLE "table" ADD COLUMN legacy BOOLEAN DEFAULT FALSE; +ALTER TABLE "table" ADD COLUMN uuid UUID NOT NULL DEFAULT '00000000-0000-0000-0000-000000000000'; UPDATE "table" SET legacy = TRUE; diff --git a/migrations/sqlite/20230221081928_delta_lake_integration.down.sql b/migrations/sqlite/20230221081928_delta_lake_integration.down.sql index 6577cf3a..eb408f1e 100644 --- a/migrations/sqlite/20230221081928_delta_lake_integration.down.sql +++ b/migrations/sqlite/20230221081928_delta_lake_integration.down.sql @@ -1,2 +1,3 @@ -- Add down migration script here ALTER TABLE "table" DROP COLUMN legacy; +ALTER TABLE "table" DROP COLUMN uuid; diff --git a/migrations/sqlite/20230221081928_delta_lake_integration.up.sql b/migrations/sqlite/20230221081928_delta_lake_integration.up.sql index a2b6031b..3de5fe2f 100644 --- a/migrations/sqlite/20230221081928_delta_lake_integration.up.sql +++ b/migrations/sqlite/20230221081928_delta_lake_integration.up.sql @@ -1,4 +1,5 @@ -- Add up migration script here ALTER TABLE "table" ADD COLUMN legacy BOOLEAN DEFAULT FALSE; +ALTER TABLE "table" ADD COLUMN uuid VARCHAR NOT NULL DEFAULT '00000000-0000-0000-0000-000000000000'; UPDATE "table" SET legacy = TRUE; diff --git a/src/catalog.rs b/src/catalog.rs index 209b78ea..0e6742be 100644 --- a/src/catalog.rs +++ b/src/catalog.rs @@ -10,6 +10,7 @@ use itertools::Itertools; #[cfg(test)] use mockall::automock; use parking_lot::RwLock; +use uuid::Uuid; use crate::object_store::wrapped::InternalObjectStore; use crate::provider::SeafowlFunction; @@ -196,6 +197,7 @@ pub trait TableCatalog: Sync + Send { collection_id: CollectionId, table_name: &str, schema: &Schema, + uuid: Uuid, ) -> Result<(TableId, TableVersionId)>; async fn delete_old_table_versions( @@ -385,15 +387,12 @@ impl DefaultCatalog { // Build a delta table but don't load it yet; we'll do that only for tables that are // actually referenced in a statement, via the async `table` method of the schema provider. - // TODO: if the table has no columns, the result set will be empty, so we use the default DB and schema names. - let (database_name, collection_name) = table_columns_vec.get(0).map_or_else( - || (DEFAULT_DB, DEFAULT_SCHEMA), - |v| (&v.database_name, &v.collection_name), - ); + // TODO: if the table has no columns, the result set will be empty, so we use the default UUID (all zeros). + let table_uuid = table_columns_vec + .get(0) + .map_or_else(Uuid::default, |v| v.table_uuid); - let table_object_store = - self.object_store - .for_delta_table(database_name, collection_name, table_name); + let table_object_store = self.object_store.for_delta_table(table_uuid); let table = DeltaTable::new(table_object_store, Default::default()); (Arc::from(table_name.to_string()), Arc::new(table) as _) @@ -509,9 +508,10 @@ impl TableCatalog for DefaultCatalog { collection_id: CollectionId, table_name: &str, schema: &Schema, + uuid: Uuid, ) -> Result<(TableId, TableVersionId)> { self.repository - .create_table(collection_id, table_name, schema) + .create_table(collection_id, table_name, schema, uuid) .await .map_err(|e| match e { RepositoryError::UniqueConstraintViolation(_) => { diff --git a/src/context.rs b/src/context.rs index 22e24222..6078c99f 100644 --- a/src/context.rs +++ b/src/context.rs @@ -93,10 +93,13 @@ use parking_lot::RwLock; use prost::Message; use tempfile::TempPath; use tokio::sync::Semaphore; +use uuid::Uuid; use crate::catalog::{PartitionCatalog, DEFAULT_SCHEMA, STAGING_SCHEMA}; use crate::data_types::PhysicalPartitionId; use crate::datafusion::visit::VisitorMut; +#[cfg(test)] +use crate::frontend::http::tests::deterministic_uuid; use crate::provider::{ project_expressions, PartitionColumn, SeafowlPartition, SeafowlPruningStatistics, SeafowlTable, @@ -724,6 +727,40 @@ impl DefaultSeafowlContext { Ok(seafowl_table.clone()) } + // Parse the uuid from the Delta table uri if available + async fn get_table_uuid<'a>( + &self, + name: impl Into>, + ) -> Result { + match self + .inner + .table_provider(name) + .await? + .as_any() + .downcast_ref::() + { + None => { + // TODO: try to load from DB if missing? + Err(DataFusionError::Execution( + "Couldn't fetch table uuid".to_string(), + )) + } + Some(delta_table) => { + let table_uri = Path::from(delta_table.table_uri()); + let uuid = table_uri.parts().last().ok_or_else(|| { + DataFusionError::Execution(format!( + "Failed parsing the uuid suffix from uri {table_uri} for table {delta_table}" + )) + })?; + Ok(Uuid::try_parse(uuid.as_ref()).map_err(|err| { + DataFusionError::Execution(format!( + "Failed parsing uuid from {uuid:?}: {err}" + )) + })?) + } + } + } + async fn create_delta_table<'a>( &self, name: impl Into>, @@ -746,11 +783,18 @@ impl DefaultSeafowlContext { })?; let delta_schema = DeltaSchema::try_from(schema)?; - let table_object_store = self.internal_object_store.for_delta_table( - &resolved_ref.catalog, - &resolved_ref.schema, - &resolved_ref.table, - ); + + // TODO: we could be doing this inside the DB itself (i.e. `... DEFAULT gen_random_uuid()` + // in Postgres and `... DEFAULT (uuid())` in SQLite) however we won't be able to do it until + // sqlx 0.7 is released (which has libsqlite3-sys > 0.25, with the SQLite version that has + // the `uuid()` function). + // Then we could create the table in our catalog first and try to create the delta table itself + // with the returned uuid (and delete the catalog entry if the object store creation fails). + #[cfg(test)] + let table_uuid = deterministic_uuid(); + #[cfg(not(test))] + let table_uuid = Uuid::new_v4(); + let table_object_store = self.internal_object_store.for_delta_table(table_uuid); let table = CreateBuilder::new() .with_object_store(table_object_store) @@ -762,11 +806,12 @@ impl DefaultSeafowlContext { )) .await?; - // We still persist the table into our own catalog, the principal reason is us being able to - // load all tables and their schemas in bulk to satisfy information_schema queries. - // We should look into doing this via delta-rs somehow eventually. + // We still persist the table into our own catalog, one reason is us being able to load all + // tables and their schemas in bulk to satisfy information_schema queries. + // Another is to keep track of table uuid's, which are used to construct the table uri. + // We may look into doing this via delta-rs somehow eventually. self.table_catalog - .create_table(collection_id, &table_name, &sf_schema) + .create_table(collection_id, &table_name, &sf_schema, table_uuid) .await?; debug!("Created new table {table}"); @@ -779,14 +824,8 @@ impl DefaultSeafowlContext { plan: &Arc, name: impl Into>, ) -> Result { - let table_ref: TableReference = name.into(); - let resolved_ref = table_ref.resolve(&self.database, DEFAULT_SCHEMA); - - let table_object_store = self.internal_object_store.for_delta_table( - &resolved_ref.catalog, - &resolved_ref.schema, - &resolved_ref.table, - ); + let table_uuid = self.get_table_uuid(name).await?; + let table_object_store = self.internal_object_store.for_delta_table(table_uuid); let table = WriteBuilder::new() .with_input_execution_plan(plan.clone()) @@ -1085,8 +1124,9 @@ impl SeafowlContext for DefaultSeafowlContext { tables_by_version[table_version_id].clone() } else { // We only support datetime DeltaTable version specification for start + let table_uuid = self.get_table_uuid(resolved_ref.clone()).await?; let table_object_store = - self.internal_object_store.for_delta_table(&resolved_ref.catalog, &resolved_ref.schema, &resolved_ref.table); + self.internal_object_store.for_delta_table(table_uuid); let datetime = DateTime::::from(DateTime::::parse_from_rfc3339(version).map_err(|_| DataFusionError::Execution(format!( "Failed to parse version {version} as RFC3339 timestamp" )))?); @@ -1415,6 +1455,7 @@ impl SeafowlContext for DefaultSeafowlContext { let _table = self .create_delta_table(name, physical.schema().as_ref()) .await?; + self.reload_schema().await?; self.plan_to_delta_table(&physical, name).await?; Ok(make_dummy_exec()) @@ -1692,22 +1733,6 @@ impl SeafowlContext for DefaultSeafowlContext { DataFusionError::Execution("Table {name} not found".to_string()) })?; - // TODO: delay the actual delete and perform it during the subsequent vacuuming - // somehow, or when a new CREATE TABLE targets the same uri simply overwrite - let table_prefix = Path::from(format!( - "{}/{}/{}", - &resolved_ref.catalog, &resolved_ref.schema, &resolved_ref.table - )); - - self.internal_object_store - .delete_in_prefix(&table_prefix) - .await - .map_err(|err| { - DataFusionError::Execution(format!( - "Failed to delete table {name} : {}", - err - )) - })?; self.table_catalog.drop_table(table_id).await?; Ok(make_dummy_exec()) } @@ -1758,23 +1783,11 @@ impl SeafowlContext for DefaultSeafowlContext { .to_string(), )); } - let new_path_prefix = Path::from(format!( - "{}/{}/{}", - &resolved_new_ref.catalog, - &resolved_new_ref.schema, - &resolved_new_ref.table - )); // Resolve old table reference and fetch the table id let old_table_ref = TableReference::from(old_name.as_str()); let resolved_old_ref = old_table_ref.resolve(&self.database, DEFAULT_SCHEMA); - let old_path_prefix = Path::from(format!( - "{}/{}/{}", - &resolved_old_ref.catalog, - &resolved_old_ref.schema, - &resolved_old_ref.table - )); let table_id = self .table_catalog @@ -1812,28 +1825,6 @@ impl SeafowlContext for DefaultSeafowlContext { None }; - // If we're using the local FS store ensure the full Delta log folder - // path exists prior to the move - self.internal_object_store - .ensure_path_exists(&new_path_prefix.child("_delta_log")) - .await - .map_err(|err| { - DataFusionError::Execution(format!( - "Failed to ensure path existence for table {new_name} : {}", - err - )) - })?; - // Iterate over all objects in the old table path and rename to new path - self.internal_object_store - .rename_in_prefix(&old_path_prefix, &new_path_prefix) - .await - .map_err(|err| { - DataFusionError::Execution(format!( - "Failed to rename table objects in {old_path_prefix} to {new_path_prefix}: {}", - err - )) - })?; - // Finally update our catalog entry self.table_catalog .move_table( @@ -1974,6 +1965,9 @@ impl SeafowlContext for DefaultSeafowlContext { if !table_exists { self.create_delta_table(table_ref.clone(), plan.schema().as_ref()) .await?; + // TODO: This is really only needed here and for CREATE TABLE AS statements only to be + // able to get the uuid without hitting the catalog DB in `get_table_uuid` + self.reload_schema().await?; } self.plan_to_delta_table(&plan, table_ref).await?; diff --git a/src/frontend/http.rs b/src/frontend/http.rs index a5bbf45c..450f358a 100644 --- a/src/frontend/http.rs +++ b/src/frontend/http.rs @@ -551,7 +551,7 @@ pub async fn run_server( } #[cfg(test)] -mod tests { +pub mod tests { use bytes::Bytes; use itertools::Itertools; @@ -559,7 +559,13 @@ mod tests { use std::fmt::Display; use std::{collections::HashMap, sync::Arc}; + use rand::{rngs::mock::StepRng, Rng}; + use std::cell::RefCell; + use uuid::Builder; + thread_local!(static STEP_RNG: RefCell = RefCell::new(StepRng::new(1, 1))); + use rstest::rstest; + use uuid::Uuid; use warp::{Filter, Rejection, Reply}; use warp::http::Response; @@ -685,14 +691,18 @@ mod tests { const CREATE_QUERY: &str = "CREATE TABLE other_test_table(col_1 INT)"; const SELECT_QUERY_HASH: &str = "7fbbf7dddfd330d03e5e08cc5885ad8ca823e1b56e7cbadd156daa0e21c288f6"; - const V1_ETAG_DEFAULT_DB: &str = - "c2d5a6434eb700587e0e95a5d22015e2616873d22353e83ce436075f5e01e740"; - const V1_ETAG_TEST_DB: &str = - "b044e9f114a6627dd2449496514e938234dc1ff6e400cb177fe38b3ed977619b"; - const V2_ETAG_DEFAULT_DB: &str = - "780c8dc5587a5a8cd32a4fabcdf1425e3c6f6c859cd24b7963a3007f450ce55e"; - const V2_ETAG_TEST_DB: &str = - "26e1be6caca4a0fefb798b7d197033f6f181719f59042e87d95754bcf2ebdea0"; + const V1_ETAG: &str = + "1230e7ce41e2f7c2050b75e36b6f313f5cc4dd99b255f2761f589d60a44eee00"; + const V2_ETAG: &str = + "b17259a6a4e10c9a8b42ce23e683b919ada82b2ed1fafbbcd10ff42c63ff2443"; + + pub fn deterministic_uuid() -> Uuid { + // A crude hack to get reproducible bytes as source for UUID generation, to enable etag asserts + STEP_RNG.with(|rng| { + let bytes: [u8; 16] = rng.borrow_mut().gen(); + Builder::from_random_bytes(bytes).into_uuid() + }) + } #[rstest] #[tokio::test] @@ -766,10 +776,10 @@ mod tests { } #[rstest] - #[case(None, V1_ETAG_DEFAULT_DB)] - #[case(Some("test_db"), V1_ETAG_TEST_DB)] #[tokio::test] - async fn test_get_cached_no_etag(#[case] new_db: Option<&str>, #[case] etag: &str) { + async fn test_get_cached_no_etag( + #[values(None, Some("test_db"))] new_db: Option<&str>, + ) { let context = in_memory_context_with_single_table(new_db).await; let handler = filters(context, http_config_from_access_policy(free_for_all())); @@ -783,7 +793,7 @@ mod tests { assert_eq!(resp.body(), "{\"c\":1}\n"); assert_eq!( resp.headers().get(header::ETAG).unwrap().to_str().unwrap(), - etag, + V1_ETAG, ); } @@ -807,12 +817,9 @@ mod tests { } #[rstest] - #[case(None, V1_ETAG_DEFAULT_DB)] - #[case(Some("test_db"), V1_ETAG_TEST_DB)] #[tokio::test] async fn test_get_cached_no_etag_query_in_body( - #[case] new_db: Option<&str>, - #[case] etag: &str, + #[values(None, Some("test_db"))] new_db: Option<&str>, ) { let context = in_memory_context_with_single_table(new_db).await; let handler = filters(context, http_config_from_access_policy(free_for_all())); @@ -827,17 +834,14 @@ mod tests { assert_eq!(resp.body(), "{\"c\":1}\n"); assert_eq!( resp.headers().get(header::ETAG).unwrap().to_str().unwrap(), - etag + V1_ETAG ); } #[rstest] - #[case(None, V1_ETAG_DEFAULT_DB)] - #[case(Some("test_db"), V1_ETAG_TEST_DB)] #[tokio::test] async fn test_get_cached_no_etag_extension( - #[case] new_db: Option<&str>, - #[case] etag: &str, + #[values(None, Some("test_db"))] new_db: Option<&str>, ) { let context = in_memory_context_with_single_table(new_db).await; let handler = filters(context, http_config_from_access_policy(free_for_all())); @@ -852,19 +856,16 @@ mod tests { assert_eq!(resp.body(), "{\"c\":1}\n"); assert_eq!( resp.headers().get(header::ETAG).unwrap().to_str().unwrap(), - etag + V1_ETAG ); } #[rstest] - #[case(None, V1_ETAG_DEFAULT_DB)] - #[case(Some("test_db"), V1_ETAG_TEST_DB)] #[tokio::test] async fn test_get_cached_reuse_etag( - #[case] new_db: Option<&str>, - #[case] etag: &str, + #[values(None, Some("test_db"))] new_db: Option<&str>, ) { - // Pass the same ETag as If-None-Match, should return a 301 + // Pass the same ETag as If-None-Match, should return a 304 let context = in_memory_context_with_single_table(new_db).await; let handler = filters(context, http_config_from_access_policy(free_for_all())); @@ -873,7 +874,7 @@ mod tests { .method("GET") .path(make_uri(format!("/q/{SELECT_QUERY_HASH}"), new_db).as_str()) .header(QUERY_HEADER, SELECT_QUERY) - .header(IF_NONE_MATCH, etag) + .header(IF_NONE_MATCH, V1_ETAG) .reply(&handler) .await; assert_eq!(resp.status(), StatusCode::NOT_MODIFIED); @@ -881,12 +882,9 @@ mod tests { } #[rstest] - #[case(None, V1_ETAG_DEFAULT_DB)] - #[case(Some("test_db"), V1_ETAG_TEST_DB)] #[tokio::test] async fn test_get_encoded_query_special_chars( - #[case] new_db: Option<&str>, - #[case] etag: &str, + #[values(None, Some("test_db"))] new_db: Option<&str>, ) { let context = in_memory_context_with_single_table(new_db).await; let handler = filters(context, http_config_from_access_policy(free_for_all())); @@ -901,7 +899,7 @@ mod tests { assert_eq!(resp.body(), "{\"c\":1}\n"); assert_eq!( resp.headers().get(header::ETAG).unwrap().to_str().unwrap(), - etag + V1_ETAG ); } @@ -924,12 +922,9 @@ mod tests { } #[rstest] - #[case(None, V2_ETAG_DEFAULT_DB)] - #[case(Some("test_db"), V2_ETAG_TEST_DB)] #[tokio::test] async fn test_get_cached_etag_new_version( - #[case] new_db: Option<&str>, - #[case] etag: &str, + #[values(None, Some("test_db"))] new_db: Option<&str>, ) { // Pass the same ETag as If-None-Match, but the table version changed -> reruns the query @@ -940,14 +935,14 @@ mod tests { .method("GET") .path(make_uri(format!("/q/{SELECT_QUERY_HASH}"), new_db).as_str()) .header(QUERY_HEADER, SELECT_QUERY) - .header(header::ETAG, V1_ETAG_DEFAULT_DB) + .header(header::ETAG, V1_ETAG) .reply(&handler) .await; assert_eq!(resp.status(), StatusCode::OK); assert_eq!(resp.body(), "{\"c\":2}\n"); assert_eq!( resp.headers().get(header::ETAG).unwrap().to_str().unwrap(), - etag + V2_ETAG ); } diff --git a/src/object_store/wrapped.rs b/src/object_store/wrapped.rs index 400e8593..9681d901 100644 --- a/src/object_store/wrapped.rs +++ b/src/object_store/wrapped.rs @@ -11,15 +11,15 @@ use std::fmt::{Debug, Display, Formatter}; use std::ops::Range; use tokio::io::AsyncWrite; -use tokio::fs::{copy, create_dir_all, remove_file, rename}; +use tokio::fs::{copy, remove_file, rename}; use deltalake::storage::DeltaObjectStore; -use itertools::Itertools; use object_store::prefix::PrefixObjectStore; use std::path::Path as StdPath; use std::str::FromStr; use std::sync::Arc; use url::Url; +use uuid::Uuid; /// Wrapper around the object_store crate that holds on to the original config /// in order to provide a more efficient "upload" for the local object store (since it's @@ -63,19 +63,13 @@ impl InternalObjectStore { // `InternalObjectStore` comes in. // This does mean that we have 3 layers of indirection before we hit the "real" object store // (`DeltaObjectStore` -> `PrefixObjectStore` -> `InternalObjectStore` -> `inner`). - pub fn for_delta_table( - &self, - database_name: &str, - collection_name: &str, - table_name: &str, - ) -> Arc { - let prefix = format!("{}/{}/{}", database_name, collection_name, table_name); + pub fn for_delta_table(&self, table_uuid: Uuid) -> Arc { let prefixed_store: PrefixObjectStore = - PrefixObjectStore::new(self.clone(), prefix.clone()); + PrefixObjectStore::new(self.clone(), table_uuid.to_string()); Arc::from(DeltaObjectStore::new( Arc::from(prefixed_store), - Url::from_str(format!("{}/{}", self.root_uri.as_str(), prefix).as_str()) + Url::from_str(format!("{}/{}", self.root_uri.as_str(), table_uuid).as_str()) .unwrap(), )) } @@ -91,46 +85,6 @@ impl InternalObjectStore { Ok(()) } - /// Create any missing intermediate directories in the given path, so that further actions don't - /// error out. Applicable only to local FS, since in other stores the directories are virtual. - pub async fn ensure_path_exists(&self, path: &Path) -> Result<(), Error> { - if let schema::ObjectStore::Local(_) = self.config.clone() { - let full_path = StdPath::new(self.root_uri.path()).join(path.to_string()); - create_dir_all(full_path) - .await - .map_err(|e| Error::Generic { - store: "local", - source: Box::new(e), - })?; - } - - Ok(()) - } - - /// Moving all objects in paths with `from_prefix` to paths with `to_prefix`. The main purpose - /// of this is to ensure that `to_prefix` actually exists when using the local FS store, otherwise - /// we get "No such file or directory" on rename. - pub async fn rename_in_prefix( - &self, - from_prefix: &Path, - to_prefix: &Path, - ) -> Result<(), Error> { - // Go over all objects with the `from_prefix` prefix and rename them to be with `to_prefix` - let mut path_stream = self.inner.list(Some(from_prefix)).await?; - - while let Some(maybe_object) = path_stream.next().await { - if let Ok(ObjectMeta { location, .. }) = maybe_object { - // We unwrap since the path must match the from prefix - let mut new_path_parts = to_prefix.parts().collect_vec(); - new_path_parts - .extend(location.prefix_match(from_prefix).unwrap().collect_vec()); - let new_location = Path::from_iter(new_path_parts); - self.inner.rename(&location, &new_location).await?; - } - } - Ok(()) - } - /// For local filesystem object stores, try "uploading" by just moving the file. /// Returns a None if the store isn't local. pub async fn fast_upload( diff --git a/src/repository/default.rs b/src/repository/default.rs index a2f10fb9..3ba1d741 100644 --- a/src/repository/default.rs +++ b/src/repository/default.rs @@ -113,6 +113,7 @@ impl Repository for $repo { collection.name AS collection_name, "table".name AS table_name, "table".id AS table_id, + "table".uuid AS table_uuid, "table".legacy AS table_legacy, desired_table_versions.id AS table_version_id, table_column.name AS column_name, @@ -215,7 +216,7 @@ impl Repository for $repo { database_name: &str, collection_name: &str, table_name: &str, - ) -> Result { + ) -> Result { let id = sqlx::query( r#" SELECT "table".id @@ -266,13 +267,15 @@ impl Repository for $repo { collection_id: CollectionId, table_name: &str, schema: &Schema, + uuid: Uuid, ) -> Result<(TableId, TableVersionId), Error> { // Create new (empty) table let new_table_id: i64 = sqlx::query( - r#"INSERT INTO "table" (collection_id, name) VALUES ($1, $2) RETURNING (id)"#, + r#"INSERT INTO "table" (collection_id, name, uuid) VALUES ($1, $2, $3) RETURNING (id)"#, ) .bind(collection_id) .bind(table_name) + .bind(uuid) .fetch_one(&self.executor) .await.map_err($repo::interpret_error)? .try_get("id").map_err($repo::interpret_error)?; diff --git a/src/repository/interface.rs b/src/repository/interface.rs index 91c3e245..678072a6 100644 --- a/src/repository/interface.rs +++ b/src/repository/interface.rs @@ -1,6 +1,7 @@ use std::fmt::Debug; use async_trait::async_trait; +use uuid::Uuid; use crate::wasm_udf::data_types::CreateFunctionDetails; use crate::{ @@ -18,6 +19,7 @@ pub struct AllDatabaseColumnsResult { pub collection_name: String, pub table_name: String, pub table_id: TableId, + pub table_uuid: Uuid, pub table_legacy: bool, pub table_version_id: TableVersionId, pub column_name: String, @@ -117,7 +119,7 @@ pub trait Repository: Send + Sync + Debug { database_name: &str, collection_name: &str, table_name: &str, - ) -> Result; + ) -> Result; async fn get_all_database_ids(&self) -> Result, Error>; @@ -134,6 +136,7 @@ pub trait Repository: Send + Sync + Debug { collection_id: CollectionId, table_name: &str, schema: &Schema, + uuid: Uuid, ) -> Result<(TableId, TableVersionId), Error>; async fn delete_old_table_versions( @@ -275,7 +278,7 @@ pub mod tests { }; let (table_id, table_version_id) = repository - .create_table(collection_id, "testtable", &schema) + .create_table(collection_id, "testtable", &schema, Uuid::default()) .await .expect("Error creating table"); @@ -316,6 +319,7 @@ pub mod tests { collection_name: collection_name.clone(), table_name: table_name.clone(), table_id: 1, + table_uuid: Default::default(), table_legacy: false, table_version_id: version, column_name: "date".to_string(), @@ -326,6 +330,7 @@ pub mod tests { collection_name, table_name, table_id: 1, + table_uuid: Default::default(), table_legacy: false, table_version_id: version, column_name: "value".to_string(), @@ -630,7 +635,7 @@ pub mod tests { assert!(matches!( repository - .create_table(collection_id_2, "testtable2", &schema) + .create_table(collection_id_2, "testtable2", &schema, Uuid::default()) .await .unwrap_err(), Error::UniqueConstraintViolation(_) @@ -638,7 +643,7 @@ pub mod tests { // Make a new table in the previous collection, try renaming let (new_table_id, _) = repository - .create_table(collection_id_1, "testtable2", &schema) + .create_table(collection_id_1, "testtable2", &schema, Uuid::default()) .await .unwrap(); diff --git a/src/repository/postgres.rs b/src/repository/postgres.rs index 58765510..620ef6a1 100644 --- a/src/repository/postgres.rs +++ b/src/repository/postgres.rs @@ -7,6 +7,7 @@ use sqlx::{ postgres::PgPoolOptions, Executor, PgPool, Postgres, QueryBuilder, Row, }; +use uuid::Uuid; use crate::{ data_types::{ diff --git a/src/repository/sqlite.rs b/src/repository/sqlite.rs index 26aadd9a..0337690a 100644 --- a/src/repository/sqlite.rs +++ b/src/repository/sqlite.rs @@ -8,6 +8,7 @@ use sqlx::{ sqlite::{SqliteConnectOptions, SqlitePoolOptions}, Pool, QueryBuilder, Row, Sqlite, }; +use uuid::Uuid; use crate::{ data_types::{ diff --git a/tests/statements/ddl.rs b/tests/statements/ddl.rs index eaaa997f..174b1fb9 100644 --- a/tests/statements/ddl.rs +++ b/tests/statements/ddl.rs @@ -358,7 +358,7 @@ async fn test_create_table_schema_already_exists() { .unwrap_err(); assert_eq!( err.to_string(), - "External error: Generic error: A Delta Lake table already exists at that location." + "Error during planning: Table \"some_table\" already exists" ); let err = context From aead9eb62ea26177b823c77d9d54f6cdbfdf0f0a Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Tue, 7 Mar 2023 14:09:19 +0100 Subject: [PATCH 11/21] Test legacy tables by bootstraping from an existing catalog --- ...230221081928_delta_lake_integration.up.sql | 6 +- src/context.rs | 3 +- src/frontend/http.rs | 5 +- src/repository/default.rs | 1 + src/repository/interface.rs | 1 + src/system_tables.rs | 11 +- ...4cbfb07c2d8d7116f3d60d28bc02900861.parquet | Bin 0 -> 541 bytes ...1bd9ae7ab9a7a78ee541b66209cfd7722d.parquet | Bin 0 -> 1402 bytes ...3ca5ea8e703d141b14835025923a66ab50.parquet | Bin 0 -> 541 bytes ...8e2c260ece02a810e5f1e2be41b0d6c0f6.parquet | Bin 0 -> 541 bytes tests/data/seafowl-legacy-data/seafowl.sqlite | Bin 0 -> 4096 bytes .../seafowl-legacy-data/seafowl.sqlite-shm | Bin 0 -> 32768 bytes .../seafowl-legacy-data/seafowl.sqlite-wal | Bin 0 -> 280192 bytes tests/statements/mod.rs | 2 +- tests/statements/query_legacy.rs | 355 ++++++++++++++++++ 15 files changed, 377 insertions(+), 7 deletions(-) create mode 100644 tests/data/seafowl-legacy-data/534e5cc396e5b24725993145821b864cbfb07c2d8d7116f3d60d28bc02900861.parquet create mode 100644 tests/data/seafowl-legacy-data/7fbfeeeade71978b4ae82cd3d97b8c1bd9ae7ab9a7a78ee541b66209cfd7722d.parquet create mode 100644 tests/data/seafowl-legacy-data/9ae6f4222893474551037d0e44ff223ca5ea8e703d141b14835025923a66ab50.parquet create mode 100644 tests/data/seafowl-legacy-data/ea192fa7ae3b4abca9ded70e480c188e2c260ece02a810e5f1e2be41b0d6c0f6.parquet create mode 100644 tests/data/seafowl-legacy-data/seafowl.sqlite create mode 100644 tests/data/seafowl-legacy-data/seafowl.sqlite-shm create mode 100644 tests/data/seafowl-legacy-data/seafowl.sqlite-wal create mode 100644 tests/statements/query_legacy.rs diff --git a/migrations/sqlite/20230221081928_delta_lake_integration.up.sql b/migrations/sqlite/20230221081928_delta_lake_integration.up.sql index 3de5fe2f..019e86d4 100644 --- a/migrations/sqlite/20230221081928_delta_lake_integration.up.sql +++ b/migrations/sqlite/20230221081928_delta_lake_integration.up.sql @@ -1,5 +1,9 @@ -- Add up migration script here ALTER TABLE "table" ADD COLUMN legacy BOOLEAN DEFAULT FALSE; -ALTER TABLE "table" ADD COLUMN uuid VARCHAR NOT NULL DEFAULT '00000000-0000-0000-0000-000000000000'; +-- The main reason to go with BLOB (which isn't human readable) instead of TEXT is not performance but instead the lack +-- of support for decoding TEXT to uuid::Uuid in SQLite by sqlx: https://github.com/launchbadge/sqlx/issues/1083 +-- On the other hand, while decoding TEXT to uuid::fmt::Hyphenated is supported in SQLite it isn't in Postgres, so this is +-- the only approach that works for now. +ALTER TABLE "table" ADD COLUMN uuid BLOB NOT NULL DEFAULT x'00000000000000000000000000000000'; UPDATE "table" SET legacy = TRUE; diff --git a/src/context.rs b/src/context.rs index 6078c99f..b3f6dc02 100644 --- a/src/context.rs +++ b/src/context.rs @@ -790,6 +790,7 @@ impl DefaultSeafowlContext { // the `uuid()` function). // Then we could create the table in our catalog first and try to create the delta table itself // with the returned uuid (and delete the catalog entry if the object store creation fails). + // On the other hand that would complicate etag testing logic. #[cfg(test)] let table_uuid = deterministic_uuid(); #[cfg(not(test))] @@ -1106,7 +1107,7 @@ impl SeafowlContext for DefaultSeafowlContext { // We now have table_version_ids for each table with version specified; do another // run over the query AST to rewrite the table. version_processor.visit_query(&mut q); - println!("Time travel query rewritten to: {}", q); + debug!("Time travel query rewritten to: {}", q); let tables_by_version = self .table_catalog diff --git a/src/frontend/http.rs b/src/frontend/http.rs index 450f358a..758efebc 100644 --- a/src/frontend/http.rs +++ b/src/frontend/http.rs @@ -697,7 +697,8 @@ pub mod tests { "b17259a6a4e10c9a8b42ce23e683b919ada82b2ed1fafbbcd10ff42c63ff2443"; pub fn deterministic_uuid() -> Uuid { - // A crude hack to get reproducible bytes as source for UUID generation, to enable etag asserts + // A crude hack to get reproducible bytes as source for table UUID generation, to enable + // transparent etag asserts STEP_RNG.with(|rng| { let bytes: [u8; 16] = rng.borrow_mut().gen(); Builder::from_random_bytes(bytes).into_uuid() @@ -793,7 +794,7 @@ pub mod tests { assert_eq!(resp.body(), "{\"c\":1}\n"); assert_eq!( resp.headers().get(header::ETAG).unwrap().to_str().unwrap(), - V1_ETAG, + V1_ETAG ); } diff --git a/src/repository/default.rs b/src/repository/default.rs index 3ba1d741..9aabb712 100644 --- a/src/repository/default.rs +++ b/src/repository/default.rs @@ -514,6 +514,7 @@ impl Repository for $repo { database.name AS database_name, collection.name AS collection_name, "table".name AS table_name, + "table".legacy AS table_legacy, table_version.id AS table_version_id, physical_partition.id AS table_partition_id, physical_partition.object_storage_id, diff --git a/src/repository/interface.rs b/src/repository/interface.rs index 678072a6..1e9175c1 100644 --- a/src/repository/interface.rs +++ b/src/repository/interface.rs @@ -41,6 +41,7 @@ pub struct TablePartitionsResult { pub database_name: String, pub collection_name: String, pub table_name: String, + pub table_legacy: bool, pub table_version_id: TableVersionId, pub table_partition_id: Option, pub object_storage_id: Option, diff --git a/src/system_tables.rs b/src/system_tables.rs index 79fecaa9..ea34d7ce 100644 --- a/src/system_tables.rs +++ b/src/system_tables.rs @@ -2,6 +2,7 @@ //! and datafusion's information_schema. use crate::catalog::TableCatalog; +use crate::repository::interface::{TablePartitionsResult, TableVersionsResult}; use arrow::array::{ Int32Builder, Int64Builder, StringBuilder, StructBuilder, TimestampSecondBuilder, }; @@ -167,7 +168,10 @@ impl SeafowlSystemTable for TableVersionsTable { let table_versions = self .table_catalog .get_all_table_versions(&self.database, None) - .await?; + .await? + .into_iter() + .filter(|tv| tv.table_legacy) + .collect::>(); let mut builder = StructBuilder::from_fields( self.schema.fields().clone(), @@ -239,7 +243,10 @@ impl SeafowlSystemTable for TablePartitionsTable { let table_partitions = self .table_catalog .get_all_table_partitions(&self.database) - .await?; + .await? + .into_iter() + .filter(|tv| tv.table_legacy) + .collect::>(); let mut builder = StructBuilder::from_fields( self.schema.fields().clone(), diff --git a/tests/data/seafowl-legacy-data/534e5cc396e5b24725993145821b864cbfb07c2d8d7116f3d60d28bc02900861.parquet b/tests/data/seafowl-legacy-data/534e5cc396e5b24725993145821b864cbfb07c2d8d7116f3d60d28bc02900861.parquet new file mode 100644 index 0000000000000000000000000000000000000000..f2ba541c14007c2b5fea2fa4fca37063feababaa GIT binary patch literal 541 zcmbtS%Syvg5S?7hMWl-=y_a0bB8wV|)IJfcyNQaekJ=WrwhK$BAqv)N(uZH*$A}9N z{2Bj6aBkW{_a3-2b7nGg<~0mqbFCV&`0N0b2ms@4_$CJRoFUfDAp`6(G{W$K0{0H|`T9j#$@>TFXrA zShucppsJ&XM*%h$C+my-uAbL)&}d!6oT?MV0Np_67)v$sfjB5v$Gz)nx!ZXy^~fQZ h>e&llCjNLgjO?i&Mh~MWyHd^<^F_Ed0Gjk;`~g#uZlVAH literal 0 HcmV?d00001 diff --git a/tests/data/seafowl-legacy-data/7fbfeeeade71978b4ae82cd3d97b8c1bd9ae7ab9a7a78ee541b66209cfd7722d.parquet b/tests/data/seafowl-legacy-data/7fbfeeeade71978b4ae82cd3d97b8c1bd9ae7ab9a7a78ee541b66209cfd7722d.parquet new file mode 100644 index 0000000000000000000000000000000000000000..aad7d38de33eedb81fa9f3c03f3513a344e0f9b0 GIT binary patch literal 1402 zcmb_cPiqrF6o0!(){R7|r8Dlr9(vHFNKH2_g;dhwqzNXrHBG^!_L6R5Yzs+Kle7Up zK=Dxg5S}DpZ+aJ`Nbw{+_8=mD1O-9x&1@!rY)D}h>NUV#-Vet*1g?M(74Dt{4hfJkVozf32rkW9*Ja$$F<*)Yk_AxFqbaKI z1`xkF3)+ZjTy4~<<-*QxvvA;6_RAzhHF~VjkBYr%>y|MDFgnBs2#B=fgS6k_buX*0 ze$Jmzc%M1f7f^qTj9f>3k-3Z*diRVhe>n4?VBXxRc1eFy>@%PJT}9d#(V|0qg0?|! z4bpab-zMS!lenDt1*+IsCvM=*b2<|mq^NhAds$HIHTF=UZN+YD_Sb;L0VGVc?;F6p zrkcI^;inub0h7wGY!ZfLt$krZ&h)P6xGlO#ns^*ci=O$qpKXmi=3b(j%c+oCuh&|I zMrpfTb%};Uy{Z2BLOf@$`IVzlD~13Fp<{Mm$v`#JGGq3cV*U5$M&Aa>7(LR-0@xq? zJ^UUo`I7(QQP9&Mq4ZRMzqn+Pe;X(L%+a0)Nf_uCAJuRGMu@GgW%75tMjPkql#muf z$?^?liLn-2++z2+3pHbL;YSl~#?stEm(*yj;9BGx%kWJ=EoM1%lACb}olFROrH+v( z284yW#*>~Erz4g1nT_pYDpz4F!T9+UMmQf&l1t*!j>Xs`&gUN2o(Kvuiu@TXP0uwq za&wNGk07SF@gXOl*kY@llGB~~1V|S@n=bKu#79u9yXDm@E0Nq0hq2UygJM4S#Ldq- pJ^pMS40F$|Kiw}kZ`T{+2jzNWr?xvjGaYuq4*bAh;wt`ie*=!>B-j7| literal 0 HcmV?d00001 diff --git a/tests/data/seafowl-legacy-data/9ae6f4222893474551037d0e44ff223ca5ea8e703d141b14835025923a66ab50.parquet b/tests/data/seafowl-legacy-data/9ae6f4222893474551037d0e44ff223ca5ea8e703d141b14835025923a66ab50.parquet new file mode 100644 index 0000000000000000000000000000000000000000..3accffc4a719d5762e49bb3030728200c29c2a94 GIT binary patch literal 541 zcmbtS%Syvg5S?60DAGlh-jEAfbfH0{_NmZ{n~938kJ=WrwhK$BAqv)N(uZH*$A}9N z{2Bj6a3*b`dk@^1Ip^M)GmG26Rzb$3DYwt1E@2&D>pS-gRTl`m4nUdCfg?etq@AHB zNjkc96Jyn+$ZzI7O#nee*Tl*O$hg`AJ*mvqa5(UK6ZdiK$%LUBZ<_J0*>yW^arRwlho%U;?OB;fz gp55SO?2UGUFg@{t@L~9rE)}!+Y#wekfChbxKOn_mT@)bfH0{_KDDn3nwbJK5ARg+Ab`ihA3F8NgsZJA0sYA z@MruN!I`v$?mcj4=A3(H&Ma;NTLlplrr16ghJbZ|t?%3~R9zsH9e{m02aW)d5_X23 zBaQJNy5$$@Z&IK@LBPg$aj=tMeE;)pe7BrP(r zW0tNsP}SzkBOe>{k+pe$hwrN#7|n}_Q+0yqqvLaquyCV25c|c-sC!*0cG|CnE^P>= hdUk`Cu{YWY!t}%o!iV8gx>U^Ovw67H02=f${s4Uka0&nb literal 0 HcmV?d00001 diff --git a/tests/data/seafowl-legacy-data/seafowl.sqlite b/tests/data/seafowl-legacy-data/seafowl.sqlite new file mode 100644 index 0000000000000000000000000000000000000000..3d1e885c117696bd5e798b7c2570aa90f2e7f3e6 GIT binary patch literal 4096 zcmWFz^vNtqRY=P(%1ta$FlG>7U}9o$P*7lCU|@t|AVoG{WYCK?;st3JAlr;ljiVtj n8UmvsFd71*Aut*OqaiRF0;3@?8UmvsFd71*Aut*O6ovo*V(tc> literal 0 HcmV?d00001 diff --git a/tests/data/seafowl-legacy-data/seafowl.sqlite-shm b/tests/data/seafowl-legacy-data/seafowl.sqlite-shm new file mode 100644 index 0000000000000000000000000000000000000000..1ca495285ba185d2a07e0dbc0eaa5c018f32f625 GIT binary patch literal 32768 zcmeI*H%_fe}1bL z8@%6+`d-1gpLOeJ{ClxytJxB>)PHWV@2!?LOWcyMv|Eyv4ok|Cwqz_>OU{zF1Ra9& zYG(%5M$W1~zjkizIj+6`zQ=wp3jqWWKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{ z1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|0009ILKmY**5I_I{1Q0*~0R#|;fI!pt zJ0T)al!^$11e)#5Dn05|UucH`0=^5x?43eI-%HXD0jEG*{mwZ<1pF6BXu$uHG(^BD z(5^w}oFM}K3nVqorSJ}4(nq`+Qw4{#)%F*q?Lc0{&?Zr*oOy7u)zLiMl+Hed7K2W z6Z4&CTjSAR^ZWll^OtP>^_%&Jud<98y7$a67#a=Ivirq%3w!tVIL_Mh=bto~?_NWj zjFsEJ_0;{{Z#6A_d+^-l$-L-FWU?cC-sPu9n@om9E*F(G(KjPKrnE?pXVqWmR^DI5 z@3mGz{}fB*y_009U<00Izzz$Z#ztixz+>+UvQ zZO-$tlsJ@IpV~Myk{r(R`D7-Y8~?j$@yh<*RsF72y?x93UE}X{wL4t2TrOsF^k7%t z(j`j=R=Lhux%BMbm1|tf`qy;H_X}b!o=uKQ0bHy4FIeRoSV7;bmoHcDPNs8tK9v%M zA?g05XZH`T>OK1$t&r>V{xf=4FJI+ayn5xz{()6PS`p>pxk5ZH=5nsS6)Tqa_YP=} zQ|^tg72}uX3L~z*1zaO1(>ZpRbJ<+!5eAE$=5zZ;&=BLkZFv76?E90uX=z1Rwwb z2tWV=5P$##W~P8NM!=|f1o0~6$Bsv0SG_<0uX=z1Rwwb z2tc4LP<0-`znt~Xny)>$@w!RQBWNl=+#n4h9QJtxvrI3^=MgkK@4T@7pBnzPE?=K@ zY_``qdTsrd7ue@5Uo*YL-dPb1s}T6;1h(}yTHB60&UnpgIY%i`NXMlt9rborRZfyy z-Rv%9BP9hFWg+!1>0ha(`_xED;%V1>P8_1rrL-kA@5i;W|BQZ0G+Nw0=n_iB+LJ9Z&!i~pi_)4lO;BL*ih;pZD|@A+ zE>(F*YVJ@Wom^iKU8@I{p1ZogUE0*8SrKW6qvPaRmbS&mO_V#O?aiyMHEmnyw6=A2 z8m|e;2VKcd8|Ab4q;%}Z{mNB!fpK@1`hl{wl$$omajq(>cd)y=U8iT#3-W3ga0y~c zlq#5WWfCr}x~@}BbIJR3WyOS;71NZV=HltgmE#j49rCQ;;)i)D^G(i>qunYyN)MNJ zl5^$P(&=J+R&?d)Fr~`-5?rxOu3TnB%&$$RhvhwtuHBSN#`)BcR)NWB`HKN2bslrHl z!aJN*onpdbhw?H}V<-AA7zaK;uQ%zubm%;zN0RBG%lQ;N|D>Eo<%o^&8xPo)E~HXJ z@l1hUeM&Eg?E!jb78loB+q_=mm~sRTtnqYcz!6C z&t&PfQR>IzUtS#@V}6IV?Zgv}S9i$0Md?gR2dP}ItEzeBJ%>_f<#8YJOUdCK{EC-k zrL_~kAdjy#9s6*drES@X`cKi)ZcEQk<+u9c2D`Ow!2;v9Zh5a&yRQ1hQ3G7f4_2~; z59;r;vQPcznp-a9)_a1*v!a~WtR(oBp9NZnQLDzzGX2XJx z4(aK##cFM9Ycp7rl_Aw`N#SVEf>^lt4@Ea=rjtfRJP_H z@p35THNC@TX*;z|zr!d!#Xe>=TifQ(H*Qm2zsq|nuUV?d%G(b`P5qZ_5;L{fhZieX zbK<9(8cjj-TP$r$=1+uHu|IEl-FgKh|Ib@oCLbYnhw3d`C8usZ6t|k)qx2sf{kz8M z{kx{d8}h1aW}HV*K7nD{&LjBIe=ND~x4X>mN#_w5xmykNg#`i-fB*y_009U<00Izz z00bZafoT%B#Ar1y@Y_?yCc${lYC#^A6qOH%az2mG-|eBVu*(za2?Tlq(eE$2W#9WP zm)-De*RtNacW=6a-R(Yp`;P2I!lO4V`pxwptnbR5_Qr*mkG*}~Pu7Lzefy{Pen8EV zuOXZTwkrb- z)GYZN0CikokMsDtb*r|>`2x7Uz>GTRXbc1(009U<00Izz00bZa0SG{#mVjB>sE!Lb zubf@?tLvlpN%;a87myIJKmY;|fB*y_009U<00Izz00d^VfJJJZIxev9+M93Nv+uNZ zn5S??A9FMg0uX=z1Rwwb2tWV=5P$##AfO6Zr4@BtfVursMef#@<{)1{<-sikAOHaf zKmY;|fB*y_009UE@1ibmJ>n`3y2tWV=5P$##AOHafKmY;| zn9%}GsdehOz_G#fEzjNa+AziiX7n*f;~)S52tWV=5P$##AOHafKmY=&04J@e;{vm8 zwf!k`;?oacTtMZ)Ed(F{0SG_<0uX=z1Rwwb2tZ&)3p7ftQ^y5*R<7AC{;Ai7>kG{2 zV~)l_00Izz00bZa0SG_<0uX=z1XO`p(uz7Ra7FvE4_+SX`5p2FR36+y00Izz00bZa z0SG_<0uX=z1ZK2A6Sb~M9TzzFb5Gy-Qhsg``2sWgn4@tJfB*y_009U<00Izz00bZa z0ad_6E84igHFtGh$G49DP|6oDa$hsh7ZwOW00Izz00bZa0SG_<0uX=z1g1scQlr(p zz;916O@i^9)j~Wg@_Esf=VK``=kxgd-5&Z1yF8JeK%gfO{r<9B_PyV7*$vNjE$gj& z_oge@-R|SJ@5o*xJbJ^T-(3H}`mWq*Z(Ml!*xTp*WL;?9w|{!?rym}qW=Z1$+Ia+* zEO_X_o?Q=oMj96|aX}gvpdVNu009U<00Izz00bZa0SG_<0uU$*44bUxD7{@bHVO5= z7uh^G|iY z^Y~3qJ*MOf_@#US|IyxX)E@#6fB*y_009U<00Izz00bcL5eg(r*#eZ@|FLEZG;Urp zCHVrZWTRep@blrOAHMU|-~NwseStR(+?yYv9Cif(2tWV=5P$##AOHafKmY;|fWWj1 zIE;*8t{^7(LMl&{G1Iqp9>JF$fAtHuJvJ~`I*-7}y=9;;ED(SI1Rwwb2tWV=5P$## zAOHaf%p?H^GuK`o7qCc8Q}YF`erwKCbD#O?CCC?;Ne3KlfdB*`009U<00Izz00bZa z0SJ@@97fh)KW4NLOC{s9)k@!5zQ8~DFMfIM%`Ll-FHrWv8U!E!0SG_<0uX=z1Rwwb z2tZ&)2{_ne>~?Kjz$Ud#%@=riXl>7{3 z00Izz00bZa0SG_<0uY#40#0_m-Ci{=APoU%;{rGBf9bz2z3zAarsNCkH*ouBR`bys z2tWV=5P$##AOHafKmY;|fB*z+Qq}+~Sl%v@MV;iKo=31{!)YD!Z#?56X>b5;E;AwyCDZw#%lTA6+~SU} zO{Rpbn0EJEba9tE&5wxg9(Q#iH*GJZQc{w*yCZl0iAl38k|UF_Op+u=DJ8ZRCNXNT~tZxWF6C|F%8xtvTl@=Mn5OaJ#g6UPFq#d<;&Dtk5le`o$O~fF8x2Qd0bY#w;(}j@hGUVq zHzq`RG0exJe3%bML@^lf#zG;VCmK%(;jqsqoHWX3*B8V*HOEQc+PJ_!p11e;mmS{A zq;Ubpxxv5{oFC8)76?E90uX=z1Rwwb2tWV=5P-l(DNr;sb~#zUwbeHD`TBGf{`Nll z;pxkps2Y^HZ!jp;n40AMqRmKQ=`@k1CMB$v2rz|bibS5WMYbPH2es|`9vAl9h~?IiywDa`$S)N@YrQ--w6Nug8QT-0tdH9y6)g= zy7J&6y7u5@boId<+#YW16Z$6r&4vI3AOHafKmY;|fB*y_009UbAke~g7{^@{(BzoS zc9u4l$K_1+R<^sewPIw$WNT#Smo_Wc1+;SPywV0~49aBrxUW0NNsUtHDg5i3ZR~kB z{rDs)U%<${W1ufA5P$##AOHafKmY;|fB*y_009WhI04qk80{u%RwI3D*BxBwK6&H% zmmc|eNx;k+?ee&Qlp3Io3tZIN75(Ka z;udM6iMy2MDWo4*AOHafKmY;|fB*y_009U<00Iy=EP(-&GOr*_8yHFid_G?!>JNki z!JyaU4+|bK5J)6^K7X7KihM*2d;EfzrWEuBBL1Mq7mWJ+d?>`nf*y5V!H;{M!n^`h~AOHafKmY;|fB*y_009U< z00Kv)fQ>25PPpx0<|+KczPn!?TXDy|(zpO;yT9fiUy9R52K0WnCY81&Pg?(iQ|+SlrKH4H=kR%l+JW+X?{d>_qer*;~FRqO&q7h zskoT6UZ9F%I*~ra#BudhDVjJ=snpa?99Ku7Y2r8~^wdlohj|JM;sAqd9(B)3%29v{SQE>OlBE}?qL@yg!+&M_M9)(=``78ZV3b`bU3ZXkz97$2=zLw8snGd? zJXfJjo~_U$%~#0HoBn?TPy+}+00Izz00bZa0SG_<0uX?}gaS1t5oEYplL#`-nv)1J z4dW&eWa_IY5oGFWggC0_DLmt=XD;~7&57HT>kht2^AvJ#PUwp}AOHafKmY;|fB*y_ z009U<00Iy=>I7I5%}~hHo~Q7hYrk;*nLFNIDdh{8xHqJ6fj5r22BN|cfB*y_009U< z00Izz00bZa0SHVaV4(>Ljf_c~r;w8h?tbyz!rpy7j@jQm=E=VsX#1>k-NAiwz5usx zqFrG-1Rwwb2tWV=5P$##AOHafKmYQ2}OrzJQb((4^)ItQbf=(tq_ki+y#L+Fu}%@cI04J}B}LG3@aRUfPK_5b*~+zF^en=R+Yr z7W8QQGb;NFCt?Xv6nQ}md!yk9)lrQ2;(}j@hGUVqHzrWPFdvIj3&Igm4AM?RA)hB2 zPYB_#&nKw+BNr!qYvTgXnA3L@=h}WGjSH9?L<3iFe&F2D@G7lhfdB*`009U<00Izz z00bZafx{IjTFiENQqR^_+ceDUNpI;Qee`2dU(v#96*0+k15W7#qg020RFPBqie|=c zu+ucBBr+}aXqW2Y>C2m_8Z_mp!Jt%QiY7g^87VByvD?(7gw-aQ)SI7CD|Aq^Gfp}a zW#lPaWc#tyrKM8_v~dCR@yj+Z-|pHVjSH~c(*|xox0mK5eEM*^J{|`F2tWV=5P$## zAOHafKmY;|fWXuX*jUzJX9dgK+igr~ZJW(hS}WR&>KfIoPO6=H9>Le|`rg_ze{pM{ zG%nC$Im^I(nQL}l>73JWW5Wscchoo4-C5V}_>LoL-)TR^cE7FP`j|Cfd4dXJfdB*` z009V82^8t?h`~sM;H>-sfHt~4g+t(CGu5E8*s0VYmdT`sCjC65TuNscKFyDa?jE;P z*1dIWvAK!jbdQ;pIGOxfF*|vniefsEV#)mZNg==1-Q)E|y48 zvb>)_vW*m)K6Oz-Ptj*C29APKcT+xqo0jh8q#96nQ)*D|NP0cKTxxQAt-Cc)9O`aL zoQi&=7pS6`PNWaf-Rh}Q)ZM1x^UrJ@g{JPNgr2JI=AfX|-IR}6r=`2usRq>Dlp2&f zl3p(;m(uA3<6qru6o@7NrW5HybT=ziin`l0e9E1*P-yCIO6aNSZdaKt z@(0S(_NP=}sRGp5Ow8vP4bPv2U;K z0xfKZ(I~$I)&|+wmf37)X=8bmfo*POyGvUu#st`=Ms|K_v;6LzoyD>9N*km%)of#p z5DZr<1mmn3f@v5Rf~l_#!PM0VVRF<8VY1f@VX}=2VX{_-Fj<<}4*NJLa|_#9+E(E- zo9!-btYBg|c7AEAtYm6n=h^Mr$O9dws(A{zf#Lqq_wL#uUtfUxeQBOT?j>5q0s#m> z00Izz00bZa0SG_<0uX?}Q6^wE)6|5liDen1ojr(o3cvn0=Zjy>Zn;*<7cg_rm*y$t z{z$7>AOHafKmY;|fB*y_009U<00Iy=iUj0|30ad_o|uqjCz_|QDqmpXIltte-?H2) z!rt0Vy>=yYAr4XWagN z`~u-QX(Q|Wfq~n_UCK4m3Kj@J00Izz00bZa0SG_<0uX?}5hyUgnhnO{aZJ!35QFi! zKN=E)F<&6;3r3@UZy*@)d1H}KARbG^JmI)chzMb?HYSyeEzF z+4Ti6Kfoya6M1jcm*B&^=#K^XSe%avq7e3ofruyWjYLFW+!ykQana-BBVLafOn60K zObmEq9w8L>BtqK$Ov?VEyckLZd_G?!>JNki!JyaU4-2%vKqBGu`Qv<0d-8VdP5(Re}#hkZUl-5(uNCw*(<0?$N7U)=l+$5-U@2&J7K( zHi+~K76?E90uX=z1Rwwb2tWV=N0~s;VzJAU8@9IEreXG0(O_`((U11NqQ$ILWE$r6 zq&h^TiY%gvuv$f?Va7+QLqMv?DSbsVW0xlbq(d|H^8r#7+NFAU`tl~K2F(m;FeufS zn#lo+HY0_lNl%-al(5>240`i3YK0DJcE(Vk%bm;(CWq6a;O@z13!*mbG3|&Z7mhq- zi)=rZp3>4O1KPO2g@sj(*MDnlpENFD=6+`2_H%n_F2bj26$=C)009U<00Izz00bZa z0SG_<0!Orf&CD9?tYCTj^ES4$w%x{**0$M9r8OFtP}itAbyAadzxZxp@4g<#SI$`6 z`7`5!PfOzh$1u+sxG!_f&MTdB8g6Vjq5h8grn)=p+8y6qvxe8)DIl`yt zaidvU%_TEwcaNX$&&Bx^)y(VJx^+`4Rce0mVx>~@EY3g zH0)ay-#xf=V9D}+0?E#%&@>FLgr1^7a6=0PrS7JDG^34f>$O=cq|@&50hn|MTx_Nq zP2RVbIzsl)j0)L z{YWoRMKPU7AELX}Q>Cc8O~dD(**XeM-AxHSRo%@&L8-ebAG2ybN$(J7g%0WfOy1q> zR0Ha6N)5^#Nv{g!QaW`Q-`#8!hq{{*r=lO}1*#~f6X`>AH!D?&y4y5-%AK`PXzFfC z=&9*$SD7vH2g=j-pPRr^1*o&h6)5*2y)IBLrPBk(^|dC3!cbq6!&LMky*L%cbOIfI zSKDZ!I#E}X>oleRwyif(Wa?>h06@qQ98G>yZ7lNHt9fEDF5rW}rg~?MBfqF#MJcZldeCyQfp0)h9lrLcC9;uqAkoy&l2t0DoeFz01009U<00Izz z00bZa0SG_<0uY!A0kfUvBs4Q-n{;Dht&GubKFoOvqnpos#{T^suSxj=cJ6ys^AvJV zaF5U?ED(SI1Rwwb2tWV=5P$##AOHaf90>yQM1*t&N9ERJv#>_H(QG}$c?zrY1q{(! zj{V~;|JNtw3plucuaPf68?itD0uX=z1Rwwb2tWV=5P$##AaDc-m>p7v0Q(W<3+y`n z)Jxv!jomEe3plvlHSz^$BNhli00Izz00bZa0SG_<0uX=z1g1|w$q!&1A3?sr{{ty$ B literal 0 HcmV?d00001 diff --git a/tests/statements/mod.rs b/tests/statements/mod.rs index 07ebfd9b..dfb1485f 100644 --- a/tests/statements/mod.rs +++ b/tests/statements/mod.rs @@ -40,6 +40,7 @@ mod function; #[path = "../../src/object_store/testutils.rs"] mod http_testutils; mod query; +mod query_legacy; mod vacuum; // Object store IDs for frequently-used test data @@ -56,7 +57,6 @@ enum ObjectStoreType { } /// Make a SeafowlContext that's connected to a real PostgreSQL database -/// (but uses an in-memory object store) async fn make_context_with_pg( object_store_type: ObjectStoreType, ) -> DefaultSeafowlContext { diff --git a/tests/statements/query_legacy.rs b/tests/statements/query_legacy.rs new file mode 100644 index 00000000..ad6be58f --- /dev/null +++ b/tests/statements/query_legacy.rs @@ -0,0 +1,355 @@ +use crate::statements::*; +use tokio::fs::{copy, read_dir}; + +/// Make a SeafowlContext that's connected to a legacy SQLite catalog copy +async fn make_context_with_local_sqlite(data_dir: String) -> DefaultSeafowlContext { + assert_ne!(data_dir.as_str(), "tests/data/seafowl-legacy-data/"); + + // Copy the legacy catalog into the provided data directory + let mut legacy_data = read_dir("tests/data/seafowl-legacy-data/").await.unwrap(); + while let Some(dir_entry) = legacy_data.next_entry().await.unwrap() { + let file = dir_entry.file_name(); + copy( + dir_entry.path(), + format!("{}/{}", data_dir.clone(), file.to_str().unwrap()), + ) + .await + .unwrap(); + } + + let config_text = format!( + r#" +[object_store] +type = "local" +data_dir = "{data_dir}" + +[catalog] +type = "sqlite" +dsn = "{data_dir}/seafowl.sqlite""# + ); + + let config = load_config_from_string(&config_text, true, None).unwrap(); + build_context(&config).await.unwrap() +} + +#[tokio::test] +async fn test_legacy_tables() { + let data_dir = TempDir::new().unwrap(); + + let context = + make_context_with_local_sqlite(data_dir.path().display().to_string()).await; + + let (_, version_timestamps) = create_table_and_some_partitions( + &context, + "test_new_table", + Some(Duration::from_secs(1)), + ) + .await; + + let timestamp_to_rfc3339 = |timestamp: Timestamp| -> String { + Utc.timestamp_opt(timestamp, 0).unwrap().to_rfc3339() + }; + + // + // Verify that the legacy table versions are shown in the corresponding system table + // + + let plan = context + .plan_query("SELECT * FROM system.table_versions") + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let expected = vec![ + "+--------------+------------+------------------+---------------------+", + "| table_schema | table_name | table_version_id | creation_time |", + "+--------------+------------+------------------+---------------------+", + "| public | test_table | 1 | 2023-03-07T08:44:49 |", + "| public | test_table | 2 | 2023-03-07T08:44:49 |", + "| public | test_table | 3 | 2023-03-07T08:44:51 |", + "| public | test_table | 4 | 2023-03-07T08:44:53 |", + "| public | test_table | 5 | 2023-03-07T08:44:55 |", + "+--------------+------------+------------------+---------------------+", + ]; + assert_batches_eq!(expected, &results); + + // + // Test that filtering the system table works, given that we provide all rows to DF and expect + // it to do it. + // + let plan = context + .plan_query(" + SELECT table_version_id FROM system.table_versions \ + WHERE table_version_id < 5 AND creation_time > to_timestamp('2023-03-07T08:44:49+00:00') + ") + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let expected = vec![ + "+------------------+", + "| table_version_id |", + "+------------------+", + "| 3 |", + "| 4 |", + "+------------------+", + ]; + assert_batches_eq!(expected, &results); + + // + // Verify that the legacy table partitions for all versions are shown in the corresponding system table + // + + let plan = context + .plan_query("SELECT table_schema, table_name, table_version_id, table_partition_id, row_count FROM system.table_partitions") + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let expected = vec![ + "+--------------+------------+------------------+--------------------+-----------+", + "| table_schema | table_name | table_version_id | table_partition_id | row_count |", + "+--------------+------------+------------------+--------------------+-----------+", + "| public | test_table | 1 | | |", + "| public | test_table | 2 | 1 | 3 |", + "| public | test_table | 3 | 1 | 3 |", + "| public | test_table | 3 | 2 | 3 |", + "| public | test_table | 4 | 1 | 3 |", + "| public | test_table | 4 | 2 | 3 |", + "| public | test_table | 4 | 3 | 3 |", + "| public | test_table | 5 | 1 | 3 |", + "| public | test_table | 5 | 2 | 3 |", + "| public | test_table | 5 | 3 | 3 |", + "| public | test_table | 5 | 4 | 3 |", + "+--------------+------------+------------------+--------------------+-----------+", + ]; + assert_batches_eq!(expected, &results); + + // + // Test some projections and aggregations + // + + let plan = context + .plan_query("SELECT MAX(some_time) FROM test_table") + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let expected = vec![ + "+---------------------------+", + "| MAX(test_table.some_time) |", + "+---------------------------+", + "| 2022-01-01T20:03:03 |", + "+---------------------------+", + ]; + + assert_batches_eq!(expected, &results); + + let plan = context + .plan_query("SELECT MAX(some_int_value), COUNT(DISTINCT some_bool_value), MAX(some_value) FROM test_table") + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let expected = vec![ + "+--------------------------------+--------------------------------------------+----------------------------+", + "| MAX(test_table.some_int_value) | COUNT(DISTINCT test_table.some_bool_value) | MAX(test_table.some_value) |", + "+--------------------------------+--------------------------------------------+----------------------------+", + "| 3333 | 0 | 48 |", + "+--------------------------------+--------------------------------------------+----------------------------+", + ]; + + assert_batches_eq!(expected, &results); + + // + // Now use the recorded timestamps to query specific legacy and new table versions and compare + // them to each other. + // + + async fn compare_legacy_and_new_table_version( + context: &DefaultSeafowlContext, + version_id: TableVersionId, + version_timestamps: &HashMap, + timestamp_converter: fn(Timestamp) -> String, + ) { + let legacy_version_timestamps = [ + "2023-03-07T08:44:49+00:00", + "2023-03-07T08:44:49+00:00", + "2023-03-07T08:44:51+00:00", + "2023-03-07T08:44:53+00:00", + "2023-03-07T08:44:55+00:00", + ]; + let plan = context + .plan_query( + format!( + "SELECT some_time, some_value, some_other_value, some_bool_value, some_int_value FROM test_table('{}') ORDER BY some_value", + legacy_version_timestamps[version_id as usize] + ) + .as_str(), + ) + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let formatted = arrow::util::pretty::pretty_format_batches(results.as_slice()) + .unwrap() + .to_string(); + + let plan = context + .plan_query( + format!( + "SELECT * FROM test_new_table('{}') ORDER BY some_value", + timestamp_converter(version_timestamps[&version_id]) + ) + .as_str(), + ) + .await + .unwrap(); + let new_results = context.collect(plan).await.unwrap(); + + let new_formatted = + arrow::util::pretty::pretty_format_batches(new_results.as_slice()) + .unwrap() + .to_string(); + + assert_eq!(formatted, new_formatted); + } + + for version_id in [1, 2, 3, 4] { + compare_legacy_and_new_table_version( + &context, + version_id as TableVersionId, + &version_timestamps, + timestamp_to_rfc3339, + ) + .await; + } + + // + // Try to query a non-existent legacy version (timestamp older than the oldest version) + // + + let err = context + .plan_query("SELECT * FROM test_table('2012-12-21 20:12:21 +00:00')") + .await + .unwrap_err(); + + assert!(err + .to_string() + .contains("No recorded table versions for the provided timestamp")); + + // + // Use multiple different version specifiers in the same complex query (including the latest + // version both explicitly and in the default notation). + // Ensures row differences between different versions are consistent: + // 5 - ((5 - 4) + (4 - 3) + (3 - 2)) = 2 + // + + let plan = context + .plan_query( + r#" + WITH diff_3_2 AS ( + SELECT * FROM test_table('2023-03-07T08:44:51+00:00') + EXCEPT + SELECT * FROM test_table('2023-03-07T08:44:49+00:00') + ), diff_4_3 AS ( + SELECT * FROM test_table('2023-03-07T08:44:53+00:00') + EXCEPT + SELECT * FROM test_table('2023-03-07T08:44:51+00:00') + ), diff_5_4 AS ( + SELECT * FROM test_table('2023-03-07T08:44:55+00:00') + EXCEPT + SELECT * FROM test_table('2023-03-07T08:44:53+00:00') + ) + SELECT * FROM test_table + EXCEPT ( + SELECT * FROM diff_5_4 + UNION + SELECT * FROM diff_4_3 + UNION + SELECT * FROM diff_3_2 + ) + ORDER BY some_int_value + "#, + ) + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let expected = vec![ + "+-----------------+----------------+------------------+---------------------+------------+", + "| some_bool_value | some_int_value | some_other_value | some_time | some_value |", + "+-----------------+----------------+------------------+---------------------+------------+", + "| | 1111 | | 2022-01-01T20:01:01 | 42 |", + "| | 2222 | | 2022-01-01T20:02:02 | 43 |", + "| | 3333 | | 2022-01-01T20:03:03 | 44 |", + "+-----------------+----------------+------------------+---------------------+------------+", + ]; + + assert_batches_eq!(expected, &results); + + // Ensure the context table map contains the versioned + the latest table entries + assert_eq!( + sorted( + context + .inner() + .state() + .catalog_list() + .catalog(DEFAULT_DB) + .unwrap() + .schema(DEFAULT_SCHEMA) + .unwrap() + .table_names() + .into_iter() + .filter(|name| !name.contains("new")) + ) + .collect::>(), + vec![ + "test_table".to_string(), + "test_table:2".to_string(), + "test_table:3".to_string(), + "test_table:4".to_string(), + "test_table:5".to_string(), + ], + ); + + // + // Verify that information schema is not polluted with versioned tables/columns + // + + let results = list_tables_query(&context).await; + + let expected = vec![ + "+--------------------+----------------+", + "| table_schema | table_name |", + "+--------------------+----------------+", + "| information_schema | columns |", + "| information_schema | df_settings |", + "| information_schema | tables |", + "| information_schema | views |", + "| public | test_new_table |", + "| public | test_table |", + "+--------------------+----------------+", + ]; + assert_batches_eq!(expected, &results); + + let results = list_columns_query(&context).await; + + let expected = vec![ + "+--------------+----------------+------------------+------------------------------+", + "| table_schema | table_name | column_name | data_type |", + "+--------------+----------------+------------------+------------------------------+", + "| public | test_new_table | some_time | Timestamp(Microsecond, None) |", + "| public | test_new_table | some_value | Float32 |", + "| public | test_new_table | some_other_value | Decimal128(38, 10) |", + "| public | test_new_table | some_bool_value | Boolean |", + "| public | test_new_table | some_int_value | Int64 |", + "| public | test_table | some_bool_value | Boolean |", + "| public | test_table | some_int_value | Int64 |", + "| public | test_table | some_other_value | Decimal128(38, 10) |", + "| public | test_table | some_time | Timestamp(Nanosecond, None) |", + "| public | test_table | some_value | Float32 |", + "+--------------+----------------+------------------+------------------------------+", + ]; + assert_batches_eq!(expected, &results); +} From c3faf29a076885950f8445f36df36140359d382d Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Tue, 7 Mar 2023 15:04:32 +0100 Subject: [PATCH 12/21] Extend legacy table test to check migration works ok --- tests/statements/query_legacy.rs | 59 ++++++++++++++++++++++++++++++++ 1 file changed, 59 insertions(+) diff --git a/tests/statements/query_legacy.rs b/tests/statements/query_legacy.rs index ad6be58f..53e3b447 100644 --- a/tests/statements/query_legacy.rs +++ b/tests/statements/query_legacy.rs @@ -39,6 +39,58 @@ async fn test_legacy_tables() { let context = make_context_with_local_sqlite(data_dir.path().display().to_string()).await; + // + // For start test that migration actually works + // TODO: make us cast automatic + // + let plan = context + .plan_query( + r" + CREATE TABLE test_migration AS + (SELECT + to_timestamp_micros(some_time) AS some_time, + some_value, + some_other_value, + some_bool_value, + some_int_value + FROM test_table ORDER BY some_value, some_int_value + )", + ) + .await + .unwrap(); + context.collect(plan).await.unwrap(); + + let plan = context + .plan_query("SELECT * FROM test_migration") + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let expected = vec![ + "+---------------------+------------+------------------+-----------------+----------------+", + "| some_time | some_value | some_other_value | some_bool_value | some_int_value |", + "+---------------------+------------+------------------+-----------------+----------------+", + "| | 40 | | | |", + "| | 41 | | | |", + "| 2022-01-01T20:01:01 | 42 | | | 1111 |", + "| | 42 | | | |", + "| 2022-01-01T20:02:02 | 43 | | | 2222 |", + "| 2022-01-01T20:03:03 | 44 | | | 3333 |", + "| | 45 | | | |", + "| | 46 | | | |", + "| | 46 | | | |", + "| | 47 | | | |", + "| | 47 | | | |", + "| | 48 | | | |", + "+---------------------+------------+------------------+-----------------+----------------+", + ]; + + assert_batches_eq!(expected, &results); + + // + // Create the accompanying new tables with same content (though different column/row order) to + // be used in comparisons + // let (_, version_timestamps) = create_table_and_some_partitions( &context, "test_new_table", @@ -305,6 +357,7 @@ async fn test_legacy_tables() { ) .collect::>(), vec![ + "test_migration".to_string(), "test_table".to_string(), "test_table:2".to_string(), "test_table:3".to_string(), @@ -327,6 +380,7 @@ async fn test_legacy_tables() { "| information_schema | df_settings |", "| information_schema | tables |", "| information_schema | views |", + "| public | test_migration |", "| public | test_new_table |", "| public | test_table |", "+--------------------+----------------+", @@ -339,6 +393,11 @@ async fn test_legacy_tables() { "+--------------+----------------+------------------+------------------------------+", "| table_schema | table_name | column_name | data_type |", "+--------------+----------------+------------------+------------------------------+", + "| public | test_migration | some_time | Timestamp(Microsecond, None) |", + "| public | test_migration | some_value | Float32 |", + "| public | test_migration | some_other_value | Decimal128(38, 10) |", + "| public | test_migration | some_bool_value | Boolean |", + "| public | test_migration | some_int_value | Int64 |", "| public | test_new_table | some_time | Timestamp(Microsecond, None) |", "| public | test_new_table | some_value | Float32 |", "| public | test_new_table | some_other_value | Decimal128(38, 10) |", From 513ee02925a3f64f1c1d31223da226a4ffcc033e Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Thu, 9 Mar 2023 14:58:58 +0100 Subject: [PATCH 13/21] Re-implement DELETE in accordance with delta-rs This uses some private delta-rs code, as well as some WIP code for native delta-rs DELETE support. --- src/context.rs | 444 ++++++++++++++++++++++++++++++---------- tests/statements/dml.rs | 147 +++---------- 2 files changed, 370 insertions(+), 221 deletions(-) diff --git a/src/context.rs b/src/context.rs index b3f6dc02..1097eb79 100644 --- a/src/context.rs +++ b/src/context.rs @@ -16,7 +16,7 @@ use tokio::io::{AsyncReadExt, AsyncWriteExt, BufReader}; use std::fs::File; use datafusion::datasource::listing::{ - ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, + ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, PartitionedFile, }; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::execution::context::{default_session_builder, SessionState}; @@ -34,19 +34,22 @@ use futures::{StreamExt, TryStreamExt}; #[cfg(test)] use mockall::automock; -use object_store::{path::Path, ObjectStore}; +use object_store::{path::Path, ObjectMeta, ObjectStore}; use sqlparser::ast::{ AlterTableOperation, CreateFunctionBody, FunctionDefinition, Ident, ObjectName, ObjectType, SchemaName, Statement, TableFactor, TableWithJoins, }; +use arrow::compute::{cast_with_options, CastOptions}; use arrow_integration_test::field_to_json; -use arrow_schema::DataType; -use chrono::{DateTime, FixedOffset, Utc}; +use arrow_schema::{ArrowError, DataType, TimeUnit}; +use chrono::{DateTime, FixedOffset, NaiveDateTime, Utc}; use std::iter::zip; +use std::ops::Deref; use std::str::FromStr; use std::sync::Arc; +use std::time::{SystemTime, UNIX_EPOCH}; use datafusion::common::{DFSchema, ToDFSchema}; use datafusion::datasource::file_format::avro::AvroFormat; @@ -60,6 +63,9 @@ use datafusion::optimizer::type_coercion::TypeCoercion; use datafusion::optimizer::{OptimizerContext, OptimizerRule}; use datafusion::physical_expr::create_physical_expr; use datafusion::physical_expr::execution_props::ExecutionProps; +use datafusion::physical_optimizer::pruning::PruningPredicate; +use datafusion::physical_plan::file_format::{partition_type_wrap, FileScanConfig}; +use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::prelude::SessionConfig; use datafusion::scalar::ScalarValue; @@ -85,14 +91,20 @@ use datafusion_expr::logical_plan::{ CreateCatalog, CreateCatalogSchema, CreateExternalTable, CreateMemoryTable, DropTable, Extension, LogicalPlan, Projection, }; -use datafusion_expr::{DmlStatement, Filter, WriteOp}; +use datafusion_expr::{DmlStatement, Expr, Filter, WriteOp}; +use deltalake::action::{Action, Add, Remove}; +use deltalake::delta_datafusion::DeltaDataChecker; +use deltalake::operations::writer::{DeltaWriter, WriterConfig}; use deltalake::operations::{create::CreateBuilder, write::WriteBuilder}; -use deltalake::{DeltaTable, Schema as DeltaSchema}; +use deltalake::storage::DeltaObjectStore; +use deltalake::{DeltaResult, DeltaTable, Schema as DeltaSchema}; use log::{debug, info, warn}; +use object_store::path::DELIMITER; use parking_lot::RwLock; use prost::Message; use tempfile::TempPath; use tokio::sync::Semaphore; +use url::Url; use uuid::Uuid; use crate::catalog::{PartitionCatalog, DEFAULT_SCHEMA, STAGING_SCHEMA}; @@ -520,6 +532,231 @@ pub fn is_statement_read_only(statement: &DFStatement) -> bool { } } +// Appropriated from https://github.com/delta-io/delta-rs/pull/1176; once the DELETE and UPDATE ops +// are available through delta-rs this will be obsolete. +/// Write the provide ExecutionPlan to the underlying storage +/// The table's invariants are checked during this proccess +pub async fn write_execution_plan( + table: &DeltaTable, + state: SessionState, + plan: Arc, + partition_columns: Vec, + object_store: Arc, + target_file_size: Option, + write_batch_size: Option, +) -> Result> { + let invariants = table + .get_metadata() + .and_then(|meta| meta.schema.get_invariants()) + .unwrap_or_default(); + let checker = DeltaDataChecker::new(invariants); + + // Write data to disk + let mut tasks = vec![]; + for i in 0..plan.output_partitioning().partition_count() { + let inner_plan = plan.clone(); + let task_ctx = Arc::new(TaskContext::from(&state)); + + let config = WriterConfig::new( + inner_plan.schema(), + partition_columns.clone(), + None, + target_file_size, + write_batch_size, + ); + let mut writer = DeltaWriter::new(object_store.clone(), config); + let checker_stream = checker.clone(); + let mut stream = inner_plan.execute(i, task_ctx)?; + let handle: tokio::task::JoinHandle>> = + tokio::task::spawn(async move { + while let Some(maybe_batch) = stream.next().await { + let batch = maybe_batch?; + checker_stream.check_batch(&batch).await?; + writer.write(&batch).await?; + } + writer.close().await + }); + + tasks.push(handle); + } + + // Collect add actions to add to commit + Ok(futures::future::join_all(tasks) + .await + .into_iter() + .collect::, _>>() + .map_err(|err| { + DataFusionError::Execution(format!( + "Failed writing to delta table {table}: {err}" + )) + })? + .into_iter() + .collect::, _>>()? + .concat() + .into_iter() + .collect::>()) +} + +// Appropriated from https://github.com/delta-io/delta-rs/pull/1176; once the DELETE and UPDATE ops +// are available through delta-rs this will be obsolete. +/// Create a Parquet scan limited to a set of files +pub async fn parquet_scan_from_actions( + table: &DeltaTable, + actions: &[Add], + schema: &Schema, + filters: &[Expr], + state: &SessionState, + projection: Option<&Vec>, + limit: Option, +) -> Result> { + // TODO we group files together by their partition values. If the table is partitioned + // and partitions are somewhat evenly distributed, probably not the worst choice ... + // However we may want to do some additional balancing in case we are far off from the above. + let mut file_groups: HashMap, Vec> = HashMap::new(); + for action in actions { + let part = partitioned_file_from_action(action, schema); + file_groups + .entry(part.partition_values.clone()) + .or_default() + .push(part); + } + + let table_partition_cols = table.get_metadata()?.partition_columns.clone(); + let file_schema = Arc::new(Schema::new( + schema + .fields() + .iter() + .filter(|f| !table_partition_cols.contains(f.name())) + .cloned() + .collect(), + )); + + let url = Url::parse(&table.table_uri()).unwrap(); + let host = format!( + "{}-{}{}", + url.scheme(), + url.host_str().unwrap_or_default(), + url.path().replace(DELIMITER, "-").replace(':', "-") + ); + state + .runtime_env() + .register_object_store("delta-rs", &host, table.object_store()); + let object_store_url = ObjectStoreUrl::parse(format!("delta-rs://{host}"))?; + + ParquetFormat::new() + .create_physical_plan( + state, + FileScanConfig { + object_store_url, + file_schema, + file_groups: file_groups.into_values().collect(), + statistics: table.datafusion_table_statistics(), + projection: projection.cloned(), + limit, + table_partition_cols: table_partition_cols + .iter() + .map(|c| { + Ok(( + c.to_owned(), + partition_type_wrap( + schema.field_with_name(c)?.data_type().clone(), + ), + )) + }) + .collect::, ArrowError>>()?, + output_ordering: None, + infinite_source: false, + }, + filters, + ) + .await +} + +// Copied from delta-rs as it's private there; once the DELETE and UPDATE ops +// are available through delta-rs this will be obsolete. +fn partitioned_file_from_action(action: &Add, schema: &Schema) -> PartitionedFile { + let partition_values = schema + .fields() + .iter() + .filter_map(|f| { + action.partition_values.get(f.name()).map(|val| match val { + Some(value) => to_correct_scalar_value( + &serde_json::Value::String(value.to_string()), + f.data_type(), + ) + .unwrap_or(ScalarValue::Null), + None => ScalarValue::Null, + }) + }) + .collect::>(); + + let ts_secs = action.modification_time / 1000; + let ts_ns = (action.modification_time % 1000) * 1_000_000; + let last_modified = DateTime::::from_utc( + NaiveDateTime::from_timestamp_opt(ts_secs, ts_ns as u32).unwrap(), + Utc, + ); + PartitionedFile { + object_meta: ObjectMeta { + location: Path::from(action.path.clone()), + last_modified, + size: action.size as usize, + }, + partition_values, + range: None, + extensions: None, + } +} + +// Copied from delta-rs as it's private there; once the DELETE and UPDATE ops +// are available through delta-rs this will be obsolete. +fn to_correct_scalar_value( + stat_val: &serde_json::Value, + field_dt: &DataType, +) -> Option { + match stat_val { + serde_json::Value::Array(_) => None, + serde_json::Value::Object(_) => None, + serde_json::Value::Null => None, + serde_json::Value::String(string_val) => match field_dt { + DataType::Timestamp(_, _) => { + let time_nanos = ScalarValue::try_from_string( + string_val.to_owned(), + &DataType::Timestamp(TimeUnit::Nanosecond, None), + ) + .ok()?; + let cast_arr = cast_with_options( + &time_nanos.to_array(), + field_dt, + &CastOptions { safe: false }, + ) + .ok()?; + Some(ScalarValue::try_from_array(&cast_arr, 0).ok()?) + } + _ => { + Some(ScalarValue::try_from_string(string_val.to_owned(), field_dt).ok()?) + } + }, + other => match field_dt { + DataType::Timestamp(_, _) => { + let time_nanos = ScalarValue::try_from_string( + other.to_string(), + &DataType::Timestamp(TimeUnit::Nanosecond, None), + ) + .ok()?; + let cast_arr = cast_with_options( + &time_nanos.to_array(), + field_dt, + &CastOptions { safe: false }, + ) + .ok()?; + Some(ScalarValue::try_from_array(&cast_arr, 0).ok()?) + } + _ => Some(ScalarValue::try_from_string(other.to_string(), field_dt).ok()?), + }, + } +} + // The only reason to keep this trait around (instead of migrating all the functions directly into // DefaultSeafowlContext), is that `create_physical_plan` would then be a recursive async function, // which works for traits, but not for structs: https://stackoverflow.com/a/74737853 @@ -822,8 +1059,8 @@ impl DefaultSeafowlContext { /// Generate the Delta table builder and execute the write pub async fn plan_to_delta_table<'a>( &self, - plan: &Arc, name: impl Into>, + plan: &Arc, ) -> Result { let table_uuid = self.get_table_uuid(name).await?; let table_object_store = self.internal_object_store.for_delta_table(table_uuid); @@ -1457,7 +1694,7 @@ impl SeafowlContext for DefaultSeafowlContext { .create_delta_table(name, physical.schema().as_ref()) .await?; self.reload_schema().await?; - self.plan_to_delta_table(&physical, name).await?; + self.plan_to_delta_table(name, &physical).await?; Ok(make_dummy_exec()) } @@ -1469,7 +1706,7 @@ impl SeafowlContext for DefaultSeafowlContext { }) => { let physical = self.create_physical_plan(input).await?; - self.plan_to_delta_table(&physical, table_name).await?; + self.plan_to_delta_table(table_name, &physical).await?; Ok(make_dummy_exec()) } @@ -1591,120 +1828,116 @@ impl SeafowlContext for DefaultSeafowlContext { } LogicalPlan::Dml(DmlStatement { table_name, + table_schema, op: WriteOp::Delete, input, - .. }) => { - let table = self.try_get_seafowl_table(table_name.to_string()).await?; - - // If no qualifier is specified we're basically truncating the table; - // Make a new (empty) table version and finish. - let new_table_version_id = self - .table_catalog - .create_new_table_version(table.table_version_id, false) - .await?; - - if let LogicalPlan::Filter(Filter { predicate, .. }) = &**input { + // TODO: Once https://github.com/delta-io/delta-rs/pull/1176 is merged use that instead + let table_object_store = self + .inner + .table_provider(table_name) + .await? + .as_any() + .downcast_ref::() + .ok_or_else(|| { + DataFusionError::Execution( + "Table {table_name} not found".to_string(), + ) + })? + .object_store(); + // Can't just keep hold of the downcasted ref from above because of + // `temporary value dropped while borrowed` + let mut table = DeltaTable::new(table_object_store, Default::default()); + table.load().await?; + let schema_ref = SchemaRef::from(table_schema.deref().clone()); + + let (adds, removes) = if let LogicalPlan::Filter(Filter { + predicate, + .. + }) = &**input + { // A WHERE clause has been used; employ it to prune the filtration // down to only a subset of partitions, re-use the rest as is - // Load all pre-existing partitions - let partitions = self - .partition_catalog - .load_table_partitions(table.table_version_id) - .await?; + let state = self.inner.state(); // To simulate the effect of a WHERE clause from a DELETE, we // need to use the inverse clause in a SELECT when filtering let filter = create_physical_expr( &predicate.clone().not(), - &table.schema.arrow_schema.clone().to_dfschema()?, - table.schema().as_ref(), + table_schema, + schema_ref.as_ref(), &ExecutionProps::new(), )?; - let mut final_partition_ids = Vec::with_capacity(partitions.len()); - - match SeafowlPruningStatistics::from_partitions( - partitions.clone(), - table.schema(), - ) { - Ok(pruning_stats) => { - // Determine the set of all partition ids that will need to - // be filtered - let partitions_to_filter = - HashSet::::from_iter( - pruning_stats - .prune(&[predicate.clone()]) - .await - .iter() - .map(|p| p.partition_id.unwrap()), - ); - - for (keep, group) in - group_partitions(partitions, |p: &SeafowlPartition| { - !partitions_to_filter - .contains(&p.partition_id.unwrap()) - }) - { - if keep { - // Inherit the partition(s) as is from the previous - // table version - final_partition_ids.extend( - group.iter().map(|p| p.partition_id.unwrap()), - ); - continue; - } - - // Get the plan which will eliminate the affected rows - let filter_plan = table - .partition_filter_plan( - group, - filter.clone(), - &[predicate.clone().not()], - self.internal_object_store.inner.clone(), - ) - .await?; - - debug!("Prepared delete filter plan: {:?}", &filter_plan); - - final_partition_ids.extend( - self.execute_plan_to_partitions(&filter_plan, None) - .await?, - ); - } - } - Err(error) => { - warn!( - "Failed constructing pruning statistics for table {} (version: {}) during DELETE execution: {}", - table.name, table.table_version_id, error - ); + let pruning_predicate = + PruningPredicate::try_new(predicate.clone(), schema_ref.clone())?; + let prune_map = pruning_predicate.prune(&table)?; + let files_to_prune = table + .get_state() + .files() + .iter() + .zip(prune_map.into_iter()) + .filter_map( + |(add, keep)| if keep { Some(add.clone()) } else { None }, + ) + .collect::>(); + + let base_scan = parquet_scan_from_actions( + &table, + files_to_prune.as_slice(), + schema_ref.as_ref(), + &[predicate.clone().not()], + &state, + None, + None, + ) + .await?; - // Fallback to scan + filter across all partitions - let filter_plan = table - .partition_filter_plan( - partitions, - filter.clone(), - &[predicate.clone().not()], - self.internal_object_store.inner.clone(), - ) - .await?; + let filter_plan = Arc::new(FilterExec::try_new(filter, base_scan)?); + + // Write the filtered out data + let adds = write_execution_plan( + &table, + state, + filter_plan, + vec![], + table.object_store(), + None, + None, + ) + .await?; - final_partition_ids = self - .execute_plan_to_partitions(&filter_plan, None) - .await?; - } - } + (adds, files_to_prune) + } else { + // If no qualifier is specified we're basically truncating the table. + // Remove all files. + (vec![], table.get_state().files().clone()) + }; - // Link the new table version with the corresponding partitions - self.partition_catalog - .append_partitions_to_table( - final_partition_ids, - new_table_version_id, - ) - .await?; + let deletion_timestamp = SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_millis() as i64; + + let mut actions: Vec = + adds.into_iter().map(Action::add).collect(); + for remove in removes { + actions.push(Action::remove(Remove { + path: remove.path, + deletion_timestamp: Some(deletion_timestamp), + data_change: true, + extended_file_metadata: Some(true), + partition_values: Some(remove.partition_values), + size: Some(remove.size), + tags: None, + })) } + let mut tx = table.create_transaction(None); + tx.add_actions(actions); + tx.commit(None, None).await?; + Ok(make_dummy_exec()) } LogicalPlan::DropTable(DropTable { @@ -1835,6 +2068,7 @@ impl SeafowlContext for DefaultSeafowlContext { ) .await?; + // TODO: Update table metadata with the new table name during writes, Ok(make_dummy_exec()) } SeafowlExtensionNode::DropSchema(DropSchema { name, .. }) => { @@ -1971,7 +2205,7 @@ impl SeafowlContext for DefaultSeafowlContext { self.reload_schema().await?; } - self.plan_to_delta_table(&plan, table_ref).await?; + self.plan_to_delta_table(table_ref, &plan).await?; Ok(()) } diff --git a/tests/statements/dml.rs b/tests/statements/dml.rs index 909ee587..d529cac8 100644 --- a/tests/statements/dml.rs +++ b/tests/statements/dml.rs @@ -210,16 +210,15 @@ async fn test_table_partitioning_and_rechunking() { assert_batches_eq!(expected, &results); } -#[ignore = "not yet implemented"] #[tokio::test] async fn test_delete_statement() { let context = make_context_with_pg(ObjectStoreType::InMemory).await; create_table_and_some_partitions(&context, "test_table", None).await; - // Check DELETE's query plan to make sure 46 (int) gets cast to a float value - // by the optimizer - // (NB: EXPLAIN isn't supported for user-defined nodes) + // + // Check DELETE's query plan to make sure 46 (int) gets cast to a float value by the optimizer + // let plan = context .create_logical_plan("DELETE FROM test_table WHERE some_value > 46") .await @@ -228,11 +227,11 @@ async fn test_delete_statement() { format!("{}", plan.display_indent()), r#"Dml: op=[Delete] table=[test_table] Filter: some_value > Float32(46) - TableScan: test_table projection=[some_bool_value, some_int_value, some_other_value, some_time, some_value], partial_filters=[some_value > Float32(46)]"# + TableScan: test_table projection=[some_time, some_value, some_other_value, some_bool_value, some_int_value], partial_filters=[some_value > Float32(46)]"# ); // - // Execute DELETE affecting partitions 2, 3 and creating table_version 6 + // Execute DELETE affecting partitions two of the partitions and creating new table_version // let plan = context .plan_query("DELETE FROM test_table WHERE some_value > 46") @@ -240,36 +239,9 @@ async fn test_delete_statement() { .unwrap(); context.collect(plan).await.unwrap(); - assert_partition_ids(&context, 6, vec![1, 4, 5]).await; - - let partitions = context - .partition_catalog - .load_table_partitions(6 as TableVersionId) - .await - .unwrap(); - - // Assert result of the new partition with id 5 - let results = scan_partition( - &context, - Some(&vec![4]), - partitions[2].clone(), - "test_table", - ) - .await; - let expected = vec![ - "+------------+", - "| some_value |", - "+------------+", - "| 45 |", - "| 46 |", - "| 46 |", - "+------------+", - ]; - assert_batches_eq!(expected, &results); - // Verify results let plan = context - .plan_query("SELECT some_value FROM test_table") + .plan_query("SELECT some_value FROM test_table ORDER BY some_value") .await .unwrap(); let results = context.collect(plan).await.unwrap(); @@ -278,12 +250,12 @@ async fn test_delete_statement() { "+------------+", "| some_value |", "+------------+", + "| 40 |", + "| 41 |", + "| 42 |", "| 42 |", "| 43 |", "| 44 |", - "| 42 |", - "| 41 |", - "| 40 |", "| 45 |", "| 46 |", "| 46 |", @@ -301,10 +273,15 @@ async fn test_delete_statement() { .unwrap(); context.collect(plan).await.unwrap(); - assert_partition_ids(&context, 7, vec![1, 4, 5]).await; + let plan = context + .plan_query("SELECT some_value FROM test_table ORDER BY some_value") + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + assert_batches_eq!(expected, &results); // - // Add another partition for table_version 8 + // Add another partition for a new table_version // let plan = context .plan_query("INSERT INTO test_table (some_value) VALUES (48), (49), (50)") @@ -312,8 +289,6 @@ async fn test_delete_statement() { .unwrap(); context.collect(plan).await.unwrap(); - assert_partition_ids(&context, 8, vec![1, 4, 5, 6]).await; - // // Execute DELETE not affecting only partition with id 4, while trimming/combining the rest // @@ -323,43 +298,21 @@ async fn test_delete_statement() { .unwrap(); context.collect(plan).await.unwrap(); - assert_partition_ids(&context, 9, vec![4, 7, 8]).await; - - // Verify new partition contents - let partitions = context - .partition_catalog - .load_table_partitions(9 as TableVersionId) + let plan = context + .plan_query("SELECT some_value FROM test_table ORDER BY some_value") .await .unwrap(); + let results = context.collect(plan).await.unwrap(); - let results = scan_partition( - &context, - Some(&vec![4]), - partitions[1].clone(), - "test_table", - ) - .await; let expected = vec![ "+------------+", "| some_value |", "+------------+", + "| 40 |", + "| 41 |", + "| 42 |", "| 42 |", "| 44 |", - "+------------+", - ]; - assert_batches_eq!(expected, &results); - - let results = scan_partition( - &context, - Some(&vec![4]), - partitions[2].clone(), - "test_table", - ) - .await; - let expected = vec![ - "+------------+", - "| some_value |", - "+------------+", "| 46 |", "| 46 |", "| 48 |", @@ -377,43 +330,20 @@ async fn test_delete_statement() { .unwrap(); context.collect(plan).await.unwrap(); - assert_partition_ids(&context, 10, vec![7, 9, 10]).await; - - // Verify new partition contents - let partitions = context - .partition_catalog - .load_table_partitions(10 as TableVersionId) + let plan = context + .plan_query("SELECT some_value FROM test_table ORDER BY some_value") .await .unwrap(); + let results = context.collect(plan).await.unwrap(); - let results = scan_partition( - &context, - Some(&vec![4]), - partitions[1].clone(), - "test_table", - ) - .await; let expected = vec![ "+------------+", "| some_value |", "+------------+", - "| 42 |", "| 41 |", - "+------------+", - ]; - assert_batches_eq!(expected, &results); - - let results = scan_partition( - &context, - Some(&vec![4]), - partitions[2].clone(), - "test_table", - ) - .await; - let expected = vec![ - "+------------+", - "| some_value |", - "+------------+", + "| 42 |", + "| 42 |", + "| 44 |", "| 46 |", "| 46 |", "+------------+", @@ -438,7 +368,6 @@ async fn test_delete_statement() { assert!(results.is_empty()); } -#[ignore = "not yet implemented"] #[tokio::test] async fn test_delete_with_string_filter_exact_match() { let context = make_context_with_pg(ObjectStoreType::InMemory).await; @@ -492,18 +421,11 @@ async fn test_delete_with_string_filter_exact_match() { .await .unwrap(); - let partitions = context - .partition_catalog - .load_table_partitions(5 as TableVersionId) + let plan = context + .plan_query("SELECT * FROM test_table ORDER BY value ASC") .await .unwrap(); - - // For some reason, the initial pruning in a DELETE doesn't discard the two - // partitions that definitely don't match (partition != 'two'), so we end up - // with a new partition (if we delete where value = 2, this does result in the other - // two partitions being kept as-is, so could have something to do with strings) - let results = - scan_partition(&context, None, partitions[0].clone(), "test_table").await; + let results = context.collect(plan).await.unwrap(); let expected = vec![ "+-----------+-------+", "| partition | value |", @@ -513,13 +435,6 @@ async fn test_delete_with_string_filter_exact_match() { "+-----------+-------+", ]; assert_batches_eq!(expected, &results); - - let plan = context - .plan_query("SELECT * FROM test_table ORDER BY value ASC") - .await - .unwrap(); - let results = context.collect(plan).await.unwrap(); - assert_batches_eq!(expected, &results); } #[ignore = "not yet implemented"] From fa65b169b918b4820802a8f97028dbd87d460de5 Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Thu, 9 Mar 2023 21:42:26 +0100 Subject: [PATCH 14/21] Enable UPDATE statements for delta tables --- src/context.rs | 261 +++++++++++++++++-------------------- tests/statements/dml.rs | 279 +++++++--------------------------------- tests/statements/mod.rs | 27 ---- 3 files changed, 169 insertions(+), 398 deletions(-) diff --git a/src/context.rs b/src/context.rs index 1097eb79..584adcad 100644 --- a/src/context.rs +++ b/src/context.rs @@ -9,7 +9,7 @@ use datafusion::datasource::TableProvider; use datafusion::parquet::basic::Compression; use itertools::Itertools; use object_store::local::LocalFileSystem; -use std::collections::{HashMap, HashSet}; +use std::collections::HashMap; use tokio::fs::File as AsyncFile; use tokio::io::{AsyncReadExt, AsyncWriteExt, BufReader}; @@ -28,7 +28,7 @@ use crate::datafusion::parser::{DFParser, Statement as DFStatement}; use crate::datafusion::utils::build_schema; use crate::object_store::http::try_prepare_http_url; use crate::object_store::wrapped::InternalObjectStore; -use crate::utils::{gc_partitions, group_partitions, hash_file}; +use crate::utils::{gc_partitions, hash_file}; use crate::wasm_udf::wasm::create_udf_from_wasm; use futures::{StreamExt, TryStreamExt}; @@ -51,7 +51,7 @@ use std::str::FromStr; use std::sync::Arc; use std::time::{SystemTime, UNIX_EPOCH}; -use datafusion::common::{DFSchema, ToDFSchema}; +use datafusion::common::DFSchema; use datafusion::datasource::file_format::avro::AvroFormat; use datafusion::datasource::file_format::csv::CsvFormat; use datafusion::datasource::file_format::file_type::{FileCompressionType, FileType}; @@ -108,13 +108,11 @@ use url::Url; use uuid::Uuid; use crate::catalog::{PartitionCatalog, DEFAULT_SCHEMA, STAGING_SCHEMA}; -use crate::data_types::PhysicalPartitionId; use crate::datafusion::visit::VisitorMut; #[cfg(test)] use crate::frontend::http::tests::deterministic_uuid; use crate::provider::{ - project_expressions, PartitionColumn, SeafowlPartition, SeafowlPruningStatistics, - SeafowlTable, + project_expressions, PartitionColumn, SeafowlPartition, SeafowlTable, }; use crate::wasm_udf::data_types::{get_volatility, CreateFunctionDetails}; use crate::{ @@ -943,10 +941,6 @@ impl DefaultSeafowlContext { }) } - fn get_internal_object_store(&self) -> Arc { - self.internal_object_store.clone() - } - /// Resolve a table reference into a Seafowl table pub async fn try_get_seafowl_table( &self, @@ -1124,43 +1118,6 @@ impl DefaultSeafowlContext { physical_plan.execute(partition, task_context) } - // Generate new physical Parquet partition files from the provided plan, upload to object store - // and persist partition metadata. - async fn execute_plan_to_partitions( - &self, - physical_plan: &Arc, - output_schema: Option, - ) -> Result> { - let disk_manager = self.inner.runtime_env().disk_manager.clone(); - let store = self.get_internal_object_store(); - - // Generate new physical partition objects - let partitions = plan_to_object_store( - &self.inner.state(), - physical_plan, - output_schema, - store, - disk_manager, - self.max_partition_size, - ) - .await?; - - debug!( - "execute_plan_to_partition completed, metrics: {:?}", - physical_plan.metrics() - ); - - // Record partition metadata to the catalog - self.partition_catalog - .create_partitions(partitions) - .await - .map_err(|e| { - DataFusionError::Execution(format!( - "Failed persisting partition metadata {e:?}" - )) - }) - } - // Copied from DataFusion's source code (private functions) async fn create_external_table( &self, @@ -1716,113 +1673,135 @@ impl SeafowlContext for DefaultSeafowlContext { input, .. }) => { - let table = self.try_get_seafowl_table(table_name.to_string()).await?; - // Destructure input into projection expressions and the upstream scan/filter plan let LogicalPlan::Projection(Projection { expr, input, .. }) = &**input else { return Err(DataFusionError::Plan("Update plan doesn't contain a Projection node".to_string())) }; - // Load all pre-existing partitions - let partitions = self - .partition_catalog - .load_table_partitions(table.table_version_id) - .await?; + // TODO: Once https://github.com/delta-io/delta-rs/issues/1126 is closed use the + // native delta-rs UPDATE op + let table_object_store = self + .inner + .table_provider(table_name) + .await? + .as_any() + .downcast_ref::() + .ok_or_else(|| { + DataFusionError::Execution( + "Table {table_name} not found".to_string(), + ) + })? + .object_store(); + // Can't just keep hold of the downcasted ref from above because of + // `temporary value dropped while borrowed` + let mut table = DeltaTable::new(table_object_store, Default::default()); + table.load().await?; - // By default (e.g. when there is no qualifier/selection, or we somehow - // fail to prune partitions) update all partitions - let mut partitions_to_update = HashSet::::from_iter( - partitions.iter().map(|p| p.partition_id.unwrap()), - ); - - let schema = table.schema().as_ref().clone(); - let df_schema = - DFSchema::try_from_qualified_schema(&table.name, &schema)?; - let mut selection_expr = None; - - // Try to scope down partition ids which need to be updated with pruning - if let LogicalPlan::Filter(Filter { predicate, .. }) = &**input { - selection_expr = Some(create_physical_expr( - &predicate.clone(), - &schema.clone().to_dfschema()?, - &schema, - &ExecutionProps::new(), - )?); - - match SeafowlPruningStatistics::from_partitions( - partitions.clone(), - table.schema(), - ) { - Ok(pruning_stats) => { - partitions_to_update = HashSet::from_iter( - pruning_stats - .prune(& [predicate.clone()]) - .await - .iter() - .map( | p| p.partition_id.unwrap()), - ); - } - Err(error) => warn ! ( - "Failed constructing pruning statistics for table {} (version: {}) during UPDATE execution: {}", - table.name, table.table_version_id, error + let schema_ref = TableProvider::schema(&table); + let df_schema = DFSchema::try_from_qualified_schema( + table_name.table(), + schema_ref.as_ref(), + )?; + + let selection_expr = + if let LogicalPlan::Filter(Filter { predicate, .. }) = &**input { + Some(create_physical_expr( + &predicate.clone(), + &df_schema, + schema_ref.as_ref(), + &ExecutionProps::new(), + )?) + } else { + None + }; + + let projections = project_expressions( + expr, + &df_schema, + schema_ref.as_ref(), + selection_expr, + )?; + let state = self.inner.state(); + + let (filters, removes) = + if let LogicalPlan::Filter(Filter { predicate, .. }) = &**input { + // A WHERE clause has been used; employ it to prune the update down to only + // a subset of files, while inheriting the rest from the previous version + let pruning_predicate = PruningPredicate::try_new( + predicate.clone(), + schema_ref.clone(), + )?; + let prune_map = pruning_predicate.prune(&table)?; + + let files_to_prune = table + .get_state() + .files() + .iter() + .zip(prune_map.into_iter()) + .filter_map( + |(add, keep)| if keep { Some(add.clone()) } else { None }, ) - } - } + .collect::>(); - let mut final_partition_ids = Vec::with_capacity(partitions.len()); + (vec![predicate.clone().not()], files_to_prune) + } else { + // If no qualifier is specified we're basically updating the whole table. + (vec![], table.get_state().files().clone()) + }; - let mut update_plan: Arc; - let projections = - project_expressions(expr, &df_schema, &schema, selection_expr)?; + if removes.is_empty() { + // Nothing to update + return Ok(make_dummy_exec()); + } - // Iterate over partitions, updating the ones affected by the selection, - // while re-using the rest - for (keep, group) in - group_partitions(partitions, |p: &SeafowlPartition| { - !partitions_to_update.contains(&p.partition_id.unwrap()) - }) - { - if keep { - // Inherit the partition(s) as is from the previous - // table version - final_partition_ids - .extend(group.iter().map(|p| p.partition_id.unwrap())); - continue; - } + let base_scan = parquet_scan_from_actions( + &table, + removes.as_slice(), + schema_ref.as_ref(), + filters.as_slice(), + &state, + None, + None, + ) + .await?; - let scan_plan = table - .partition_scan_plan( - None, - group, - &[], - None, - self.internal_object_store.inner.clone(), - ) - .await?; + // Apply the provided assignments + let update_plan = + Arc::new(ProjectionExec::try_new(projections.clone(), base_scan)?); + + // Write the updated data + let adds = write_execution_plan( + &table, + state, + update_plan, + vec![], + table.object_store(), + None, + None, + ) + .await?; - update_plan = Arc::new(ProjectionExec::try_new( - projections.clone(), - scan_plan, - )?); + let deletion_timestamp = SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_millis() as i64; - final_partition_ids.extend( - self.execute_plan_to_partitions( - &update_plan, - Some(table.schema()), - ) - .await?, - ); + let mut actions: Vec = + adds.into_iter().map(Action::add).collect(); + for remove in removes { + actions.push(Action::remove(Remove { + path: remove.path, + deletion_timestamp: Some(deletion_timestamp), + data_change: true, + extended_file_metadata: Some(true), + partition_values: Some(remove.partition_values), + size: Some(remove.size), + tags: None, + })) } - // Create a new blank table version - let new_table_version_id = self - .table_catalog - .create_new_table_version(table.table_version_id, false) - .await?; - - // Link the new table version with the corresponding partitions - self.partition_catalog - .append_partitions_to_table(final_partition_ids, new_table_version_id) - .await?; + let mut tx = table.create_transaction(None); + tx.add_actions(actions); + tx.commit(None, None).await?; Ok(make_dummy_exec()) } @@ -2978,7 +2957,7 @@ mod tests { .await .unwrap(); - let store = sf_context.get_internal_object_store(); + let store = sf_context.internal_object_store.clone(); assert_uploaded_objects( store, vec![Path::from(EXPECTED_INSERT_FILE_NAME.to_string())], diff --git a/tests/statements/dml.rs b/tests/statements/dml.rs index d529cac8..4a4fe1e9 100644 --- a/tests/statements/dml.rs +++ b/tests/statements/dml.rs @@ -37,179 +37,6 @@ async fn test_insert_two_different_schemas() { assert_batches_eq!(expected, &results); } -#[ignore = "not yet implemented"] -#[tokio::test] -async fn test_table_partitioning_and_rechunking() { - let context = make_context_with_pg(ObjectStoreType::InMemory).await; - - // Make table versions 1 and 2 - create_table_and_insert(&context, "test_table").await; - - // Make table version 3 - let plan = context - .plan_query( - "INSERT INTO test_table (some_int_value, some_value) VALUES - (4444, 45), - (5555, 46), - (6666, 47)", - ) - .await - .unwrap(); - context.collect(plan).await.unwrap(); - - let partitions = context - .partition_catalog - .load_table_partitions(3 as TableVersionId) - .await - .unwrap(); - - // Ensure we have 2 partitions, originating from 2 INSERTS - assert_eq!(partitions.len(), 2); - assert_eq!( - partitions[0].object_storage_id, - Arc::from(FILENAME_1.to_string()) - ); - assert_eq!(partitions[0].row_count, 3); - assert_eq!(partitions[0].columns.len(), 3); - assert_eq!( - partitions[1].object_storage_id, - Arc::from(FILENAME_2.to_string()) - ); - assert_eq!(partitions[1].row_count, 3); - assert_eq!(partitions[1].columns.len(), 2); - - // Test table_partitions system table shows correct state - let plan = context - .plan_query("SELECT * FROM system.table_partitions") - .await - .unwrap(); - let results = context.collect(plan).await.unwrap(); - - let expected_row_2 = format!("| public | test_table | 2 | 1 | {FILENAME_1} | 3 |"); - let expected_row_3 = format!("| public | test_table | 3 | 1 | {FILENAME_1} | 3 |"); - let expected_row_4 = format!("| public | test_table | 3 | 2 | {FILENAME_2} | 3 |"); - let expected = vec![ - "+--------------+------------+------------------+--------------------+--------------------------------------------------------------------------+-----------+", - "| table_schema | table_name | table_version_id | table_partition_id | object_storage_id | row_count |", - "+--------------+------------+------------------+--------------------+--------------------------------------------------------------------------+-----------+", - "| public | test_table | 1 | | | |", - expected_row_2.as_str(), - expected_row_3.as_str(), - expected_row_4.as_str(), - "+--------------+------------+------------------+--------------------+--------------------------------------------------------------------------+-----------+", - ]; - assert_batches_eq!(expected, &results); - - // - // Test partition pruning during scans works - // - - // Assert that only a single partition is going to be used - let plan = context - .plan_query( - "EXPLAIN SELECT some_value, some_int_value FROM test_table WHERE some_value > 45", - ) - .await - .unwrap(); - let results = context.collect(plan).await.unwrap(); - - let formatted = arrow::util::pretty::pretty_format_batches(results.as_slice()) - .unwrap() - .to_string(); - - let actual_lines: Vec<&str> = formatted.trim().lines().collect(); - assert_contains!( - actual_lines[10], - format!(r#"partitions={{1 group: [[{FILENAME_2:}]]}}"#) - ); - - // Assert query results - let plan = context - .plan_query( - "SELECT some_value, some_int_value FROM test_table WHERE some_value > 45", - ) - .await - .unwrap(); - let results = context.collect(plan).await.unwrap(); - - let expected = vec![ - "+------------+----------------+", - "| some_value | some_int_value |", - "+------------+----------------+", - "| 46 | 5555 |", - "| 47 | 6666 |", - "+------------+----------------+", - ]; - assert_batches_eq!(expected, &results); - - // - // Re-chunk by creating a new table - // - let plan = context - .plan_query("CREATE TABLE table_rechunked AS SELECT * FROM test_table") - .await - .unwrap(); - context.collect(plan).await.unwrap(); - - let partitions = context - .partition_catalog - .load_table_partitions(4 as TableVersionId) - .await - .unwrap(); - - // Ensure we have re-chunked the 2 partitions into 1 - assert_eq!(partitions.len(), 1); - assert_eq!( - partitions[0].object_storage_id, - Arc::from(FILENAME_RECHUNKED.to_string()) - ); - assert_eq!(partitions[0].row_count, 6); - assert_eq!(partitions[0].columns.len(), 5); - - // Test table_partitions system table shows correct state with filtering - let plan = context - .plan_query( - r#" - SELECT object_storage_id FROM system.table_partitions - WHERE table_version_id = 4 AND table_partition_id = 3 AND row_count = 6 - "#, - ) - .await - .unwrap(); - let results = context.collect(plan).await.unwrap(); - - let expected_row = format!("| {FILENAME_RECHUNKED} |"); - let expected = vec![ - "+--------------------------------------------------------------------------+", - "| object_storage_id |", - "+--------------------------------------------------------------------------+", - expected_row.as_str(), - "+--------------------------------------------------------------------------+", - ]; - assert_batches_eq!(expected, &results); - - // Ensure table contents - let plan = context - .plan_query("SELECT some_value, some_int_value FROM table_rechunked") - .await - .unwrap(); - let results = context.collect(plan).await.unwrap(); - - let expected = vec![ - "+------------+----------------+", - "| some_value | some_int_value |", - "+------------+----------------+", - "| 42 | 1111 |", - "| 43 | 2222 |", - "| 44 | 3333 |", - "| 45 | 4444 |", - "| 46 | 5555 |", - "| 47 | 6666 |", - "+------------+----------------+", - ]; - assert_batches_eq!(expected, &results); -} - #[tokio::test] async fn test_delete_statement() { let context = make_context_with_pg(ObjectStoreType::InMemory).await; @@ -437,65 +264,56 @@ async fn test_delete_with_string_filter_exact_match() { assert_batches_eq!(expected, &results); } -#[ignore = "not yet implemented"] #[tokio::test] async fn test_update_statement() { let context = make_context_with_pg(ObjectStoreType::InMemory).await; create_table_and_some_partitions(&context, "test_table", None).await; - // Check the UPDATE query plan to make sure IN (41, 42, 43) (int) get cast to a float value + // + // Execute UPDATE with a selection, affecting only some files + // let query = "UPDATE test_table SET some_time = '2022-01-01 21:21:21Z', some_int_value = 5555, some_value = some_value - 10 WHERE some_value IN (41, 42, 43)"; let plan = context.create_logical_plan(query).await.unwrap(); + + // Check the UPDATE query plan to make sure IN (41, 42, 43) (int) get cast to a float value assert_eq!( format!("{}", plan.display_indent()), r#"Dml: op=[Update] table=[test_table] - Projection: test_table.some_bool_value AS some_bool_value, Int64(5555) AS some_int_value, test_table.some_other_value AS some_other_value, Utf8("2022-01-01 21:21:21Z") AS some_time, test_table.some_value - Float32(10) AS some_value + Projection: Utf8("2022-01-01 21:21:21Z") AS some_time, test_table.some_value - Float32(10) AS some_value, test_table.some_other_value AS some_other_value, test_table.some_bool_value AS some_bool_value, Int64(5555) AS some_int_value Filter: some_value = Float32(43) OR some_value = Float32(42) OR some_value = Float32(41) TableScan: test_table"# ); - // - // Execute UPDATE with a selection, affecting partitions 1 and 4, and creating table_version 6 - // + // Now check the results let plan = context.plan_query(query).await.unwrap(); context.collect(plan).await.unwrap(); - assert_partition_ids(&context, 6, vec![2, 3, 5, 6]).await; - - // Verify new partition contents - let partitions = context - .partition_catalog - .load_table_partitions(6 as TableVersionId) + let plan = context + .plan_query("SELECT * FROM test_table ORDER BY some_value") .await .unwrap(); - - let results = - scan_partition(&context, None, partitions[2].clone(), "test_table").await; - let expected = vec![ - "+-----------------+----------------+------------------+---------------------+------------+", - "| some_bool_value | some_int_value | some_other_value | some_time | some_value |", - "+-----------------+----------------+------------------+---------------------+------------+", - "| | 5555 | | 2022-01-01T21:21:21 | 32 |", - "| | 5555 | | 2022-01-01T21:21:21 | 33 |", - "| | 3333 | | 2022-01-01T20:03:03 | 44 |", - "+-----------------+----------------+------------------+---------------------+------------+", - ]; - assert_batches_eq!(expected, &results); - - let results = - scan_partition(&context, None, partitions[3].clone(), "test_table").await; + let results = context.collect(plan).await.unwrap(); let expected = vec![ - "+-----------------+----------------+------------------+---------------------+------------+", - "| some_bool_value | some_int_value | some_other_value | some_time | some_value |", - "+-----------------+----------------+------------------+---------------------+------------+", - "| | 5555 | | 2022-01-01T21:21:21 | 32 |", - "| | 5555 | | 2022-01-01T21:21:21 | 31 |", - "| | | | | 40 |", - "+-----------------+----------------+------------------+---------------------+------------+" + "+---------------------+------------+------------------+-----------------+----------------+", + "| some_time | some_value | some_other_value | some_bool_value | some_int_value |", + "+---------------------+------------+------------------+-----------------+----------------+", + "| 2022-01-01T21:21:21 | 31 | | | 5555 |", + "| 2022-01-01T21:21:21 | 32 | | | 5555 |", + "| 2022-01-01T21:21:21 | 32 | | | 5555 |", + "| 2022-01-01T21:21:21 | 33 | | | 5555 |", + "| | 40 | | | |", + "| 2022-01-01T20:03:03 | 44 | | | 3333 |", + "| | 45 | | | |", + "| | 46 | | | |", + "| | 46 | | | |", + "| | 47 | | | |", + "| | 47 | | | |", + "| | 48 | | | |", + "+---------------------+------------+------------------+-----------------+----------------+", ]; assert_batches_eq!(expected, &results); @@ -508,7 +326,12 @@ async fn test_update_statement() { .unwrap(); context.collect(plan).await.unwrap(); - assert_partition_ids(&context, 7, vec![2, 3, 5, 6]).await; + let plan = context + .plan_query("SELECT * FROM test_table ORDER BY some_value") + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + assert_batches_eq!(expected, &results); // // Execute UPDATE that causes an error during planning/execution, to test that the subsequent @@ -524,8 +347,7 @@ async fn test_update_statement() { .contains("Cannot cast string 'nope' to value of Decimal128(38, 10) type")); // - // Execute complex UPDATE (redundant assignment and a case assignment) without a selection, - // creating new table_version with a single new partition + // Execute complex UPDATE (redundant assignment and a case assignment) without a selection // let plan = context .plan_query( @@ -536,8 +358,6 @@ async fn test_update_statement() { .unwrap(); context.collect(plan).await.unwrap(); - assert_partition_ids(&context, 8, vec![7]).await; - // Verify results let plan = context .plan_query("SELECT * FROM test_table") @@ -546,27 +366,26 @@ async fn test_update_statement() { let results = context.collect(plan).await.unwrap(); let expected = vec![ - "+-----------------+----------------+------------------+---------------------+------------+", - "| some_bool_value | some_int_value | some_other_value | some_time | some_value |", - "+-----------------+----------------+------------------+---------------------+------------+", - "| | | 0.0000000000 | | 42 |", - "| | | 0.0000000000 | | 42 |", - "| | | 0.0000000000 | | 42 |", - "| | | 0.0000000000 | | 42 |", - "| | | 0.0000000000 | | 42 |", - "| | | 0.0000000000 | | 42 |", - "| true | 5555 | 5.5550000000 | 2022-01-01T21:21:21 | 42 |", - "| true | 5555 | 5.5550000000 | 2022-01-01T21:21:21 | 42 |", - "| false | 3333 | 3.3330000000 | 2022-01-01T20:03:03 | 42 |", - "| true | 5555 | 5.5550000000 | 2022-01-01T21:21:21 | 42 |", - "| true | 5555 | 5.5550000000 | 2022-01-01T21:21:21 | 42 |", - "| | | 0.0000000000 | | 42 |", - "+-----------------+----------------+------------------+---------------------+------------+", + "+---------------------+------------+------------------+-----------------+----------------+", + "| some_time | some_value | some_other_value | some_bool_value | some_int_value |", + "+---------------------+------------+------------------+-----------------+----------------+", + "| | 42 | 0.0000000000 | | |", + "| | 42 | 0.0000000000 | | |", + "| | 42 | 0.0000000000 | | |", + "| | 42 | 0.0000000000 | | |", + "| | 42 | 0.0000000000 | | |", + "| | 42 | 0.0000000000 | | |", + "| 2022-01-01T21:21:21 | 42 | 5.5550000000 | true | 5555 |", + "| 2022-01-01T21:21:21 | 42 | 5.5550000000 | true | 5555 |", + "| 2022-01-01T20:03:03 | 42 | 3.3330000000 | false | 3333 |", + "| 2022-01-01T21:21:21 | 42 | 5.5550000000 | true | 5555 |", + "| 2022-01-01T21:21:21 | 42 | 5.5550000000 | true | 5555 |", + "| | 42 | 0.0000000000 | | |", + "+---------------------+------------+------------------+-----------------+----------------+", ]; assert_batches_eq!(expected, &results); } -#[ignore = "not yet implemented"] #[tokio::test] async fn test_update_statement_errors() { let context = make_context_with_pg(ObjectStoreType::InMemory).await; diff --git a/tests/statements/mod.rs b/tests/statements/mod.rs index dfb1485f..99f5db20 100644 --- a/tests/statements/mod.rs +++ b/tests/statements/mod.rs @@ -28,7 +28,6 @@ use seafowl::config::schema::load_config_from_string; use seafowl::context::DefaultSeafowlContext; use seafowl::context::SeafowlContext; use seafowl::data_types::{TableVersionId, Timestamp}; -use seafowl::provider::SeafowlPartition; use seafowl::repository::postgres::testutils::get_random_schema; use seafowl::system_tables::SYSTEM_SCHEMA; @@ -46,10 +45,6 @@ mod vacuum; // Object store IDs for frequently-used test data const FILENAME_1: &str = "7fbfeeeade71978b4ae82cd3d97b8c1bd9ae7ab9a7a78ee541b66209cfd7722d.parquet"; -const FILENAME_2: &str = - "48b15ba6156370b0f4cf0522fcdaedebfedb3a99aca2222be722e49264c476c1.parquet"; -const FILENAME_RECHUNKED: &str = - "ece2031a3e121f8e4e31b2f2ef632a92b00dc479a6643172da8950def9fdb16e.parquet"; enum ObjectStoreType { Local(String), @@ -274,28 +269,6 @@ async fn create_table_and_some_partitions( (version_results, version_timestamps) } -// A helper function for asserting contents of a given partition -async fn scan_partition( - context: &DefaultSeafowlContext, - projection: Option<&Vec>, - partition: SeafowlPartition, - table_name: &str, -) -> Vec { - let table = context.try_get_seafowl_table(table_name).await.unwrap(); - let plan = table - .partition_scan_plan( - projection, - vec![partition], - &[], - None, - context.internal_object_store.inner.clone(), - ) - .await - .unwrap(); - - context.collect(plan).await.unwrap() -} - // Used for checking partition ids making up a given table version async fn assert_partition_ids( context: &DefaultSeafowlContext, From dd09c7bacaf69d0a9f1f814e10136f3359d65632 Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Thu, 9 Mar 2023 22:40:58 +0100 Subject: [PATCH 15/21] Skip eager deletion of schema files used initially --- src/context.rs | 81 ++++++++++++++++++++++--------------------------- src/provider.rs | 2 +- 2 files changed, 38 insertions(+), 45 deletions(-) diff --git a/src/context.rs b/src/context.rs index 584adcad..d8922d65 100644 --- a/src/context.rs +++ b/src/context.rs @@ -595,8 +595,8 @@ pub async fn write_execution_plan( .collect::>()) } -// Appropriated from https://github.com/delta-io/delta-rs/pull/1176; once the DELETE and UPDATE ops -// are available through delta-rs this will be obsolete. +// Appropriated from https://github.com/delta-io/delta-rs/pull/1176 with minor changes. +// Once the DELETE and UPDATE ops are available through delta-rs this will be obsolete. /// Create a Parquet scan limited to a set of files pub async fn parquet_scan_from_actions( table: &DeltaTable, @@ -958,6 +958,27 @@ impl DefaultSeafowlContext { Ok(seafowl_table.clone()) } + /// Resolve a table reference into a Delta table + pub async fn try_get_delta_table<'a>( + &self, + table_name: impl Into>, + ) -> Result { + let table_object_store = self + .inner + .table_provider(table_name) + .await? + .as_any() + .downcast_ref::() + .ok_or_else(|| { + DataFusionError::Execution("Table {table_name} not found".to_string()) + })? + .object_store(); + + // We can't just keep hold of the downcasted ref from above because of + // `temporary value dropped while borrowed` + Ok(DeltaTable::new(table_object_store, Default::default())) + } + // Parse the uuid from the Delta table uri if available async fn get_table_uuid<'a>( &self, @@ -1059,6 +1080,7 @@ impl DefaultSeafowlContext { let table_uuid = self.get_table_uuid(name).await?; let table_object_store = self.internal_object_store.for_delta_table(table_uuid); + // We're not exposing `target_file_size` nor `write_batch_size` atm let table = WriteBuilder::new() .with_input_execution_plan(plan.clone()) .with_input_session_state(self.inner.state()) @@ -1647,6 +1669,9 @@ impl SeafowlContext for DefaultSeafowlContext { let physical = self.create_physical_plan(input).await?; // First create the table and then insert the data from the subqeury + // TODO: this means we'll have 2 table versions at the end, 1st from the create + // and 2nd from the insert, while it seems more reasonable that in this case we have + // only one let _table = self .create_delta_table(name, physical.schema().as_ref()) .await?; @@ -1679,21 +1704,8 @@ impl SeafowlContext for DefaultSeafowlContext { // TODO: Once https://github.com/delta-io/delta-rs/issues/1126 is closed use the // native delta-rs UPDATE op - let table_object_store = self - .inner - .table_provider(table_name) - .await? - .as_any() - .downcast_ref::() - .ok_or_else(|| { - DataFusionError::Execution( - "Table {table_name} not found".to_string(), - ) - })? - .object_store(); - // Can't just keep hold of the downcasted ref from above because of - // `temporary value dropped while borrowed` - let mut table = DeltaTable::new(table_object_store, Default::default()); + + let mut table = self.try_get_delta_table(table_name).await?; table.load().await?; let schema_ref = TableProvider::schema(&table); @@ -1812,21 +1824,8 @@ impl SeafowlContext for DefaultSeafowlContext { input, }) => { // TODO: Once https://github.com/delta-io/delta-rs/pull/1176 is merged use that instead - let table_object_store = self - .inner - .table_provider(table_name) - .await? - .as_any() - .downcast_ref::() - .ok_or_else(|| { - DataFusionError::Execution( - "Table {table_name} not found".to_string(), - ) - })? - .object_store(); - // Can't just keep hold of the downcasted ref from above because of - // `temporary value dropped while borrowed` - let mut table = DeltaTable::new(table_object_store, Default::default()); + + let mut table = self.try_get_delta_table(table_name).await?; table.load().await?; let schema_ref = SchemaRef::from(table_schema.deref().clone()); @@ -1946,6 +1945,8 @@ impl SeafowlContext for DefaultSeafowlContext { DataFusionError::Execution("Table {name} not found".to_string()) })?; + // TODO: We should keep track of dropped delta tables, so that we can do lazy deletion + // of actual files in that path via `VACUUM` at some point. self.table_catalog.drop_table(table_id).await?; Ok(make_dummy_exec()) } @@ -2056,18 +2057,10 @@ impl SeafowlContext for DefaultSeafowlContext { .get_collection_id_by_name(&self.database, name) .await? { - let schema_prefix = - Path::from(format!("{}/{}", &self.database, name)); - // This is very bad. - self.internal_object_store - .delete_in_prefix(&schema_prefix) - .await - .map_err(|err| { - DataFusionError::Execution(format!( - "Failed to delete objects in schema {name}: {}", - err - )) - })?; + // Similar as for DROP TABLE; we should really only flag all tables + // in the schema as DROPED, and then try to lazy delete them during + // subsequent `VACUUM`s. Only once all the contained tables directories + // are deleted is it safe to drop the collection. self.table_catalog.drop_collection(collection_id).await? }; diff --git a/src/provider.rs b/src/provider.rs index 5bca003f..f8d2449d 100644 --- a/src/provider.rs +++ b/src/provider.rs @@ -145,7 +145,7 @@ impl SchemaProvider for SeafowlCollection { let table_object_store = match self.tables.read().get(name) { None => return None, Some(table) => match table.as_any().downcast_ref::() { - // This shouldn't happen since we stsore only DeltaTable's in the map + // This shouldn't happen since we store only DeltaTable's in the map None => return Some(table.clone()), Some(delta_table) => { if delta_table.version() != -1 { From 6b54c1d24d6d3ca8600bd4deea3de497a43f2876 Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Mon, 13 Mar 2023 07:14:06 +0100 Subject: [PATCH 16/21] Implement repository changes needed for lazy cleanup of dropped tables --- ...0221081920_delta_lake_integration.down.sql | 2 + ...230221081920_delta_lake_integration.up.sql | 11 +- ...0221081928_delta_lake_integration.down.sql | 2 + ...230221081928_delta_lake_integration.up.sql | 11 +- src/catalog.rs | 18 +++ src/context.rs | 6 +- src/repository/default.rs | 57 ++++++- src/repository/interface.rs | 9 ++ tests/statements/query_legacy.rs | 146 ++++++++++++++++++ 9 files changed, 254 insertions(+), 8 deletions(-) diff --git a/migrations/postgres/20230221081920_delta_lake_integration.down.sql b/migrations/postgres/20230221081920_delta_lake_integration.down.sql index eb408f1e..7e3b0a47 100644 --- a/migrations/postgres/20230221081920_delta_lake_integration.down.sql +++ b/migrations/postgres/20230221081920_delta_lake_integration.down.sql @@ -1,3 +1,5 @@ -- Add down migration script here ALTER TABLE "table" DROP COLUMN legacy; ALTER TABLE "table" DROP COLUMN uuid; + +DROP TABLE dropped_table; \ No newline at end of file diff --git a/migrations/postgres/20230221081920_delta_lake_integration.up.sql b/migrations/postgres/20230221081920_delta_lake_integration.up.sql index ee8247cb..130b6332 100644 --- a/migrations/postgres/20230221081920_delta_lake_integration.up.sql +++ b/migrations/postgres/20230221081920_delta_lake_integration.up.sql @@ -1,5 +1,14 @@ -- Add up migration script here ALTER TABLE "table" ADD COLUMN legacy BOOLEAN DEFAULT FALSE; ALTER TABLE "table" ADD COLUMN uuid UUID NOT NULL DEFAULT '00000000-0000-0000-0000-000000000000'; - UPDATE "table" SET legacy = TRUE; + +-- Table for facilitating soft-dropping of tables, via deferring the actual file deletion for later. +CREATE TABLE dropped_table ( + database_name VARCHAR NOT NULL, + collection_name VARCHAR NOT NULL, + name VARCHAR NOT NULL, + uuid UUID NOT NULL, + deletion_status VARCHAR DEFAULT 'PENDING' CHECK ( deletion_status in ('PENDING', 'RETRY', 'FAILED') ), + drop_time TIMESTAMP WITH TIME ZONE NOT NULL DEFAULT(now()) +); diff --git a/migrations/sqlite/20230221081928_delta_lake_integration.down.sql b/migrations/sqlite/20230221081928_delta_lake_integration.down.sql index eb408f1e..0c59763d 100644 --- a/migrations/sqlite/20230221081928_delta_lake_integration.down.sql +++ b/migrations/sqlite/20230221081928_delta_lake_integration.down.sql @@ -1,3 +1,5 @@ -- Add down migration script here ALTER TABLE "table" DROP COLUMN legacy; ALTER TABLE "table" DROP COLUMN uuid; + +DROP TABLE dropped_table; diff --git a/migrations/sqlite/20230221081928_delta_lake_integration.up.sql b/migrations/sqlite/20230221081928_delta_lake_integration.up.sql index 019e86d4..653ca9c4 100644 --- a/migrations/sqlite/20230221081928_delta_lake_integration.up.sql +++ b/migrations/sqlite/20230221081928_delta_lake_integration.up.sql @@ -5,5 +5,14 @@ ALTER TABLE "table" ADD COLUMN legacy BOOLEAN DEFAULT FALSE; -- On the other hand, while decoding TEXT to uuid::fmt::Hyphenated is supported in SQLite it isn't in Postgres, so this is -- the only approach that works for now. ALTER TABLE "table" ADD COLUMN uuid BLOB NOT NULL DEFAULT x'00000000000000000000000000000000'; - UPDATE "table" SET legacy = TRUE; + +-- Table for facilitating soft-dropping of tables, via deferring the actual file deletion for later. +CREATE TABLE dropped_table ( + database_name VARCHAR NOT NULL, + collection_name VARCHAR NOT NULL, + name VARCHAR NOT NULL, + uuid BLOB NOT NULL, + deletion_status VARCHAR DEFAULT 'PENDING' CHECK ( deletion_status in ('PENDING', 'RETRY', 'FAILED') ), + drop_time INTEGER(4) NOT NULL DEFAULT((strftime('%s','now'))) +); diff --git a/src/catalog.rs b/src/catalog.rs index 0e6742be..f7858464 100644 --- a/src/catalog.rs +++ b/src/catalog.rs @@ -46,6 +46,7 @@ pub enum Error { DatabaseDoesNotExist { id: DatabaseId }, CollectionDoesNotExist { id: CollectionId }, TableDoesNotExist { id: TableId }, + TableUuidDoesNotExist { uuid: Uuid }, TableVersionDoesNotExist { id: TableVersionId }, // We were inserting a vector of partitions and can't find which one // caused the error without parsing the error message, so just @@ -115,6 +116,9 @@ impl From for DataFusionError { Error::TableDoesNotExist { id } => { DataFusionError::Internal(format!("Table with ID {id} doesn't exist")) } + Error::TableUuidDoesNotExist { uuid } => { + DataFusionError::Internal(format!("Table with UUID {uuid} doesn't exist")) + } // Raised by append_partitions_to_table and create_new_table_version (non-existent version), also internal issue Error::TableVersionDoesNotExist { id } => DataFusionError::Internal(format!( "Table version with ID {id} doesn't exist" @@ -234,6 +238,8 @@ pub trait TableCatalog: Sync + Send { async fn drop_collection(&self, collection_id: CollectionId) -> Result<()>; async fn drop_database(&self, database_id: DatabaseId) -> Result<()>; + + async fn delete_dropped_table(&self, uuid: Uuid) -> Result<()>; } #[cfg_attr(test, automock)] @@ -724,6 +730,18 @@ impl TableCatalog for DefaultCatalog { _ => Self::to_sqlx_error(e), }) } + + async fn delete_dropped_table(&self, uuid: Uuid) -> Result<()> { + self.repository + .delete_dropped_table(uuid) + .await + .map_err(|e| match e { + RepositoryError::SqlxError(sqlx::error::Error::RowNotFound) => { + Error::TableUuidDoesNotExist { uuid } + } + _ => Self::to_sqlx_error(e), + }) + } } #[async_trait] diff --git a/src/context.rs b/src/context.rs index d8922d65..30089b10 100644 --- a/src/context.rs +++ b/src/context.rs @@ -1945,8 +1945,6 @@ impl SeafowlContext for DefaultSeafowlContext { DataFusionError::Execution("Table {name} not found".to_string()) })?; - // TODO: We should keep track of dropped delta tables, so that we can do lazy deletion - // of actual files in that path via `VACUUM` at some point. self.table_catalog.drop_table(table_id).await?; Ok(make_dummy_exec()) } @@ -2057,9 +2055,9 @@ impl SeafowlContext for DefaultSeafowlContext { .get_collection_id_by_name(&self.database, name) .await? { - // Similar as for DROP TABLE; we should really only flag all tables + // TODO: as for DROP TABLE, we should really only flag all tables // in the schema as DROPED, and then try to lazy delete them during - // subsequent `VACUUM`s. Only once all the contained tables directories + // subsequent `VACUUM`s. Only once all the underlying table files // are deleted is it safe to drop the collection. self.table_catalog.drop_collection(collection_id).await? }; diff --git a/src/repository/default.rs b/src/repository/default.rs index 9aabb712..cc400275 100644 --- a/src/repository/default.rs +++ b/src/repository/default.rs @@ -605,12 +605,12 @@ impl Repository for $repo { } // Drop table/collection/database - // Currently we actually delete these, though we could mark them as deleted - // to allow for undeletion // In these methods, return the ID back so that we get an error if the // table/collection/schema didn't actually exist async fn drop_table(&self, table_id: TableId) -> Result<(), Error> { + self.insert_dropped_tables(Some(table_id), None, None).await?; + sqlx::query("DELETE FROM \"table\" WHERE id = $1 RETURNING id") .bind(table_id) .fetch_one(&self.executor) @@ -619,6 +619,8 @@ impl Repository for $repo { } async fn drop_collection(&self, collection_id: CollectionId) -> Result<(), Error> { + self.insert_dropped_tables(None, Some(collection_id), None).await?; + sqlx::query("DELETE FROM collection WHERE id = $1 RETURNING id") .bind(collection_id) .fetch_one(&self.executor) @@ -627,12 +629,63 @@ impl Repository for $repo { } async fn drop_database(&self, database_id: DatabaseId) -> Result<(), Error> { + self.insert_dropped_tables(None, None, Some(database_id)).await?; + sqlx::query("DELETE FROM database WHERE id = $1 RETURNING id") .bind(database_id) .fetch_one(&self.executor) .await.map_err($repo::interpret_error)?; Ok(()) } + + async fn insert_dropped_tables( + &self, + maybe_table_id: Option, + maybe_collection_id: Option, + maybe_database_id: Option, + ) -> Result<(), Error> { + // Currently we hard delete only legacy tables, the others are soft-deleted by moving + // them to a special table that is used for lazy cleanup of files via `VACUUM`. + // TODO: We could do this via a trigger, but then we'd lose the ability to actually + // perform hard deletes at the DB-level. + // NB: we also persist db/col name on the off chance that we want to add table restore/undrop + // at some point. + let mut builder: QueryBuilder<_> = QueryBuilder::new( + r#"INSERT INTO dropped_table(database_name, collection_name, name, uuid) + SELECT * FROM ( + SELECT database.name, collection.name, "table".name, "table".uuid + FROM "table" + JOIN collection ON "table".collection_id = collection.id + JOIN database ON collection.database_id = database.id + WHERE "table".legacy IS FALSE AND "#, + ); + + if let Some(table_id) = maybe_table_id { + builder.push("\"table\".id = "); + builder.push_bind(table_id); + } else if let Some(collection_id) = maybe_collection_id { + builder.push("collection.id = "); + builder.push_bind(collection_id); + } else { + let database_id = maybe_database_id.unwrap(); + builder.push("database.id = "); + builder.push_bind(database_id); + } + + builder.push(") as table_to_drop"); + + let query = builder.build(); + query.execute(&self.executor).await.map_err($repo::interpret_error)?; + Ok(()) + } + + async fn delete_dropped_table(&self, uuid: Uuid) -> Result<(), Error> { + sqlx::query("DELETE FROM dropped_table WHERE uuid = $1 RETURNING uuid") + .bind(uuid) + .fetch_one(&self.executor) + .await.map_err($repo::interpret_error)?; + Ok(()) + } } }; diff --git a/src/repository/interface.rs b/src/repository/interface.rs index 1e9175c1..8d169fa1 100644 --- a/src/repository/interface.rs +++ b/src/repository/interface.rs @@ -204,6 +204,15 @@ pub trait Repository: Send + Sync + Debug { async fn drop_collection(&self, collection_id: CollectionId) -> Result<(), Error>; async fn drop_database(&self, database_id: DatabaseId) -> Result<(), Error>; + + async fn insert_dropped_tables( + &self, + maybe_table_id: Option, + maybe_collection_id: Option, + maybe_database_id: Option, + ) -> Result<(), Error>; + + async fn delete_dropped_table(&self, uuid: Uuid) -> Result<(), Error>; } #[cfg(test)] diff --git a/tests/statements/query_legacy.rs b/tests/statements/query_legacy.rs index 53e3b447..cc0c631f 100644 --- a/tests/statements/query_legacy.rs +++ b/tests/statements/query_legacy.rs @@ -412,3 +412,149 @@ async fn test_legacy_tables() { ]; assert_batches_eq!(expected, &results); } + +#[tokio::test] +async fn test_vacuum_legacy_tables() { + let data_dir = TempDir::new().unwrap(); + + let context = Arc::new( + make_context_with_local_sqlite(data_dir.path().display().to_string()).await, + ); + + let plan = context + .plan_query("SELECT * FROM system.table_versions") + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let expected = vec![ + "+--------------+------------+------------------+---------------------+", + "| table_schema | table_name | table_version_id | creation_time |", + "+--------------+------------+------------------+---------------------+", + "| public | test_table | 1 | 2023-03-07T08:44:49 |", + "| public | test_table | 2 | 2023-03-07T08:44:49 |", + "| public | test_table | 3 | 2023-03-07T08:44:51 |", + "| public | test_table | 4 | 2023-03-07T08:44:53 |", + "| public | test_table | 5 | 2023-03-07T08:44:55 |", + "+--------------+------------+------------------+---------------------+", + ]; + + assert_batches_eq!(expected, &results); + + context + .collect(context.plan_query("VACUUM TABLES").await.unwrap()) + .await + .unwrap(); + + let plan = context + .plan_query("SELECT * FROM system.table_versions") + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let expected = vec![ + "+--------------+------------+------------------+---------------------+", + "| table_schema | table_name | table_version_id | creation_time |", + "+--------------+------------+------------------+---------------------+", + "| public | test_table | 5 | 2023-03-07T08:44:55 |", + "+--------------+------------+------------------+---------------------+", + ]; + + assert_batches_eq!(expected, &results); + + // Make sure vacuuming partitions and all tables changes nothing + context + .collect(context.plan_query("VACUUM PARTITIONS").await.unwrap()) + .await + .unwrap(); + context + .collect(context.plan_query("VACUUM TABLES").await.unwrap()) + .await + .unwrap(); + + // Check the test_table is still queryable + let plan = context + .plan_query("SELECT some_value FROM test_table ORDER BY some_value LIMIT 4") + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let expected = vec![ + "+------------+", + "| some_value |", + "+------------+", + "| 40 |", + "| 41 |", + "| 42 |", + "| 42 |", + "+------------+", + ]; + + assert_batches_eq!(expected, &results); + + // DROP TABLE + context + .collect(context.plan_query("DROP TABLE test_table").await.unwrap()) + .await + .unwrap(); + + let plan = context + .plan_query("SELECT * FROM system.table_versions") + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let expected = vec![ + "+--------------+------------+------------------+---------------+", + "| table_schema | table_name | table_version_id | creation_time |", + "+--------------+------------+------------------+---------------+", + "+--------------+------------+------------------+---------------+", + ]; + + assert_batches_eq!(expected, &results); + + let get_object_metas = || async { + context + .internal_object_store + .inner + .list(None) + .await + .unwrap() + .try_collect::>() + .await + .unwrap() + }; + + // Check we have orphan partitions + // NB: we deduplicate object storage IDs here to avoid running the DELETE call + // twice, but we can have two different partition IDs with same object storage ID + // See https://github.com/splitgraph/seafowl/issues/5 + let orphans = vec![ + "ea192fa7ae3b4abca9ded70e480c188e2c260ece02a810e5f1e2be41b0d6c0f6.parquet", + "534e5cc396e5b24725993145821b864cbfb07c2d8d7116f3d60d28bc02900861.parquet", + "9ae6f4222893474551037d0e44ff223ca5ea8e703d141b14835025923a66ab50.parquet", + "7fbfeeeade71978b4ae82cd3d97b8c1bd9ae7ab9a7a78ee541b66209cfd7722d.parquet", + ]; + + assert_orphan_partitions(context.clone(), orphans.clone()).await; + let object_metas = get_object_metas().await; + assert_eq!(object_metas.len(), 7); + for object_meta in object_metas { + // We're skipping the 3 seafowl.sqlite* files + let path = object_meta.location.to_string(); + if !path.starts_with("seafowl.sqlite") { + assert!(orphans.contains(&path.as_str())); + } + } + + // Run vacuum on partitions + context + .collect(context.plan_query("VACUUM PARTITIONS").await.unwrap()) + .await + .unwrap(); + + // Ensure no orphan partitions are left + assert_orphan_partitions(context.clone(), vec![]).await; + let object_metas = get_object_metas().await; + assert_eq!(object_metas.len(), 3); +} From b3e71ddb1dd9af4a434d5ec9cad4f13e50fdf0d1 Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Mon, 13 Mar 2023 10:34:32 +0100 Subject: [PATCH 17/21] Add dropped_tables system table and extend tests --- ...230221081920_delta_lake_integration.up.sql | 2 +- ...230221081928_delta_lake_integration.up.sql | 2 +- src/catalog.rs | 17 ++- src/context.rs | 29 +++++ src/repository/default.rs | 43 +++++++- src/repository/interface.rs | 15 +++ src/repository/postgres.rs | 16 +-- src/repository/sqlite.rs | 16 +-- src/system_tables.rs | 104 +++++++++++++++++- tests/statements/ddl.rs | 20 ++++ tests/statements/query.rs | 6 + 11 files changed, 235 insertions(+), 35 deletions(-) diff --git a/migrations/postgres/20230221081920_delta_lake_integration.up.sql b/migrations/postgres/20230221081920_delta_lake_integration.up.sql index 130b6332..a251c29b 100644 --- a/migrations/postgres/20230221081920_delta_lake_integration.up.sql +++ b/migrations/postgres/20230221081920_delta_lake_integration.up.sql @@ -7,7 +7,7 @@ UPDATE "table" SET legacy = TRUE; CREATE TABLE dropped_table ( database_name VARCHAR NOT NULL, collection_name VARCHAR NOT NULL, - name VARCHAR NOT NULL, + table_name VARCHAR NOT NULL, uuid UUID NOT NULL, deletion_status VARCHAR DEFAULT 'PENDING' CHECK ( deletion_status in ('PENDING', 'RETRY', 'FAILED') ), drop_time TIMESTAMP WITH TIME ZONE NOT NULL DEFAULT(now()) diff --git a/migrations/sqlite/20230221081928_delta_lake_integration.up.sql b/migrations/sqlite/20230221081928_delta_lake_integration.up.sql index 653ca9c4..460c3095 100644 --- a/migrations/sqlite/20230221081928_delta_lake_integration.up.sql +++ b/migrations/sqlite/20230221081928_delta_lake_integration.up.sql @@ -11,7 +11,7 @@ UPDATE "table" SET legacy = TRUE; CREATE TABLE dropped_table ( database_name VARCHAR NOT NULL, collection_name VARCHAR NOT NULL, - name VARCHAR NOT NULL, + table_name VARCHAR NOT NULL, uuid BLOB NOT NULL, deletion_status VARCHAR DEFAULT 'PENDING' CHECK ( deletion_status in ('PENDING', 'RETRY', 'FAILED') ), drop_time INTEGER(4) NOT NULL DEFAULT((strftime('%s','now'))) diff --git a/src/catalog.rs b/src/catalog.rs index f7858464..be57a4b1 100644 --- a/src/catalog.rs +++ b/src/catalog.rs @@ -14,7 +14,7 @@ use uuid::Uuid; use crate::object_store::wrapped::InternalObjectStore; use crate::provider::SeafowlFunction; -use crate::repository::interface::TablePartitionsResult; +use crate::repository::interface::{DroppedTablesResult, TablePartitionsResult}; use crate::system_tables::SystemSchemaProvider; use crate::wasm_udf::data_types::{ CreateFunctionDataType, CreateFunctionDetails, CreateFunctionLanguage, @@ -239,6 +239,11 @@ pub trait TableCatalog: Sync + Send { async fn drop_database(&self, database_id: DatabaseId) -> Result<()>; + async fn get_dropped_tables( + &self, + database_name: &str, + ) -> Result>; + async fn delete_dropped_table(&self, uuid: Uuid) -> Result<()>; } @@ -731,6 +736,16 @@ impl TableCatalog for DefaultCatalog { }) } + async fn get_dropped_tables( + &self, + database_name: &str, + ) -> Result> { + self.repository + .get_dropped_tables(database_name) + .await + .map_err(Self::to_sqlx_error) + } + async fn delete_dropped_table(&self, uuid: Uuid) -> Result<()> { self.repository .delete_dropped_table(uuid) diff --git a/src/context.rs b/src/context.rs index 30089b10..fa7a79c4 100644 --- a/src/context.rs +++ b/src/context.rs @@ -2690,6 +2690,35 @@ mod tests { \n TableScan: testdb.testcol.some_table projection=[date, value]"); } + #[tokio::test] + async fn test_drop_table_pending_deletion() -> Result<()> { + let context = Arc::new(in_memory_context().await); + let plan = context + .plan_query("CREATE TABLE test_table (\"key\" INTEGER, value STRING)") + .await + .unwrap(); + context.collect(plan).await.unwrap(); + let plan = context.plan_query("DROP TABLE test_table").await.unwrap(); + context.collect(plan).await.unwrap(); + + let plan = context + .plan_query("SELECT table_schema, table_name, uuid, deletion_status FROM system.dropped_tables") + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let expected = vec![ + "+--------------+------------+--------------------------------------+-----------------+", + "| table_schema | table_name | uuid | deletion_status |", + "+--------------+------------+--------------------------------------+-----------------+", + "| public | test_table | 01020304-0506-4708-890a-0b0c0d0e0f10 | PENDING |", + "+--------------+------------+--------------------------------------+-----------------+", + ]; + assert_batches_eq!(expected, &results); + + Ok(()) + } + #[tokio::test] async fn test_execute_insert_from_other_table() -> Result<()> { let context = Arc::new(in_memory_context().await); diff --git a/src/repository/default.rs b/src/repository/default.rs index cc400275..decb3a7d 100644 --- a/src/repository/default.rs +++ b/src/repository/default.rs @@ -50,7 +50,7 @@ /// Queries that are different between SQLite and PG pub struct RepositoryQueries { pub latest_table_versions: &'static str, - pub all_table_versions: &'static str, + pub cast_timestamp: &'static str, } #[macro_export] @@ -477,8 +477,22 @@ impl Repository for $repo { database_name: &str, table_names: Option>, ) -> Result, Error> { + let query = format!(r#"SELECT + database.name AS database_name, + collection.name AS collection_name, + "table".name AS table_name, + table_version.id AS table_version_id, + "table".legacy AS table_legacy, + {} AS creation_time + FROM table_version + INNER JOIN "table" ON "table".id = table_version.table_id + INNER JOIN collection ON collection.id = "table".collection_id + INNER JOIN database ON database.id = collection.database_id"#, + $repo::QUERIES.cast_timestamp.replace("timestamp_column", "table_version.creation_time") + ); + // We have to manually construct the query since SQLite doesn't have the proper Encode trait - let mut builder: QueryBuilder<_> = QueryBuilder::new($repo::QUERIES.all_table_versions); + let mut builder: QueryBuilder<_> = QueryBuilder::new(&query); builder.push(" WHERE database.name = "); builder.push_bind(database_name); @@ -651,7 +665,7 @@ impl Repository for $repo { // NB: we also persist db/col name on the off chance that we want to add table restore/undrop // at some point. let mut builder: QueryBuilder<_> = QueryBuilder::new( - r#"INSERT INTO dropped_table(database_name, collection_name, name, uuid) + r#"INSERT INTO dropped_table(database_name, collection_name, table_name, uuid) SELECT * FROM ( SELECT database.name, collection.name, "table".name, "table".uuid FROM "table" @@ -679,6 +693,29 @@ impl Repository for $repo { Ok(()) } + async fn get_dropped_tables( + &self, + database_name: &str, + ) -> Result> { + let query = format!(r#"SELECT + database_name, + collection_name, + table_name, + uuid, + deletion_status, + {} AS drop_time + FROM dropped_table WHERE database_name = $1"#, + $repo::QUERIES.cast_timestamp.replace("timestamp_column", "drop_time") + ); + + let dropped_tables = sqlx::query_as(&query) + .bind(database_name) + .fetch_all(&self.executor) + .await.map_err($repo::interpret_error)?; + + Ok(dropped_tables) + } + async fn delete_dropped_table(&self, uuid: Uuid) -> Result<(), Error> { sqlx::query("DELETE FROM dropped_table WHERE uuid = $1 RETURNING uuid") .bind(uuid) diff --git a/src/repository/interface.rs b/src/repository/interface.rs index 8d169fa1..6e51becb 100644 --- a/src/repository/interface.rs +++ b/src/repository/interface.rs @@ -48,6 +48,16 @@ pub struct TablePartitionsResult { pub row_count: Option, } +#[derive(sqlx::FromRow, Debug, PartialEq, Eq)] +pub struct DroppedTablesResult { + pub database_name: String, + pub collection_name: String, + pub table_name: String, + pub uuid: Uuid, + pub deletion_status: String, + pub drop_time: Timestamp, +} + #[derive(sqlx::FromRow, Debug, PartialEq, Eq)] pub struct AllTablePartitionColumnsResult { pub table_partition_id: i64, @@ -212,6 +222,11 @@ pub trait Repository: Send + Sync + Debug { maybe_database_id: Option, ) -> Result<(), Error>; + async fn get_dropped_tables( + &self, + database_name: &str, + ) -> Result>; + async fn delete_dropped_table(&self, uuid: Uuid) -> Result<(), Error>; } diff --git a/src/repository/postgres.rs b/src/repository/postgres.rs index 620ef6a1..3697e6f8 100644 --- a/src/repository/postgres.rs +++ b/src/repository/postgres.rs @@ -24,8 +24,8 @@ use crate::{ use super::{ default::RepositoryQueries, interface::{ - AllDatabaseColumnsResult, AllDatabaseFunctionsResult, Error, Repository, Result, - TablePartitionsResult, TableVersionsResult, + AllDatabaseColumnsResult, AllDatabaseFunctionsResult, DroppedTablesResult, Error, + Repository, Result, TablePartitionsResult, TableVersionsResult, }, }; @@ -44,17 +44,7 @@ impl PostgresRepository { FROM table_version ORDER BY table_id, creation_time DESC, id DESC )"#, - all_table_versions: r#"SELECT - database.name AS database_name, - collection.name AS collection_name, - "table".name AS table_name, - table_version.id AS table_version_id, - "table".legacy AS table_legacy, - CAST(EXTRACT(EPOCH FROM table_version.creation_time) AS INT8) AS creation_time - FROM table_version - INNER JOIN "table" ON "table".id = table_version.table_id - INNER JOIN collection ON collection.id = "table".collection_id - INNER JOIN database ON database.id = collection.database_id"#, + cast_timestamp: "CAST(EXTRACT(EPOCH FROM timestamp_column) AS INT8)", }; pub async fn try_new( diff --git a/src/repository/sqlite.rs b/src/repository/sqlite.rs index 0337690a..94735aa6 100644 --- a/src/repository/sqlite.rs +++ b/src/repository/sqlite.rs @@ -26,8 +26,8 @@ use crate::implement_repository; use super::{ default::RepositoryQueries, interface::{ - AllDatabaseColumnsResult, AllDatabaseFunctionsResult, Error, Repository, Result, - TablePartitionsResult, TableVersionsResult, + AllDatabaseColumnsResult, AllDatabaseFunctionsResult, DroppedTablesResult, Error, + Repository, Result, TablePartitionsResult, TableVersionsResult, }, }; @@ -53,17 +53,7 @@ impl SqliteRepository { FROM table_version GROUP BY table_id )"#, - all_table_versions: r#"SELECT - database.name AS database_name, - collection.name AS collection_name, - "table".name AS table_name, - table_version.id AS table_version_id, - "table".legacy AS table_legacy, - CAST(table_version.creation_time AS INTEGER(8)) AS creation_time - FROM table_version - INNER JOIN "table" ON "table".id = table_version.table_id - INNER JOIN collection ON collection.id = "table".collection_id - INNER JOIN database ON database.id = collection.database_id"#, + cast_timestamp: "CAST(timestamp_column AS INTEGER(8))", }; pub async fn try_new( diff --git a/src/system_tables.rs b/src/system_tables.rs index ea34d7ce..ce9906e3 100644 --- a/src/system_tables.rs +++ b/src/system_tables.rs @@ -2,7 +2,9 @@ //! and datafusion's information_schema. use crate::catalog::TableCatalog; -use crate::repository::interface::{TablePartitionsResult, TableVersionsResult}; +use crate::repository::interface::{ + DroppedTablesResult, TablePartitionsResult, TableVersionsResult, +}; use arrow::array::{ Int32Builder, Int64Builder, StringBuilder, StructBuilder, TimestampSecondBuilder, }; @@ -23,6 +25,7 @@ use std::sync::Arc; pub const SYSTEM_SCHEMA: &str = "system"; const TABLE_VERSIONS: &str = "table_versions"; const TABLE_PARTITIONS: &str = "table_partitions"; +const DROPPED_TABLES: &str = "dropped_tables"; pub struct SystemSchemaProvider { database: Arc, @@ -45,7 +48,11 @@ impl SchemaProvider for SystemSchemaProvider { } fn table_names(&self) -> Vec { - vec![TABLE_VERSIONS.to_string(), TABLE_PARTITIONS.to_string()] + vec![ + TABLE_VERSIONS.to_string(), + TABLE_PARTITIONS.to_string(), + DROPPED_TABLES.to_string(), + ] } async fn table(&self, name: &str) -> Option> { @@ -69,6 +76,15 @@ impl SchemaProvider for SystemSchemaProvider { table: Arc::new(table), })) } + DROPPED_TABLES => { + let table = DroppedTablesTable::new( + self.database.clone(), + self.table_catalog.clone(), + ); + Some(Arc::new(SystemTableProvider { + table: Arc::new(table), + })) + } _ => None, } } @@ -76,7 +92,7 @@ impl SchemaProvider for SystemSchemaProvider { fn table_exist(&self, name: &str) -> bool { matches!( name.to_ascii_lowercase().as_str(), - TABLE_VERSIONS | TABLE_PARTITIONS + TABLE_VERSIONS | TABLE_PARTITIONS | DROPPED_TABLES ) } } @@ -289,3 +305,85 @@ impl SeafowlSystemTable for TablePartitionsTable { .map_err(DataFusionError::from) } } + +// Table listing all dropped tables that are pending lazy deletion on subsequent `VACUUM`s +struct DroppedTablesTable { + database: Arc, + schema: SchemaRef, + table_catalog: Arc, +} + +impl DroppedTablesTable { + fn new(database: Arc, table_catalog: Arc) -> Self { + Self { + // This is dictated by the output of `get_dropped_tables`, except that we omit the + // database_name field, since we scope down to the database at hand. + database, + schema: Arc::new(Schema::new(vec![ + Field::new("table_schema", DataType::Utf8, false), + Field::new("table_name", DataType::Utf8, false), + Field::new("uuid", DataType::Utf8, false), + Field::new("deletion_status", DataType::Utf8, false), + Field::new( + "drop_time", + // TODO: should we be using a concrete timezone here? + DataType::Timestamp(TimeUnit::Second, None), + false, + ), + ])), + table_catalog, + } + } +} + +#[async_trait] +impl SeafowlSystemTable for DroppedTablesTable { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + async fn load_record_batch(&self) -> Result { + let dropped_tables = self + .table_catalog + .get_dropped_tables(&self.database) + .await? + .into_iter() + .collect::>(); + + let mut builder = StructBuilder::from_fields( + self.schema.fields().clone(), + dropped_tables.len(), + ); + + // Construct the table columns from the returned rows + for dropped_table in &dropped_tables { + builder + .field_builder::(0) + .unwrap() + .append_value(dropped_table.collection_name.clone()); + builder + .field_builder::(1) + .unwrap() + .append_value(&dropped_table.table_name.clone()); + builder + .field_builder::(2) + .unwrap() + .append_value(dropped_table.uuid.to_string().clone()); + builder + .field_builder::(3) + .unwrap() + .append_value(dropped_table.deletion_status.clone()); + builder + .field_builder::(4) + .unwrap() + .append_value(dropped_table.drop_time); + + builder.append(true); + } + + let struct_array = builder.finish(); + + RecordBatch::try_new(self.schema.clone(), struct_array.columns().to_vec()) + .map_err(DataFusionError::from) + } +} diff --git a/tests/statements/ddl.rs b/tests/statements/ddl.rs index 174b1fb9..3a946bef 100644 --- a/tests/statements/ddl.rs +++ b/tests/statements/ddl.rs @@ -307,6 +307,26 @@ async fn test_create_table_drop_schema() { ]; assert_batches_eq!(expected, &results); + // Check tables from the dropped schemas are pending for deletion + let plan = context + .plan_query( + "SELECT table_schema, table_name, deletion_status FROM system.dropped_tables", + ) + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let expected = vec![ + "+--------------+--------------+-----------------+", + "| table_schema | table_name | deletion_status |", + "+--------------+--------------+-----------------+", + "| public | test_table_1 | PENDING |", + "| new_schema | test_table_2 | PENDING |", + "+--------------+--------------+-----------------+", + ]; + + assert_batches_eq!(expected, &results); + // Recreate the public schema and add a table to it context .collect(context.plan_query("CREATE SCHEMA public").await.unwrap()) diff --git a/tests/statements/query.rs b/tests/statements/query.rs index 5213ee4b..b5e941f4 100644 --- a/tests/statements/query.rs +++ b/tests/statements/query.rs @@ -18,6 +18,7 @@ async fn test_information_schema() { "+---------------+--------------------+------------------+------------+", "| default | information_schema | columns | VIEW |", "| default | information_schema | df_settings | VIEW |", + "| default | system | dropped_tables | VIEW |", "| default | system | table_partitions | VIEW |", "| default | system | table_versions | VIEW |", "| default | information_schema | tables | VIEW |", @@ -45,6 +46,11 @@ async fn test_information_schema() { "+--------------+------------------+--------------------+-------------------------+-------------+", "| table_schema | table_name | column_name | data_type | is_nullable |", "+--------------+------------------+--------------------+-------------------------+-------------+", + "| system | dropped_tables | table_schema | Utf8 | NO |", + "| system | dropped_tables | table_name | Utf8 | NO |", + "| system | dropped_tables | uuid | Utf8 | NO |", + "| system | dropped_tables | deletion_status | Utf8 | NO |", + "| system | dropped_tables | drop_time | Timestamp(Second, None) | NO |", "| system | table_partitions | table_schema | Utf8 | NO |", "| system | table_partitions | table_name | Utf8 | NO |", "| system | table_partitions | table_version_id | Int64 | NO |", From 0cbfab98545b95797058edbfa48efa6c5689b907 Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Mon, 13 Mar 2023 12:24:28 +0100 Subject: [PATCH 18/21] Persist new Delta table versions in Seafowl catalog --- ...230221081920_delta_lake_integration.up.sql | 5 + ...230221081928_delta_lake_integration.up.sql | 5 + src/catalog.rs | 16 +- src/context.rs | 21 ++- src/repository/default.rs | 45 +++--- src/repository/interface.rs | 139 ++---------------- src/repository/postgres.rs | 1 + src/repository/sqlite.rs | 1 + src/system_tables.rs | 8 +- tests/statements/query.rs | 118 ++++++--------- tests/statements/query_legacy.rs | 56 +++---- 11 files changed, 151 insertions(+), 264 deletions(-) diff --git a/migrations/postgres/20230221081920_delta_lake_integration.up.sql b/migrations/postgres/20230221081920_delta_lake_integration.up.sql index a251c29b..35399220 100644 --- a/migrations/postgres/20230221081920_delta_lake_integration.up.sql +++ b/migrations/postgres/20230221081920_delta_lake_integration.up.sql @@ -3,6 +3,11 @@ ALTER TABLE "table" ADD COLUMN legacy BOOLEAN DEFAULT FALSE; ALTER TABLE "table" ADD COLUMN uuid UUID NOT NULL DEFAULT '00000000-0000-0000-0000-000000000000'; UPDATE "table" SET legacy = TRUE; +-- Add column for tracking Delta table versions; back-populate -1 for legacy tables +-- TODO: maybe version should be part of the primary key +ALTER TABLE table_version ADD COLUMN version BIGINT NOT NULL DEFAULT 0; +UPDATE table_version SET version = -1; + -- Table for facilitating soft-dropping of tables, via deferring the actual file deletion for later. CREATE TABLE dropped_table ( database_name VARCHAR NOT NULL, diff --git a/migrations/sqlite/20230221081928_delta_lake_integration.up.sql b/migrations/sqlite/20230221081928_delta_lake_integration.up.sql index 460c3095..301513ae 100644 --- a/migrations/sqlite/20230221081928_delta_lake_integration.up.sql +++ b/migrations/sqlite/20230221081928_delta_lake_integration.up.sql @@ -7,6 +7,11 @@ ALTER TABLE "table" ADD COLUMN legacy BOOLEAN DEFAULT FALSE; ALTER TABLE "table" ADD COLUMN uuid BLOB NOT NULL DEFAULT x'00000000000000000000000000000000'; UPDATE "table" SET legacy = TRUE; +-- Add column for tracking Delta table versions; back-populate -1 for legacy tables +-- TODO: maybe version should be part of the primary key +ALTER TABLE table_version ADD COLUMN version INTEGER NOT NULL DEFAULT 0; +UPDATE table_version SET version = -1; + -- Table for facilitating soft-dropping of tables, via deferring the actual file deletion for later. CREATE TABLE dropped_table ( database_name VARCHAR NOT NULL, diff --git a/src/catalog.rs b/src/catalog.rs index be57a4b1..bcb57d27 100644 --- a/src/catalog.rs +++ b/src/catalog.rs @@ -5,7 +5,7 @@ use async_trait::async_trait; use datafusion::catalog::schema::MemorySchemaProvider; use datafusion::datasource::TableProvider; use datafusion::error::DataFusionError; -use deltalake::DeltaTable; +use deltalake::{DeltaDataTypeVersion, DeltaTable}; use itertools::Itertools; #[cfg(test)] use mockall::automock; @@ -211,8 +211,8 @@ pub trait TableCatalog: Sync + Send { async fn create_new_table_version( &self, - from_version: TableVersionId, - inherit_partitions: bool, + uuid: Uuid, + version: DeltaDataTypeVersion, ) -> Result; async fn get_all_table_versions( @@ -637,15 +637,15 @@ impl TableCatalog for DefaultCatalog { async fn create_new_table_version( &self, - from_version: TableVersionId, - inherit_partitions: bool, + uuid: Uuid, + version: DeltaDataTypeVersion, ) -> Result { self.repository - .create_new_table_version(from_version, inherit_partitions) + .create_new_table_version(uuid, version) .await .map_err(|e| match e { - RepositoryError::FKConstraintViolation(_) => { - Error::TableVersionDoesNotExist { id: from_version } + RepositoryError::SqlxError(sqlx::error::Error::RowNotFound) => { + Error::TableUuidDoesNotExist { uuid } } _ => Self::to_sqlx_error(e), }) diff --git a/src/context.rs b/src/context.rs index fa7a79c4..d1238fe5 100644 --- a/src/context.rs +++ b/src/context.rs @@ -1087,6 +1087,13 @@ impl DefaultSeafowlContext { .with_object_store(table_object_store) .await?; + // TODO: if `DeltaTable::get_version_timestamp` was globally public we could also pass the + // exact version timestamp, instead of creating onw automatically in our own catalog (which + // could lead to minor timestamp differences). + self.table_catalog + .create_new_table_version(table_uuid, table.version()) + .await?; + debug!("Written table version {} for {table}", table.version()); Ok(table) } @@ -1813,7 +1820,11 @@ impl SeafowlContext for DefaultSeafowlContext { let mut tx = table.create_transaction(None); tx.add_actions(actions); - tx.commit(None, None).await?; + let version = tx.commit(None, None).await?; + let uuid = self.get_table_uuid(table_name).await?; + self.table_catalog + .create_new_table_version(uuid, version) + .await?; Ok(make_dummy_exec()) } @@ -1914,7 +1925,11 @@ impl SeafowlContext for DefaultSeafowlContext { let mut tx = table.create_transaction(None); tx.add_actions(actions); - tx.commit(None, None).await?; + let version = tx.commit(None, None).await?; + let uuid = self.get_table_uuid(table_name).await?; + self.table_catalog + .create_new_table_version(uuid, version) + .await?; Ok(make_dummy_exec()) } @@ -2964,7 +2979,7 @@ mod tests { |tables| { tables .expect_create_new_table_version() - .with(predicate::eq(0), predicate::eq(true)) + .with(predicate::eq(Uuid::parse_str("01020304-0506-4708-890a-0b0c0d0e0f10").unwrap()), predicate::eq(1)) .return_once(|_, _| Ok(1)); }, ) diff --git a/src/repository/default.rs b/src/repository/default.rs index decb3a7d..4c98231e 100644 --- a/src/repository/default.rs +++ b/src/repository/default.rs @@ -436,15 +436,26 @@ impl Repository for $repo { async fn create_new_table_version( &self, - from_version: TableVersionId, - inherit_partitions: bool, + uuid: Uuid, + version: DeltaDataTypeVersion, ) -> Result { - let new_version = sqlx::query( - "INSERT INTO table_version (table_id) - SELECT table_id FROM table_version WHERE id = $1 + // For now we only support linear history + let last_version_id: TableVersionId = sqlx::query(r#"SELECT max(table_version.id) AS id + FROM table_version + JOIN "table" ON table_version.table_id = "table".id + WHERE "table".uuid = $1"#) + .bind(uuid) + .fetch_one(&self.executor) + .await.map_err($repo::interpret_error)? + .try_get("id").map_err($repo::interpret_error)?; + + let new_version_id = sqlx::query( + "INSERT INTO table_version (table_id, version) + SELECT table_id, $1 FROM table_version WHERE id = $2 RETURNING (id)", ) - .bind(from_version) + .bind(version) + .bind(last_version_id) .fetch_one(&self.executor) .await.map_err($repo::interpret_error)? .try_get("id").map_err($repo::interpret_error)?; @@ -453,23 +464,12 @@ impl Repository for $repo { "INSERT INTO table_column (table_version_id, name, type) SELECT $2, name, type FROM table_column WHERE table_version_id = $1;", ) - .bind(from_version) - .bind(new_version) + .bind(last_version_id) + .bind(new_version_id) .execute(&self.executor) .await.map_err($repo::interpret_error)?; - if inherit_partitions { - sqlx::query( - "INSERT INTO table_partition (table_version_id, physical_partition_id) - SELECT $2, physical_partition_id FROM table_partition WHERE table_version_id = $1;", - ) - .bind(from_version) - .bind(new_version) - .execute(&self.executor) - .await.map_err($repo::interpret_error)?; - } - - Ok(new_version) + Ok(new_version_id) } async fn get_all_table_versions( @@ -482,6 +482,7 @@ impl Repository for $repo { collection.name AS collection_name, "table".name AS table_name, table_version.id AS table_version_id, + table_version.version AS version, "table".legacy AS table_legacy, {} AS creation_time FROM table_version @@ -662,8 +663,8 @@ impl Repository for $repo { // them to a special table that is used for lazy cleanup of files via `VACUUM`. // TODO: We could do this via a trigger, but then we'd lose the ability to actually // perform hard deletes at the DB-level. - // NB: we also persist db/col name on the off chance that we want to add table restore/undrop - // at some point. + // NB: We rally only need the uuid for cleanup, but we also persist db/col name on the off + // chance that we want to add table restore/undrop at some point. let mut builder: QueryBuilder<_> = QueryBuilder::new( r#"INSERT INTO dropped_table(database_name, collection_name, table_name, uuid) SELECT * FROM ( diff --git a/src/repository/interface.rs b/src/repository/interface.rs index 6e51becb..4c5e5041 100644 --- a/src/repository/interface.rs +++ b/src/repository/interface.rs @@ -1,6 +1,7 @@ use std::fmt::Debug; use async_trait::async_trait; +use deltalake::DeltaDataTypeVersion; use uuid::Uuid; use crate::wasm_udf::data_types::CreateFunctionDetails; @@ -32,6 +33,7 @@ pub struct TableVersionsResult { pub collection_name: String, pub table_name: String, pub table_version_id: TableVersionId, + pub version: DeltaDataTypeVersion, pub table_legacy: bool, pub creation_time: Timestamp, } @@ -175,8 +177,8 @@ pub trait Repository: Send + Sync + Debug { async fn create_new_table_version( &self, - from_version: TableVersionId, - inherit_partitions: bool, + uuid: Uuid, + version: DeltaDataTypeVersion, ) -> Result; async fn get_all_table_versions( @@ -238,50 +240,12 @@ pub mod tests { DataType as ArrowDataType, Field as ArrowField, Schema as ArrowSchema, }; - use crate::provider::PartitionColumn; use crate::wasm_udf::data_types::{ CreateFunctionDataType, CreateFunctionLanguage, CreateFunctionVolatility, }; use super::*; - const EXPECTED_FILE_NAME: &str = - "bdd6eef7340866d1ad99ed34ce0fa43c0d06bbed4dbcb027e9a51de48638b3ed.parquet"; - - fn get_test_partition() -> SeafowlPartition { - SeafowlPartition { - partition_id: Some(1), - object_storage_id: Arc::from(EXPECTED_FILE_NAME.to_string()), - row_count: 2, - columns: Arc::new(vec![ - PartitionColumn { - name: Arc::from("timestamp".to_string()), - r#type: Arc::from("{\"name\":\"utf8\"}".to_string()), - min_value: Arc::new(None), - max_value: Arc::new(None), - null_count: Some(1), - }, - PartitionColumn { - name: Arc::from("integer".to_string()), - r#type: Arc::from( - "{\"name\":\"int\",\"bitWidth\":64,\"isSigned\":true}" - .to_string(), - ), - min_value: Arc::new(Some([49, 50].to_vec())), - max_value: Arc::new(Some([52, 50].to_vec())), - null_count: Some(0), - }, - PartitionColumn { - name: Arc::from("varchar".to_string()), - r#type: Arc::from("{\"name\":\"utf8\"}".to_string()), - min_value: Arc::new(None), - max_value: Arc::new(None), - null_count: None, - }, - ]), - } - } - async fn make_database_with_single_table( repository: Arc, ) -> (DatabaseId, CollectionId, TableId, TableVersionId) { @@ -314,11 +278,14 @@ pub mod tests { test_get_collections_empty(repository.clone()).await; let (database_id, table_id, table_version_id) = test_create_database_collection_table(repository.clone()).await; - let new_version_id = - test_create_append_partition(repository.clone(), table_version_id).await; test_create_functions(repository.clone(), database_id).await; - test_rename_table(repository.clone(), database_id, table_id, new_version_id) - .await; + test_rename_table( + repository.clone(), + database_id, + table_id, + table_version_id + 1, + ) + .await; test_error_propagation(repository, table_id).await; } @@ -397,7 +364,7 @@ pub mod tests { // Duplicate the table let new_version_id = repository - .create_new_table_version(table_version_id, true) + .create_new_table_version(Uuid::default(), 1) .await .unwrap(); @@ -447,88 +414,6 @@ pub mod tests { (database_id, table_id, table_version_id) } - async fn test_create_append_partition( - repository: Arc, - table_version_id: TableVersionId, - ) -> TableVersionId { - let partition = get_test_partition(); - - // Create a partition - let partition_ids = repository.create_partitions(vec![partition]).await.unwrap(); - assert_eq!(partition_ids.len(), 1); - - let partition_id = partition_ids.first().unwrap(); - - // Test loading all table partitions when the partition is not yet attached - let all_partitions = repository - .get_all_table_partition_columns(table_version_id) - .await - .unwrap(); - assert_eq!(all_partitions, Vec::::new()); - - // Attach the partition to the table - repository - .append_partitions_to_table(partition_ids.clone(), table_version_id) - .await - .unwrap(); - - // Load again - let all_partitions = repository - .get_all_table_partition_columns(table_version_id) - .await - .unwrap(); - - let expected_partitions = vec![ - AllTablePartitionColumnsResult { - table_partition_id: *partition_id, - object_storage_id: EXPECTED_FILE_NAME.to_string(), - column_name: "timestamp".to_string(), - column_type: "{\"name\":\"utf8\"}".to_string(), - row_count: 2, - min_value: None, - max_value: None, - null_count: Some(1), - }, - AllTablePartitionColumnsResult { - table_partition_id: *partition_id, - object_storage_id: EXPECTED_FILE_NAME.to_string(), - column_name: "integer".to_string(), - column_type: "{\"name\":\"int\",\"bitWidth\":64,\"isSigned\":true}" - .to_string(), - row_count: 2, - min_value: Some([49, 50].to_vec()), - max_value: Some([52, 50].to_vec()), - null_count: Some(0), - }, - AllTablePartitionColumnsResult { - table_partition_id: *partition_id, - object_storage_id: EXPECTED_FILE_NAME.to_string(), - column_name: "varchar".to_string(), - column_type: "{\"name\":\"utf8\"}".to_string(), - row_count: 2, - min_value: None, - max_value: None, - null_count: None, - }, - ]; - assert_eq!(all_partitions, expected_partitions); - - // Duplicate the table, check it has the same partitions - let new_version_id = repository - .create_new_table_version(table_version_id, true) - .await - .unwrap(); - - let all_partitions = repository - .get_all_table_partition_columns(new_version_id) - .await - .unwrap(); - - assert_eq!(all_partitions, expected_partitions); - - new_version_id - } - async fn test_create_functions( repository: Arc, database_id: DatabaseId, diff --git a/src/repository/postgres.rs b/src/repository/postgres.rs index 3697e6f8..a0dce917 100644 --- a/src/repository/postgres.rs +++ b/src/repository/postgres.rs @@ -1,6 +1,7 @@ use std::{fmt::Debug, iter::zip, time::Duration}; use async_trait::async_trait; +use deltalake::DeltaDataTypeVersion; use futures::TryStreamExt; use sqlx::{ migrate::{MigrateDatabase, Migrator}, diff --git a/src/repository/sqlite.rs b/src/repository/sqlite.rs index 94735aa6..d0779e1c 100644 --- a/src/repository/sqlite.rs +++ b/src/repository/sqlite.rs @@ -1,6 +1,7 @@ use std::{fmt::Debug, iter::zip, str::FromStr}; use async_trait::async_trait; +use deltalake::DeltaDataTypeVersion; use futures::TryStreamExt; use sqlx::sqlite::SqliteJournalMode; use sqlx::{ diff --git a/src/system_tables.rs b/src/system_tables.rs index ce9906e3..3aa65037 100644 --- a/src/system_tables.rs +++ b/src/system_tables.rs @@ -162,6 +162,7 @@ impl TableVersionsTable { Field::new("table_schema", DataType::Utf8, false), Field::new("table_name", DataType::Utf8, false), Field::new("table_version_id", DataType::Int64, false), + Field::new("version", DataType::Int64, false), Field::new( "creation_time", // TODO: should we be using a concrete timezone here? @@ -186,7 +187,6 @@ impl SeafowlSystemTable for TableVersionsTable { .get_all_table_versions(&self.database, None) .await? .into_iter() - .filter(|tv| tv.table_legacy) .collect::>(); let mut builder = StructBuilder::from_fields( @@ -209,7 +209,11 @@ impl SeafowlSystemTable for TableVersionsTable { .unwrap() .append_value(table_version.table_version_id); builder - .field_builder::(3) + .field_builder::(3) + .unwrap() + .append_value(table_version.version); + builder + .field_builder::(4) .unwrap() .append_value(table_version.creation_time); diff --git a/tests/statements/query.rs b/tests/statements/query.rs index b5e941f4..208cc7ce 100644 --- a/tests/statements/query.rs +++ b/tests/statements/query.rs @@ -60,6 +60,7 @@ async fn test_information_schema() { "| system | table_versions | table_schema | Utf8 | NO |", "| system | table_versions | table_name | Utf8 | NO |", "| system | table_versions | table_version_id | Int64 | NO |", + "| system | table_versions | version | Int64 | NO |", "| system | table_versions | creation_time | Timestamp(Second, None) | NO |", "+--------------+------------------+--------------------+-------------------------+-------------+", ]; @@ -148,84 +149,53 @@ async fn test_table_time_travel() { // Verify that the new table versions are shown in the corresponding system table // - // TODO: Make `system.table_versions` table work for delta tables - // let plan = context - // .plan_query("SELECT table_schema, table_name, table_version_id FROM system.table_versions") - // .await - // .unwrap(); - // let results = context.collect(plan).await.unwrap(); - // - // let expected = vec![ - // "+--------------+------------+------------------+", - // "| table_schema | table_name | table_version_id |", - // "+--------------+------------+------------------+", - // "| public | test_table | 1 |", - // "| public | test_table | 2 |", - // "| public | test_table | 3 |", - // "| public | test_table | 4 |", - // "| public | test_table | 5 |", - // "+--------------+------------+------------------+", - // ]; - // assert_batches_eq!(expected, &results); - - // // - // // Test that filtering the system table works, given that we provide all rows to DF and expect - // // it to do it. - // // - // let plan = context - // .plan_query( - // format!( - // " - // SELECT table_version_id FROM system.table_versions \ - // WHERE table_version_id < 5 AND creation_time > to_timestamp('{}') - // ", - // timestamp_to_rfc3339(version_timestamps[&2]) - // ) - // .as_str(), - // ) - // .await - // .unwrap(); - // let results = context.collect(plan).await.unwrap(); - // - // let expected = vec![ - // "+------------------+", - // "| table_version_id |", - // "+------------------+", - // "| 3 |", - // "| 4 |", - // "+------------------+", - // ]; - // assert_batches_eq!(expected, &results); + let plan = context + .plan_query("SELECT table_schema, table_name, version FROM system.table_versions") + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); + + let expected = vec![ + "+--------------+------------+---------+", + "| table_schema | table_name | version |", + "+--------------+------------+---------+", + "| public | test_table | 0 |", + "| public | test_table | 1 |", + "| public | test_table | 2 |", + "| public | test_table | 3 |", + "| public | test_table | 4 |", + "+--------------+------------+---------+", + ]; + assert_batches_eq!(expected, &results); // - // Verify that the new table partitions for all versions are shown in the corresponding system table + // Test that filtering the system table works, given that we provide all rows to DF and expect + // it to do it. // + let plan = context + .plan_query( + format!( + " + SELECT version FROM system.table_versions \ + WHERE version < 4 AND creation_time > to_timestamp('{}') + ", + timestamp_to_rfc3339(version_timestamps[&1]) + ) + .as_str(), + ) + .await + .unwrap(); + let results = context.collect(plan).await.unwrap(); - // TODO: Make `system.table_partitions` table work for delta tables - // let plan = context - // .plan_query("SELECT table_schema, table_name, table_version_id, table_partition_id, row_count FROM system.table_partitions") - // .await - // .unwrap(); - // let results = context.collect(plan).await.unwrap(); - // - // let expected = vec![ - // "+--------------+------------+------------------+--------------------+-----------+", - // "| table_schema | table_name | table_version_id | table_partition_id | row_count |", - // "+--------------+------------+------------------+--------------------+-----------+", - // "| public | test_table | 1 | | |", - // "| public | test_table | 2 | 1 | 3 |", - // "| public | test_table | 3 | 1 | 3 |", - // "| public | test_table | 3 | 2 | 3 |", - // "| public | test_table | 4 | 1 | 3 |", - // "| public | test_table | 4 | 2 | 3 |", - // "| public | test_table | 4 | 3 | 3 |", - // "| public | test_table | 5 | 1 | 3 |", - // "| public | test_table | 5 | 2 | 3 |", - // "| public | test_table | 5 | 3 | 3 |", - // "| public | test_table | 5 | 4 | 3 |", - // "+--------------+------------+------------------+--------------------+-----------+", - // ]; - // assert_batches_eq!(expected, &results); + let expected = vec![ + "+---------+", + "| version |", + "+---------+", + "| 2 |", + "| 3 |", + "+---------+", + ]; + assert_batches_eq!(expected, &results); // // Now use the recorded timestamps to query specific earlier table versions and compare them to diff --git a/tests/statements/query_legacy.rs b/tests/statements/query_legacy.rs index cc0c631f..aaaf58aa 100644 --- a/tests/statements/query_legacy.rs +++ b/tests/statements/query_legacy.rs @@ -107,21 +107,21 @@ async fn test_legacy_tables() { // let plan = context - .plan_query("SELECT * FROM system.table_versions") + .plan_query("SELECT * FROM system.table_versions WHERE version = -1") .await .unwrap(); let results = context.collect(plan).await.unwrap(); let expected = vec![ - "+--------------+------------+------------------+---------------------+", - "| table_schema | table_name | table_version_id | creation_time |", - "+--------------+------------+------------------+---------------------+", - "| public | test_table | 1 | 2023-03-07T08:44:49 |", - "| public | test_table | 2 | 2023-03-07T08:44:49 |", - "| public | test_table | 3 | 2023-03-07T08:44:51 |", - "| public | test_table | 4 | 2023-03-07T08:44:53 |", - "| public | test_table | 5 | 2023-03-07T08:44:55 |", - "+--------------+------------+------------------+---------------------+", + "+--------------+------------+------------------+---------+---------------------+", + "| table_schema | table_name | table_version_id | version | creation_time |", + "+--------------+------------+------------------+---------+---------------------+", + "| public | test_table | 1 | -1 | 2023-03-07T08:44:49 |", + "| public | test_table | 2 | -1 | 2023-03-07T08:44:49 |", + "| public | test_table | 3 | -1 | 2023-03-07T08:44:51 |", + "| public | test_table | 4 | -1 | 2023-03-07T08:44:53 |", + "| public | test_table | 5 | -1 | 2023-03-07T08:44:55 |", + "+--------------+------------+------------------+---------+---------------------+", ]; assert_batches_eq!(expected, &results); @@ -428,15 +428,15 @@ async fn test_vacuum_legacy_tables() { let results = context.collect(plan).await.unwrap(); let expected = vec![ - "+--------------+------------+------------------+---------------------+", - "| table_schema | table_name | table_version_id | creation_time |", - "+--------------+------------+------------------+---------------------+", - "| public | test_table | 1 | 2023-03-07T08:44:49 |", - "| public | test_table | 2 | 2023-03-07T08:44:49 |", - "| public | test_table | 3 | 2023-03-07T08:44:51 |", - "| public | test_table | 4 | 2023-03-07T08:44:53 |", - "| public | test_table | 5 | 2023-03-07T08:44:55 |", - "+--------------+------------+------------------+---------------------+", + "+--------------+------------+------------------+---------+---------------------+", + "| table_schema | table_name | table_version_id | version | creation_time |", + "+--------------+------------+------------------+---------+---------------------+", + "| public | test_table | 1 | -1 | 2023-03-07T08:44:49 |", + "| public | test_table | 2 | -1 | 2023-03-07T08:44:49 |", + "| public | test_table | 3 | -1 | 2023-03-07T08:44:51 |", + "| public | test_table | 4 | -1 | 2023-03-07T08:44:53 |", + "| public | test_table | 5 | -1 | 2023-03-07T08:44:55 |", + "+--------------+------------+------------------+---------+---------------------+", ]; assert_batches_eq!(expected, &results); @@ -453,11 +453,11 @@ async fn test_vacuum_legacy_tables() { let results = context.collect(plan).await.unwrap(); let expected = vec![ - "+--------------+------------+------------------+---------------------+", - "| table_schema | table_name | table_version_id | creation_time |", - "+--------------+------------+------------------+---------------------+", - "| public | test_table | 5 | 2023-03-07T08:44:55 |", - "+--------------+------------+------------------+---------------------+", + "+--------------+------------+------------------+---------+---------------------+", + "| table_schema | table_name | table_version_id | version | creation_time |", + "+--------------+------------+------------------+---------+---------------------+", + "| public | test_table | 5 | -1 | 2023-03-07T08:44:55 |", + "+--------------+------------+------------------+---------+---------------------+", ]; assert_batches_eq!(expected, &results); @@ -505,10 +505,10 @@ async fn test_vacuum_legacy_tables() { let results = context.collect(plan).await.unwrap(); let expected = vec![ - "+--------------+------------+------------------+---------------+", - "| table_schema | table_name | table_version_id | creation_time |", - "+--------------+------------+------------------+---------------+", - "+--------------+------------+------------------+---------------+", + "+--------------+------------+------------------+---------+---------------+", + "| table_schema | table_name | table_version_id | version | creation_time |", + "+--------------+------------+------------------+---------+---------------+", + "+--------------+------------+------------------+---------+---------------+", ]; assert_batches_eq!(expected, &results); From 0249337119ecc897681e4d78b6d037b1bb66f8ab Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Tue, 14 Mar 2023 06:45:17 +0100 Subject: [PATCH 19/21] Bump to Datafusion 19 --- Cargo.lock | 261 +++++++++++++++------------- Cargo.toml | 39 +++-- datafusion_remote_tables/Cargo.toml | 12 +- src/context.rs | 19 +- src/wasm_udf/wasm.rs | 28 +-- tests/statements/ddl.rs | 6 +- tests/statements/dml.rs | 106 +++++------ tests/statements/function.rs | 30 ++-- tests/statements/query.rs | 43 ++--- tests/statements/query_legacy.rs | 40 ++--- 10 files changed, 293 insertions(+), 291 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 48996c07..e5079c1f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -109,9 +109,9 @@ checksum = "8da52d66c7071e2e3fa2a1e5c6d088fec47b593032b254f5e980de8ea54454d6" [[package]] name = "arrow" -version = "32.0.0" +version = "33.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87d948f553cf556656eb89265700258e1032d26fec9b7920cd20319336e06afd" +checksum = "f3724c874f1517cf898cd1c3ad18ab5071edf893c48e73139ab1e16cf0f2affe" dependencies = [ "ahash 0.8.1", "arrow-arith", @@ -125,17 +125,16 @@ dependencies = [ "arrow-ord", "arrow-row", "arrow-schema", - "arrow-select 32.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "arrow-select 33.0.0 (registry+https://github.com/rust-lang/crates.io-index)", "arrow-string", - "bitflags", "comfy-table", ] [[package]] name = "arrow-arith" -version = "32.0.0" +version = "33.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf30d4ebc3df9dfd8bd26883aa30687d4ddcfd7b2443e62bd7c8fedf153b8e45" +checksum = "e958823b8383ca14d0a2e973de478dd7674cd9f72837f8c41c132a0fda6a4e5e" dependencies = [ "arrow-array", "arrow-buffer", @@ -148,8 +147,8 @@ dependencies = [ [[package]] name = "arrow-array" -version = "32.0.0" -source = "git+https://github.com/splitgraph/arrow-rs?branch=backport-cast-utf8-to-timestamp-microsecond#d01096c6c9ac845f863d43f326709d3aaa99bdaf" +version = "33.0.0" +source = "git+https://github.com/splitgraph/arrow-rs?rev=57f79c03a8dee9d8bf8601bf555aa271746913fe#57f79c03a8dee9d8bf8601bf555aa271746913fe" dependencies = [ "ahash 0.8.1", "arrow-buffer", @@ -163,8 +162,8 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "32.0.0" -source = "git+https://github.com/splitgraph/arrow-rs?branch=backport-cast-utf8-to-timestamp-microsecond#d01096c6c9ac845f863d43f326709d3aaa99bdaf" +version = "33.0.0" +source = "git+https://github.com/splitgraph/arrow-rs?rev=57f79c03a8dee9d8bf8601bf555aa271746913fe#57f79c03a8dee9d8bf8601bf555aa271746913fe" dependencies = [ "half", "num", @@ -172,14 +171,14 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "32.0.0" -source = "git+https://github.com/splitgraph/arrow-rs?branch=backport-cast-utf8-to-timestamp-microsecond#d01096c6c9ac845f863d43f326709d3aaa99bdaf" +version = "33.0.0" +source = "git+https://github.com/splitgraph/arrow-rs?rev=57f79c03a8dee9d8bf8601bf555aa271746913fe#57f79c03a8dee9d8bf8601bf555aa271746913fe" dependencies = [ "arrow-array", "arrow-buffer", "arrow-data", "arrow-schema", - "arrow-select 32.0.0 (git+https://github.com/splitgraph/arrow-rs?branch=backport-cast-utf8-to-timestamp-microsecond)", + "arrow-select 33.0.0 (git+https://github.com/splitgraph/arrow-rs?rev=57f79c03a8dee9d8bf8601bf555aa271746913fe)", "chrono", "lexical-core", "num", @@ -187,9 +186,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "32.0.0" +version = "33.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b1d4fc91078dbe843c2c50d90f8119c96e8dfac2f78d30f7a8cb9397399c61d" +checksum = "0a6aa7c2531d89d01fed8c469a9b1bf97132a0bdf70b4724fe4bbb4537a50880" dependencies = [ "arrow-array", "arrow-buffer", @@ -206,8 +205,8 @@ dependencies = [ [[package]] name = "arrow-data" -version = "32.0.0" -source = "git+https://github.com/splitgraph/arrow-rs?branch=backport-cast-utf8-to-timestamp-microsecond#d01096c6c9ac845f863d43f326709d3aaa99bdaf" +version = "33.0.0" +source = "git+https://github.com/splitgraph/arrow-rs?rev=57f79c03a8dee9d8bf8601bf555aa271746913fe#57f79c03a8dee9d8bf8601bf555aa271746913fe" dependencies = [ "arrow-buffer", "arrow-schema", @@ -217,9 +216,9 @@ dependencies = [ [[package]] name = "arrow-integration-test" -version = "32.0.0" +version = "33.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eded54c92e8c82c9581cc08adef4f0486dac6e59acd0d01cc54216b92add89e2" +checksum = "8d2b237599dc4a5e2fa4cf561c64e56e9e68b874de714a6e4c4750144d7d2cad" dependencies = [ "arrow", "arrow-buffer", @@ -231,9 +230,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "32.0.0" +version = "33.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a3ca7eb8d23c83fe40805cbafec70a6a31df72de47355545ff34c850f715403" +checksum = "a4042fe6585155d1ec28a8e4937ec901a3ca7a19a22b9f6cd3f551b935cd84f5" dependencies = [ "arrow-array", "arrow-buffer", @@ -245,9 +244,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "32.0.0" +version = "33.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf65aff76d2e340d827d5cab14759e7dd90891a288347e2202e4ee28453d9bed" +checksum = "7c907c4ab4f26970a3719dc06e78e8054a01d0c96da3664d23b941e201b33d2b" dependencies = [ "arrow-array", "arrow-buffer", @@ -264,23 +263,23 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "32.0.0" +version = "33.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "074a5a55c37ae4750af4811c8861c0378d8ab2ff6c262622ad24efae6e0b73b3" +checksum = "e131b447242a32129efc7932f58ed8931b42f35d8701c1a08f9f524da13b1d3c" dependencies = [ "arrow-array", "arrow-buffer", "arrow-data", "arrow-schema", - "arrow-select 32.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "arrow-select 33.0.0 (registry+https://github.com/rust-lang/crates.io-index)", "num", ] [[package]] name = "arrow-row" -version = "32.0.0" +version = "33.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e064ac4e64960ebfbe35f218f5e7d9dc9803b59c2e56f611da28ce6d008f839e" +checksum = "b591ef70d76f4ac28dd7666093295fece0e5f9298f49af51ea49c001e1635bb6" dependencies = [ "ahash 0.8.1", "arrow-array", @@ -293,14 +292,17 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "32.0.0" -source = "git+https://github.com/splitgraph/arrow-rs?branch=backport-cast-utf8-to-timestamp-microsecond#d01096c6c9ac845f863d43f326709d3aaa99bdaf" +version = "33.0.0" +source = "git+https://github.com/splitgraph/arrow-rs?rev=57f79c03a8dee9d8bf8601bf555aa271746913fe#57f79c03a8dee9d8bf8601bf555aa271746913fe" +dependencies = [ + "bitflags", +] [[package]] name = "arrow-select" -version = "32.0.0" +version = "33.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "646b4f15b5a77c970059e748aeb1539705c68cd397ecf0f0264c4ef3737d35f3" +checksum = "79d3c389d1cea86793934f31594f914c8547d82e91e3411d4833ad0aac3266a7" dependencies = [ "arrow-array", "arrow-buffer", @@ -311,8 +313,8 @@ dependencies = [ [[package]] name = "arrow-select" -version = "32.0.0" -source = "git+https://github.com/splitgraph/arrow-rs?branch=backport-cast-utf8-to-timestamp-microsecond#d01096c6c9ac845f863d43f326709d3aaa99bdaf" +version = "33.0.0" +source = "git+https://github.com/splitgraph/arrow-rs?rev=57f79c03a8dee9d8bf8601bf555aa271746913fe#57f79c03a8dee9d8bf8601bf555aa271746913fe" dependencies = [ "arrow-array", "arrow-buffer", @@ -323,15 +325,15 @@ dependencies = [ [[package]] name = "arrow-string" -version = "32.0.0" +version = "33.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8b8bf150caaeca03f39f1a91069701387d93f7cfd256d27f423ac8496d99a51" +checksum = "30ee67790496dd310ddbf5096870324431e89aa76453e010020ac29b1184d356" dependencies = [ "arrow-array", "arrow-buffer", "arrow-data", "arrow-schema", - "arrow-select 32.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "arrow-select 33.0.0 (registry+https://github.com/rust-lang/crates.io-index)", "regex", "regex-syntax", ] @@ -451,10 +453,11 @@ checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" [[package]] name = "aws-config" -version = "0.52.0" +version = "0.54.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7688e1dfbb9f7804fab0a830820d7e827b8d973906763cf1a855ce4719292f5" +checksum = "3c3d1e2a1f1ab3ac6c4b884e37413eaa03eb9d901e4fc68ee8f5c1d49721680e" dependencies = [ + "aws-credential-types", "aws-http", "aws-sdk-sso", "aws-sdk-sts", @@ -477,11 +480,24 @@ dependencies = [ "zeroize", ] +[[package]] +name = "aws-credential-types" +version = "0.54.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bb0696a0523a39a19087747e4dafda0362dc867531e3d72a3f195564c84e5e08" +dependencies = [ + "aws-smithy-async", + "aws-smithy-types", + "tokio", + "tracing", + "zeroize", +] + [[package]] name = "aws-endpoint" -version = "0.52.0" +version = "0.54.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "253d7cd480bfa59a5323390e9e91885a8f06a275e0517d81eeb1070b6aa7d271" +checksum = "80a4f935ab6a1919fbfd6102a80c4fccd9ff5f47f94ba154074afe1051903261" dependencies = [ "aws-smithy-http", "aws-smithy-types", @@ -493,10 +509,11 @@ dependencies = [ [[package]] name = "aws-http" -version = "0.52.0" +version = "0.54.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4cd1b83859383e46ea8fda633378f9f3f02e6e3a446fd89f0240b5c3662716c9" +checksum = "82976ca4e426ee9ca3ffcf919d9b2c8d14d0cd80d43cc02173737a8f07f28d4d" dependencies = [ + "aws-credential-types", "aws-smithy-http", "aws-smithy-types", "aws-types", @@ -511,10 +528,11 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "0.22.0" +version = "0.24.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf03342c2b3f52b180f484e60586500765474f2bfc7dcd4ffe893a7a1929db1d" +checksum = "ca0119bacf0c42f587506769390983223ba834e605f049babe514b2bd646dbb2" dependencies = [ + "aws-credential-types", "aws-endpoint", "aws-http", "aws-sig-auth", @@ -527,16 +545,18 @@ dependencies = [ "aws-types", "bytes", "http", + "regex", "tokio-stream", "tower", ] [[package]] name = "aws-sdk-sts" -version = "0.22.0" +version = "0.24.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aa1de4e07ea87a30a317c7b563b3a40fd18a843ad794216dda81672b6e174bce" +checksum = "270b6a33969ebfcb193512fbd5e8ee5306888ad6c6d5d775cdbfb2d50d94de26" dependencies = [ + "aws-credential-types", "aws-endpoint", "aws-http", "aws-sig-auth", @@ -544,22 +564,25 @@ dependencies = [ "aws-smithy-client", "aws-smithy-http", "aws-smithy-http-tower", + "aws-smithy-json", "aws-smithy-query", "aws-smithy-types", "aws-smithy-xml", "aws-types", "bytes", "http", + "regex", "tower", "tracing", ] [[package]] name = "aws-sig-auth" -version = "0.52.0" +version = "0.54.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6126c4ff918e35fb9ae1bf2de71157fad36f0cc6a2b1d0f7197ee711713700fc" +checksum = "660a02a98ab1af83bd8d714afbab2d502ba9b18c49e7e4cddd6bf8837ff778cb" dependencies = [ + "aws-credential-types", "aws-sigv4", "aws-smithy-http", "aws-types", @@ -569,9 +592,9 @@ dependencies = [ [[package]] name = "aws-sigv4" -version = "0.52.0" +version = "0.54.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84c7f88d7395f5411c6eef5889b6cd577ce6b677af461356cbfc20176c26c160" +checksum = "cdaf11005b7444e6cd66f600d09861a3aeb6eb89a0f003c7c9820dbab2d15297" dependencies = [ "aws-smithy-http", "form_urlencoded", @@ -588,9 +611,9 @@ dependencies = [ [[package]] name = "aws-smithy-async" -version = "0.52.0" +version = "0.54.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e6a895d68852dd1564328e63ef1583e5eb307dd2a5ebf35d862a5c402957d5e" +checksum = "63c712a28a4f2f2139759235c08bf98aca99d4fdf1b13c78c5f95613df0a5db9" dependencies = [ "futures-util", "pin-project-lite", @@ -600,9 +623,9 @@ dependencies = [ [[package]] name = "aws-smithy-client" -version = "0.52.0" +version = "0.54.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f505bf793eb3e6d7c166ef1275c27b4b2cd5361173fe950ac8e2cfc08c29a7ef" +checksum = "104ca17f56cde00a10207169697dfe9c6810db339d52fb352707e64875b30a44" dependencies = [ "aws-smithy-async", "aws-smithy-http", @@ -623,9 +646,9 @@ dependencies = [ [[package]] name = "aws-smithy-http" -version = "0.52.0" +version = "0.54.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37e4b4304b7ea4af1af3e08535100eb7b6459d5a6264b92078bf85176d04ab85" +checksum = "873f316f1833add0d3aa54ed1b0cd252ddd88c792a0cf839886400099971e844" dependencies = [ "aws-smithy-types", "bytes", @@ -643,9 +666,9 @@ dependencies = [ [[package]] name = "aws-smithy-http-tower" -version = "0.52.0" +version = "0.54.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e86072ecc4dc4faf3e2071144285cfd539263fe7102b701d54fb991eafb04af8" +checksum = "4f38231d3f5dac9ac7976f44e12803add1385119ffca9e5f050d8e980733d164" dependencies = [ "aws-smithy-http", "aws-smithy-types", @@ -659,18 +682,18 @@ dependencies = [ [[package]] name = "aws-smithy-json" -version = "0.52.0" +version = "0.54.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e3ddd9275b167bc59e9446469eca56177ec0b51225632f90aaa2cd5f41c940e" +checksum = "4bd83ff2b79e9f729746fcc8ad798676b68fe6ea72986571569a5306a277a182" dependencies = [ "aws-smithy-types", ] [[package]] name = "aws-smithy-query" -version = "0.52.0" +version = "0.54.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13b19d2e0b3ce20e460bad0d0d974238673100edebba6978c2c1aadd925602f7" +checksum = "a2f0445dafe9d2cd50b44339ae3c3ed46549aad8ac696c52ad660b3e7ae8682b" dependencies = [ "aws-smithy-types", "urlencoding", @@ -678,9 +701,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "0.52.0" +version = "0.54.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "987b1e37febb9bd409ca0846e82d35299e572ad8279bc404778caeb5fc05ad56" +checksum = "8161232eda10290f5136610a1eb9de56aceaccd70c963a26a260af20ac24794f" dependencies = [ "base64-simd", "itoa 1.0.4", @@ -691,19 +714,20 @@ dependencies = [ [[package]] name = "aws-smithy-xml" -version = "0.52.0" +version = "0.54.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37ce3791e14eec75ffac851a5a559f1ce6b31843297f42cc8bfba82714a6a5d8" +checksum = "343ffe9a9bb3f542675f4df0e0d5933513d6ad038ca3907ad1767ba690a99684" dependencies = [ "xmlparser", ] [[package]] name = "aws-types" -version = "0.52.0" +version = "0.54.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c05adca3e2bcf686dd2c47836f216ab52ed7845c177d180c84b08522c1166a3" +checksum = "f8f15b34253b68cde08e39b0627cc6101bcca64351229484b4743392c035d057" dependencies = [ + "aws-credential-types", "aws-smithy-async", "aws-smithy-client", "aws-smithy-http", @@ -711,7 +735,6 @@ dependencies = [ "http", "rustc_version", "tracing", - "zeroize", ] [[package]] @@ -720,12 +743,6 @@ version = "0.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9e1b586273c5702936fe7b7d6896644d8be71e6314cfe09d3167c95f712589e8" -[[package]] -name = "base64" -version = "0.20.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ea22880d78093b0cbe17c89f64a7d457941e65759157ec6cb31a31d652b05e5" - [[package]] name = "base64" version = "0.21.0" @@ -734,11 +751,12 @@ checksum = "a4a4ddaa51a5bc52a6948f74c06d20aaaddb71924eab79b8c97a8c556e942d6a" [[package]] name = "base64-simd" -version = "0.7.0" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "781dd20c3aff0bd194fe7d2a977dd92f21c173891f3a03b677359e5fa457e5d5" +checksum = "339abbe78e73178762e23bea9dfd08e697eb3f3301cd4be981c0f78ba5859195" dependencies = [ - "simd-abstraction", + "outref", + "vsimd", ] [[package]] @@ -1262,7 +1280,7 @@ dependencies = [ [[package]] name = "connectorx" version = "0.3.2-alpha.2" -source = "git+https://github.com/splitgraph/connector-x?branch=datafusion-18-upgrade#c500a2907569f509b07afa30ba057fa8d5d3c97b" +source = "git+https://github.com/splitgraph/connector-x?rev=df8f50b3f53606717407c6677c7c0c2cbcc7f6ce#df8f50b3f53606717407c6677c7c0c2cbcc7f6ce" dependencies = [ "anyhow", "arrow", @@ -1328,7 +1346,7 @@ checksum = "245097e9a4535ee1e3e3931fcfcd55a796a44c643e8596ff6566d68f09b87bbc" [[package]] name = "convergence" version = "0.9.1" -source = "git+https://github.com/splitgraph/convergence?branch=datafusion-18-upgrade#fc3a217dab60612a34733c4a727fa30049421c32" +source = "git+https://github.com/splitgraph/convergence?branch=datafusion-19-upgrade#b0d155b2b6441bc5cb96fa94daaaa7e4107808c1" dependencies = [ "async-trait", "bytes", @@ -1343,7 +1361,7 @@ dependencies = [ [[package]] name = "convergence-arrow" version = "0.9.1" -source = "git+https://github.com/splitgraph/convergence?branch=datafusion-18-upgrade#fc3a217dab60612a34733c4a727fa30049421c32" +source = "git+https://github.com/splitgraph/convergence?branch=datafusion-19-upgrade#b0d155b2b6441bc5cb96fa94daaaa7e4107808c1" dependencies = [ "async-trait", "chrono", @@ -1683,9 +1701,9 @@ dependencies = [ [[package]] name = "datafusion" -version = "18.0.0" +version = "19.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd805bdf93d3137b37fd9966042df0c84ddfca0df5a8d32eaacb16cf6ab0d93d" +checksum = "12d462c103bd1cfd24f8e8a199986d89582af6280528e085c393c4be2ff25da7" dependencies = [ "ahash 0.8.1", "arrow", @@ -1730,9 +1748,9 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "18.0.0" +version = "19.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08c58d6714427f52f9815d19debab7adab5bac5b4d2a99d51c250e606acb6cf5" +checksum = "b5babdbcf102862b1f1828c1ab41094e39ba881d5ece4cee2d481d528148f592" dependencies = [ "arrow", "chrono", @@ -1744,9 +1762,9 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "18.0.0" +version = "19.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a32ee054230dd9a57d0bed587406869c4a7814d90154616aff2cb9991c1756f" +checksum = "90f0c34e87fa541a59d378dc7ee7c9c3dd1fcfa793eab09561b8b4cb35e1827a" dependencies = [ "ahash 0.8.1", "arrow", @@ -1757,9 +1775,9 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "18.0.0" +version = "19.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6de4d144924de29a835feeff8313a81fdc2c7190111301508e09ea59a80edbbc" +checksum = "7d0c6d912b7b7e4637d85947222455cd948ea193ca454ebf649e7265fd10b048" dependencies = [ "arrow", "async-trait", @@ -1774,9 +1792,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "18.0.0" +version = "19.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "943e42356f0f6f5ac37ceacd412de9c4d7d8eba1e81b6f724f88699540c7f070" +checksum = "8000e8f8efafb810ff2943323bb48bd722ac5bb919fe302a66b832ed9c25245f" dependencies = [ "ahash 0.8.1", "arrow", @@ -1805,9 +1823,9 @@ dependencies = [ [[package]] name = "datafusion-proto" -version = "18.0.0" +version = "19.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "649aec221737d8fb88956a2e6181297456f83b8560d32d775ad1cd22d67fd598" +checksum = "b62ce9a56522f03ad8b98e48f6ae924d1af216e60f61da201080bb842daab80d" dependencies = [ "arrow", "chrono", @@ -1840,9 +1858,9 @@ dependencies = [ [[package]] name = "datafusion-row" -version = "18.0.0" +version = "19.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a506f5924f8af54e0806a995da0897f8c2b548d492793e045a3896d88d6714a" +checksum = "4e900f05d7e5666e8ab714a96a28cb6f143e62aa1d501ba1199024f8635c726c" dependencies = [ "arrow", "datafusion-common", @@ -1852,9 +1870,9 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "18.0.0" +version = "19.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a3d12047a5847f9667f4e2aa8fa2e7d5a6e1094b8e3546d58de492152a50dc7" +checksum = "096f293799e8ae883e0f79f8ebaa51e4292e690ba45e0269b48ca9bd79f57094" dependencies = [ "arrow-schema", "datafusion-common", @@ -1884,8 +1902,8 @@ checksum = "eaa37046cc0f6c3cc6090fbdbf73ef0b8ef4cfcc37f6befc0020f63e8cf121e1" [[package]] name = "deltalake" -version = "0.7.0" -source = "git+https://github.com/delta-io/delta-rs#8dcf46e614370b93a6d47402456621b713768674" +version = "0.8.0" +source = "git+https://github.com/delta-io/delta-rs?rev=cf95721866c3bf8e815cfa60195181f7282ea4fd#cf95721866c3bf8e815cfa60195181f7282ea4fd" dependencies = [ "arrow", "async-trait", @@ -2039,7 +2057,7 @@ checksum = "1435fa1053d8b2fbbe9be7e97eca7f33d37b28409959813daefc1446a14247f1" [[package]] name = "dynamodb_lock" version = "0.4.3" -source = "git+https://github.com/delta-io/delta-rs#8dcf46e614370b93a6d47402456621b713768674" +source = "git+https://github.com/delta-io/delta-rs?rev=cf95721866c3bf8e815cfa60195181f7282ea4fd#cf95721866c3bf8e815cfa60195181f7282ea4fd" dependencies = [ "async-trait", "log", @@ -2578,7 +2596,7 @@ dependencies = [ [[package]] name = "glibc_version" version = "0.1.2" -source = "git+https://github.com/delta-io/delta-rs#8dcf46e614370b93a6d47402456621b713768674" +source = "git+https://github.com/delta-io/delta-rs?rev=cf95721866c3bf8e815cfa60195181f7282ea4fd#cf95721866c3bf8e815cfa60195181f7282ea4fd" dependencies = [ "regex", ] @@ -2856,7 +2874,6 @@ dependencies = [ "rustls-native-certs", "tokio", "tokio-rustls", - "webpki-roots", ] [[package]] @@ -3688,14 +3705,15 @@ dependencies = [ [[package]] name = "object_store" -version = "0.5.3" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4201837dc4c27a8670f0363b1255cd3845a4f0c521211cced1ed14c1d0cc6d2" +checksum = "e1ea8f683b4f89a64181393742c041520a1a87e9775e6b4c0dd5a3281af05fc6" dependencies = [ "async-trait", "aws-config", + "aws-credential-types", "aws-types", - "base64 0.20.0", + "base64 0.21.0", "bytes", "chrono", "futures", @@ -3799,9 +3817,9 @@ checksum = "3baf96e39c5359d2eb0dd6ccb42c62b91d9678aa68160d261b9e0ccbf9e9dea9" [[package]] name = "outref" -version = "0.1.0" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f222829ae9293e33a9f5e9f440c6760a3d450a64affe1846486b140db81c1f4" +checksum = "4030760ffd992bef45b0ae3f10ce1aba99e33464c90d14dd7c039884963ddc7a" [[package]] name = "owning_ref" @@ -3868,9 +3886,9 @@ dependencies = [ [[package]] name = "parquet" -version = "32.0.0" +version = "33.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23b3d4917209e17e1da5fb07d276da237a42465f0def2b8d5fa5ce0e85855b4c" +checksum = "b1b076829801167d889795cd1957989055543430fa1469cb1f6e32b789bfc764" dependencies = [ "ahash 0.8.1", "arrow-array", @@ -3879,7 +3897,7 @@ dependencies = [ "arrow-data", "arrow-ipc", "arrow-schema", - "arrow-select 32.0.0 (registry+https://github.com/rust-lang/crates.io-index)", + "arrow-select 33.0.0 (registry+https://github.com/rust-lang/crates.io-index)", "base64 0.21.0", "brotli", "bytes", @@ -5320,15 +5338,6 @@ dependencies = [ "libc", ] -[[package]] -name = "simd-abstraction" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9cadb29c57caadc51ff8346233b5cec1d240b68ce55cf1afc764818791876987" -dependencies = [ - "outref", -] - [[package]] name = "siphasher" version = "0.3.10" @@ -5813,9 +5822,9 @@ checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" -version = "1.21.2" +version = "1.26.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9e03c497dc955702ba729190dc4aac6f2a0ce97f913e5b1b5912fc5039d9099" +checksum = "03201d01c3c27a29c8a5cee5b55a93ddae1ccf6f08f65365c2c918f8c1b76f64" dependencies = [ "autocfg", "bytes", @@ -5828,7 +5837,7 @@ dependencies = [ "signal-hook-registry", "socket2", "tokio-macros", - "winapi", + "windows-sys 0.45.0", ] [[package]] @@ -6193,6 +6202,12 @@ version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" +[[package]] +name = "vsimd" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c3082ca00d5a5ef149bb8b555a72ae84c9c59f7250f013ac822ac2e49b19c64" + [[package]] name = "waker-fn" version = "1.1.0" diff --git a/Cargo.toml b/Cargo.toml index 15bad75c..6e74b04b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -28,13 +28,13 @@ object-store-s3 = ["object_store/aws"] remote-tables = ["dep:datafusion-remote-tables"] [dependencies] -arrow = "32.0.0" -arrow-buffer = "32.0.0" +arrow = "33.0.0" +arrow-buffer = "33.0.0" # For the JSON format support # https://github.com/apache/arrow-rs/pull/2868 # https://github.com/apache/arrow-rs/pull/2724 -arrow-integration-test = "32.0.0" -arrow-schema = "32.0.0" +arrow-integration-test = "33.0.0" +arrow-schema = "33.0.0" async-trait = "0.1.64" base64 = "0.21.0" @@ -44,18 +44,20 @@ clap = { version = "3.2.19", features = [ "derive" ] } config = "0.13.1" # PG wire protocol support -convergence = { git = "https://github.com/splitgraph/convergence", branch = "datafusion-18-upgrade", optional = true } -convergence-arrow = { git = "https://github.com/splitgraph/convergence", branch = "datafusion-18-upgrade", package = "convergence-arrow", optional = true } +convergence = { git = "https://github.com/splitgraph/convergence", branch = "datafusion-19-upgrade", optional = true } +convergence-arrow = { git = "https://github.com/splitgraph/convergence", branch = "datafusion-19-upgrade", package = "convergence-arrow", optional = true } -datafusion = "18.0.0" -datafusion-common = "18.0.0" -datafusion-expr = "18.0.0" -datafusion-proto = "18.0.0" +datafusion = "19.0.0" +datafusion-common = "19.0.0" +datafusion-expr = "19.0.0" +datafusion-proto = "19.0.0" datafusion-remote-tables = { path = "./datafusion_remote_tables", optional = true } -deltalake = { git = "https://github.com/delta-io/delta-rs", hash = "8dcf46e614370b93a6d47402456621b713768674", features = ["s3-native-tls", "datafusion-ext"] } -dynamodb_lock = { git = "https://github.com/delta-io/delta-rs", package = "dynamodb_lock", hash = "8dcf46e614370b93a6d47402456621b713768674", default_features = false, features = ["native-tls"] } + +# Pick up unique delta object store url: https://github.com/delta-io/delta-rs/pull/1212 +deltalake = { git = "https://github.com/delta-io/delta-rs", rev = "cf95721866c3bf8e815cfa60195181f7282ea4fd", features = ["s3-native-tls", "datafusion-ext"] } +dynamodb_lock = { git = "https://github.com/delta-io/delta-rs", package = "dynamodb_lock", rev = "cf95721866c3bf8e815cfa60195181f7282ea4fd", default_features = false, features = ["native-tls"] } futures = "0.3" hex = ">=0.4.0" @@ -96,15 +98,16 @@ wasmtime = "1.0.1" wasmtime-wasi = "1.0.1" [patch.crates-io] -arrow-array = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-array", branch = "backport-cast-utf8-to-timestamp-microsecond" } -arrow-buffer = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-buffer", branch = "backport-cast-utf8-to-timestamp-microsecond" } -arrow-cast = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-cast", branch = "backport-cast-utf8-to-timestamp-microsecond" } -arrow-data = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-data", branch = "backport-cast-utf8-to-timestamp-microsecond" } -arrow-schema = { git = "https://github.com/splitgraph/arrow-rs", package = "arrow-schema", branch = "backport-cast-utf8-to-timestamp-microsecond" } +# Post-33 version with string to us timestamp casting: https://github.com/apache/arrow-rs/pull/3752 +arrow-array = { git = "https://github.com/splitgraph/arrow-rs", rev = "57f79c03a8dee9d8bf8601bf555aa271746913fe", package = "arrow-array" } +arrow-buffer = { git = "https://github.com/splitgraph/arrow-rs", rev = "57f79c03a8dee9d8bf8601bf555aa271746913fe", package = "arrow-buffer" } +arrow-cast = { git = "https://github.com/splitgraph/arrow-rs", rev = "57f79c03a8dee9d8bf8601bf555aa271746913fe", package = "arrow-cast" } +arrow-data = { git = "https://github.com/splitgraph/arrow-rs", rev = "57f79c03a8dee9d8bf8601bf555aa271746913fe", package = "arrow-data" } +arrow-schema = { git = "https://github.com/splitgraph/arrow-rs", rev = "57f79c03a8dee9d8bf8601bf555aa271746913fe", package = "arrow-schema" } [dev-dependencies] assert_unordered = "0.3" -datafusion-common = "18.0.0" +datafusion-common = "19.0.0" mockall = "0.11.1" rstest = "*" wiremock = "0.5" diff --git a/datafusion_remote_tables/Cargo.toml b/datafusion_remote_tables/Cargo.toml index d8616bec..d8c6b0e5 100644 --- a/datafusion_remote_tables/Cargo.toml +++ b/datafusion_remote_tables/Cargo.toml @@ -13,16 +13,16 @@ license = "Apache-2.0" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] -arrow = "32.0.0" -arrow-buffer = "32.0.0" -arrow-schema = "32.0.0" +arrow = "33.0.0" +arrow-buffer = "33.0.0" +arrow-schema = "33.0.0" async-trait = "0.1.64" # Remote query execution for a variety of DBs -connectorx = { git = "https://github.com/splitgraph/connector-x", branch = "datafusion-18-upgrade", features = [ "dst_arrow", "src_postgres", "src_mysql", "src_sqlite" ] } +connectorx = { git = "https://github.com/splitgraph/connector-x", rev = "df8f50b3f53606717407c6677c7c0c2cbcc7f6ce", features = [ "dst_arrow", "src_postgres", "src_mysql", "src_sqlite" ] } -datafusion = "18.0.0" -datafusion-expr = "18.0.0" +datafusion = "19.0.0" +datafusion-expr = "19.0.0" itertools = ">=0.10.0" log = "0.4" tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "signal", "process"] } diff --git a/src/context.rs b/src/context.rs index d1238fe5..7f845348 100644 --- a/src/context.rs +++ b/src/context.rs @@ -2406,9 +2406,9 @@ mod tests { use super::test_utils::{in_memory_context, mock_context}; const PARTITION_1_FILE_NAME: &str = - "0d5bb8d787b39a501c1c677dc9cabf7fbdb5c10152e48499d8b365f41111aa54.parquet"; + "da10a88bcaf38ae8cca631e4835b6d75d5a948272f7f4d49d586f00071400f35.parquet"; const PARTITION_2_FILE_NAME: &str = - "27fc0c6574c0ffb3706e69abd5babac25a3773b30695a62d41621ddb698de7a6.parquet"; + "59fb7f3c71f611a88b4f915b6d223f313ce91c53557ca11b2a6cdd224371ffa6.parquet"; const EXPECTED_INSERT_FILE_NAME: &str = "67a68a0a8d05a07c80fc235ca42c63c21c853ba8f590a85220978e484118b322.parquet"; @@ -2700,9 +2700,8 @@ mod tests { .unwrap(); assert_eq!(format!("{plan:?}"), "Dml: op=[Insert] table=[testcol.some_table]\ - \n Projection: my_date AS date, my_value AS value\ - \n Projection: testdb.testcol.some_table.date AS my_date, testdb.testcol.some_table.value AS my_value\ - \n TableScan: testdb.testcol.some_table projection=[date, value]"); + \n Projection: testdb.testcol.some_table.date AS date, testdb.testcol.some_table.value AS value\ + \n TableScan: testdb.testcol.some_table projection=[date, value]"); } #[tokio::test] @@ -3033,11 +3032,11 @@ mod tests { "+-----+--------+", "| v | sintau |", "+-----+--------+", - "| 0.1 | 59 |", - "| 0.2 | 95 |", - "| 0.3 | 95 |", - "| 0.4 | 59 |", - "| 0.5 | 0 |", + "| 0.1 | 59.0 |", + "| 0.2 | 95.0 |", + "| 0.3 | 95.0 |", + "| 0.4 | 59.0 |", + "| 0.5 | 0.0 |", "+-----+--------+", ]; diff --git a/src/wasm_udf/wasm.rs b/src/wasm_udf/wasm.rs index 33a1637e..c908fd2a 100644 --- a/src/wasm_udf/wasm.rs +++ b/src/wasm_udf/wasm.rs @@ -885,15 +885,15 @@ f95f3c90f2533d2267773eac66313f1d00803ff725303d03fd3fbe17a6d1\ // decimal places, since the DataFusion's ROUND() simply discards the second argument. // https://github.com/apache/arrow-datafusion/issues/2420 let expected = vec![ - "+-----+------+-------+-------+-------+-------+", - "| sv1 | sv2 | ev1 | ev2 | lv1 | lv2 |", - "+-----+------+-------+-------+-------+-------+", - "| 588 | 951 | 1072 | 1149 | -3322 | -2322 |", - "| 951 | 951 | 2297 | 4925 | 263 | 1202 |", - "| 951 | 951 | 9849 | 19698 | 1722 | 2104 |", - "| 588 | 0 | 42224 | 5657 | 2433 | 1322 |", - "| 0 | -951 | -0 | -0 | 10270 | 9407 |", - "+-----+------+-------+-------+-------+-------+", + "+-------+--------+---------+---------+---------+---------+", + "| sv1 | sv2 | ev1 | ev2 | lv1 | lv2 |", + "+-------+--------+---------+---------+---------+---------+", + "| 588.0 | 951.0 | 1072.0 | 1149.0 | -3322.0 | -2322.0 |", + "| 951.0 | 951.0 | 2297.0 | 4925.0 | 263.0 | 1202.0 |", + "| 951.0 | 951.0 | 9849.0 | 19698.0 | 1722.0 | 2104.0 |", + "| 588.0 | 0.0 | 42224.0 | 5657.0 | 2433.0 | 1322.0 |", + "| 0.0 | -951.0 | 0.0 | 0.0 | 10270.0 | 9407.0 |", + "+-------+--------+---------+---------+---------+---------+", ]; assert_batches_eq!(expected, &results); @@ -1268,11 +1268,11 @@ c40201087f230041206b2203240020032002370318200320013703102003\ "+-----+------+---------+", "| v1 | v2 | product |", "+-----+------+---------+", - "| 0.5 | 2 | 1 |", - "| 3.5 | 4.1 | 14 |", - "| 5.4 | 6.2 | 33 |", - "| 7 | 8.9 | 62 |", - "| 9.1 | 10.2 | 93 |", + "| 0.5 | 2.0 | 1.0 |", + "| 3.5 | 4.1 | 14.0 |", + "| 5.4 | 6.2 | 33.0 |", + "| 7.0 | 8.9 | 62.0 |", + "| 9.1 | 10.2 | 93.0 |", "+-----+------+---------+", ]; diff --git a/tests/statements/ddl.rs b/tests/statements/ddl.rs index 3a946bef..fd938c99 100644 --- a/tests/statements/ddl.rs +++ b/tests/statements/ddl.rs @@ -66,9 +66,9 @@ async fn test_create_table_as() { "+------------+----------------+-------------+", "| some_value | some_int_value | some_minute |", "+------------+----------------+-------------+", - "| 49 | 3333 | 3 |", - "| 48 | 2222 | 2 |", - "| 47 | 1111 | 1 |", + "| 49.0 | 3333 | 3.0 |", + "| 48.0 | 2222 | 2.0 |", + "| 47.0 | 1111 | 1.0 |", "+------------+----------------+-------------+", ]; assert_batches_eq!(expected, &results); diff --git a/tests/statements/dml.rs b/tests/statements/dml.rs index 4a4fe1e9..603ff372 100644 --- a/tests/statements/dml.rs +++ b/tests/statements/dml.rs @@ -26,11 +26,11 @@ async fn test_insert_two_different_schemas() { "+---------------------+------------+------------------+-----------------+----------------+", "| some_time | some_value | some_other_value | some_bool_value | some_int_value |", "+---------------------+------------+------------------+-----------------+----------------+", - "| 2022-01-01T20:01:01 | 42 | | | 1111 |", - "| 2022-01-01T20:02:02 | 43 | | | 2222 |", - "| 2022-01-01T20:03:03 | 44 | | | 3333 |", - "| | 41 | 2.1500000000 | false | |", - "| | 45 | 9.1200000000 | true | |", + "| 2022-01-01T20:01:01 | 42.0 | | | 1111 |", + "| 2022-01-01T20:02:02 | 43.0 | | | 2222 |", + "| 2022-01-01T20:03:03 | 44.0 | | | 3333 |", + "| | 41.0 | 2.1500000000 | false | |", + "| | 45.0 | 9.1200000000 | true | |", "| | | 44.3400000000 | false | |", "+---------------------+------------+------------------+-----------------+----------------+", ]; @@ -77,15 +77,15 @@ async fn test_delete_statement() { "+------------+", "| some_value |", "+------------+", - "| 40 |", - "| 41 |", - "| 42 |", - "| 42 |", - "| 43 |", - "| 44 |", - "| 45 |", - "| 46 |", - "| 46 |", + "| 40.0 |", + "| 41.0 |", + "| 42.0 |", + "| 42.0 |", + "| 43.0 |", + "| 44.0 |", + "| 45.0 |", + "| 46.0 |", + "| 46.0 |", "+------------+", ]; assert_batches_eq!(expected, &results); @@ -135,15 +135,15 @@ async fn test_delete_statement() { "+------------+", "| some_value |", "+------------+", - "| 40 |", - "| 41 |", - "| 42 |", - "| 42 |", - "| 44 |", - "| 46 |", - "| 46 |", - "| 48 |", - "| 50 |", + "| 40.0 |", + "| 41.0 |", + "| 42.0 |", + "| 42.0 |", + "| 44.0 |", + "| 46.0 |", + "| 46.0 |", + "| 48.0 |", + "| 50.0 |", "+------------+", ]; assert_batches_eq!(expected, &results); @@ -167,12 +167,12 @@ async fn test_delete_statement() { "+------------+", "| some_value |", "+------------+", - "| 41 |", - "| 42 |", - "| 42 |", - "| 44 |", - "| 46 |", - "| 46 |", + "| 41.0 |", + "| 42.0 |", + "| 42.0 |", + "| 44.0 |", + "| 46.0 |", + "| 46.0 |", "+------------+", ]; assert_batches_eq!(expected, &results); @@ -301,18 +301,18 @@ async fn test_update_statement() { "+---------------------+------------+------------------+-----------------+----------------+", "| some_time | some_value | some_other_value | some_bool_value | some_int_value |", "+---------------------+------------+------------------+-----------------+----------------+", - "| 2022-01-01T21:21:21 | 31 | | | 5555 |", - "| 2022-01-01T21:21:21 | 32 | | | 5555 |", - "| 2022-01-01T21:21:21 | 32 | | | 5555 |", - "| 2022-01-01T21:21:21 | 33 | | | 5555 |", - "| | 40 | | | |", - "| 2022-01-01T20:03:03 | 44 | | | 3333 |", - "| | 45 | | | |", - "| | 46 | | | |", - "| | 46 | | | |", - "| | 47 | | | |", - "| | 47 | | | |", - "| | 48 | | | |", + "| 2022-01-01T21:21:21 | 31.0 | | | 5555 |", + "| 2022-01-01T21:21:21 | 32.0 | | | 5555 |", + "| 2022-01-01T21:21:21 | 32.0 | | | 5555 |", + "| 2022-01-01T21:21:21 | 33.0 | | | 5555 |", + "| | 40.0 | | | |", + "| 2022-01-01T20:03:03 | 44.0 | | | 3333 |", + "| | 45.0 | | | |", + "| | 46.0 | | | |", + "| | 46.0 | | | |", + "| | 47.0 | | | |", + "| | 47.0 | | | |", + "| | 48.0 | | | |", "+---------------------+------------+------------------+-----------------+----------------+", ]; assert_batches_eq!(expected, &results); @@ -369,18 +369,18 @@ async fn test_update_statement() { "+---------------------+------------+------------------+-----------------+----------------+", "| some_time | some_value | some_other_value | some_bool_value | some_int_value |", "+---------------------+------------+------------------+-----------------+----------------+", - "| | 42 | 0.0000000000 | | |", - "| | 42 | 0.0000000000 | | |", - "| | 42 | 0.0000000000 | | |", - "| | 42 | 0.0000000000 | | |", - "| | 42 | 0.0000000000 | | |", - "| | 42 | 0.0000000000 | | |", - "| 2022-01-01T21:21:21 | 42 | 5.5550000000 | true | 5555 |", - "| 2022-01-01T21:21:21 | 42 | 5.5550000000 | true | 5555 |", - "| 2022-01-01T20:03:03 | 42 | 3.3330000000 | false | 3333 |", - "| 2022-01-01T21:21:21 | 42 | 5.5550000000 | true | 5555 |", - "| 2022-01-01T21:21:21 | 42 | 5.5550000000 | true | 5555 |", - "| | 42 | 0.0000000000 | | |", + "| | 42.0 | 0.0000000000 | | |", + "| | 42.0 | 0.0000000000 | | |", + "| | 42.0 | 0.0000000000 | | |", + "| | 42.0 | 0.0000000000 | | |", + "| | 42.0 | 0.0000000000 | | |", + "| | 42.0 | 0.0000000000 | | |", + "| 2022-01-01T21:21:21 | 42.0 | 5.5550000000 | true | 5555 |", + "| 2022-01-01T21:21:21 | 42.0 | 5.5550000000 | true | 5555 |", + "| 2022-01-01T20:03:03 | 42.0 | 3.3330000000 | false | 3333 |", + "| 2022-01-01T21:21:21 | 42.0 | 5.5550000000 | true | 5555 |", + "| 2022-01-01T21:21:21 | 42.0 | 5.5550000000 | true | 5555 |", + "| | 42.0 | 0.0000000000 | | |", "+---------------------+------------+------------------+-----------------+----------------+", ]; assert_batches_eq!(expected, &results); diff --git a/tests/statements/function.rs b/tests/statements/function.rs index 013a0114..c414f653 100644 --- a/tests/statements/function.rs +++ b/tests/statements/function.rs @@ -34,11 +34,11 @@ async fn test_create_and_run_function() { "+-----+--------+", "| v | sintau |", "+-----+--------+", - "| 0.1 | 59 |", - "| 0.2 | 95 |", - "| 0.3 | 95 |", - "| 0.4 | 59 |", - "| 0.5 | 0 |", + "| 0.1 | 59.0 |", + "| 0.2 | 95.0 |", + "| 0.3 | 95.0 |", + "| 0.4 | 59.0 |", + "| 0.5 | 0.0 |", "+-----+--------+", ]; @@ -87,11 +87,11 @@ async fn test_create_and_run_function_legacy_type_names() { "+-----+--------+", "| v | sintau |", "+-----+--------+", - "| 0.1 | 59 |", - "| 0.2 | 95 |", - "| 0.3 | 95 |", - "| 0.4 | 59 |", - "| 0.5 | 0 |", + "| 0.1 | 59.0 |", + "| 0.2 | 95.0 |", + "| 0.3 | 95.0 |", + "| 0.4 | 59.0 |", + "| 0.5 | 0.0 |", "+-----+--------+", ]; @@ -140,11 +140,11 @@ async fn test_create_and_run_function_uppercase_type_names() { "+-----+--------+", "| v | sintau |", "+-----+--------+", - "| 0.1 | 59 |", - "| 0.2 | 95 |", - "| 0.3 | 95 |", - "| 0.4 | 59 |", - "| 0.5 | 0 |", + "| 0.1 | 59.0 |", + "| 0.2 | 95.0 |", + "| 0.3 | 95.0 |", + "| 0.4 | 59.0 |", + "| 0.5 | 0.0 |", "+-----+--------+", ]; diff --git a/tests/statements/query.rs b/tests/statements/query.rs index 208cc7ce..c2d69ee4 100644 --- a/tests/statements/query.rs +++ b/tests/statements/query.rs @@ -85,9 +85,9 @@ async fn test_create_table_and_insert() { "+---------------------+------------+------------------+-----------------+----------------+", "| some_time | some_value | some_other_value | some_bool_value | some_int_value |", "+---------------------+------------+------------------+-----------------+----------------+", - "| 2022-01-01T20:01:01 | 42 | | | 1111 |", - "| 2022-01-01T20:02:02 | 43 | | | 2222 |", - "| 2022-01-01T20:03:03 | 44 | | | 3333 |", + "| 2022-01-01T20:01:01 | 42.0 | | | 1111 |", + "| 2022-01-01T20:02:02 | 43.0 | | | 2222 |", + "| 2022-01-01T20:03:03 | 44.0 | | | 3333 |", "+---------------------+------------+------------------+-----------------+----------------+", ]; @@ -120,7 +120,7 @@ async fn test_create_table_and_insert() { "+--------------------------------+--------------------------------------------+----------------------------+", "| MAX(test_table.some_int_value) | COUNT(DISTINCT test_table.some_bool_value) | MAX(test_table.some_value) |", "+--------------------------------+--------------------------------------------+----------------------------+", - "| 3333 | 0 | 44 |", + "| 3333 | 0 | 44.0 |", "+--------------------------------+--------------------------------------------+----------------------------+", ]; @@ -479,31 +479,16 @@ async fn test_remote_table_querying( .unwrap(); let results = context.collect(plan).await.unwrap(); - let expected = if introspect_schema { - // Connector-X coerces the TIMESTAMP field to Date64, but that data type still has - // millisecond resolution so why isn't it being shown? - vec![ - "+---+--------+-------+------------+------------+--------------------+", - "| a | b | c | date field | e | f |", - "+---+--------+-------+------------+------------+--------------------+", - "| 1 | 1.1 | one | 2022-11-01 | 2022-11-01 | {\"rows\":[1]} |", - "| 2 | 2.22 | two | 2022-11-02 | 2022-11-02 | {\"rows\":[1,2]} |", - "| 3 | 3.333 | three | 2022-11-03 | 2022-11-03 | {\"rows\":[1,2,3]} |", - "| 4 | 4.4444 | four | 2022-11-04 | 2022-11-04 | {\"rows\":[1,2,3,4]} |", - "+---+--------+-------+------------+------------+--------------------+", - ] - } else { - vec![ - "+---+--------+-------+------------+---------------------+--------------------+", - "| a | b | c | date field | e | f |", - "+---+--------+-------+------------+---------------------+--------------------+", - "| 1 | 1.1 | one | 2022-11-01 | 2022-11-01T22:11:01 | {\"rows\":[1]} |", - "| 2 | 2.22 | two | 2022-11-02 | 2022-11-02T22:11:02 | {\"rows\":[1,2]} |", - "| 3 | 3.333 | three | 2022-11-03 | 2022-11-03T22:11:03 | {\"rows\":[1,2,3]} |", - "| 4 | 4.4444 | four | 2022-11-04 | 2022-11-04T22:11:04 | {\"rows\":[1,2,3,4]} |", - "+---+--------+-------+------------+---------------------+--------------------+", - ] - }; + let expected= vec![ + "+---+--------+-------+------------+---------------------+--------------------+", + "| a | b | c | date field | e | f |", + "+---+--------+-------+------------+---------------------+--------------------+", + "| 1 | 1.1 | one | 2022-11-01 | 2022-11-01T22:11:01 | {\"rows\":[1]} |", + "| 2 | 2.22 | two | 2022-11-02 | 2022-11-02T22:11:02 | {\"rows\":[1,2]} |", + "| 3 | 3.333 | three | 2022-11-03 | 2022-11-03T22:11:03 | {\"rows\":[1,2,3]} |", + "| 4 | 4.4444 | four | 2022-11-04 | 2022-11-04T22:11:04 | {\"rows\":[1,2,3,4]} |", + "+---+--------+-------+------------+---------------------+--------------------+", + ]; assert_batches_eq!(expected, &results); // Test that projection and filtering work diff --git a/tests/statements/query_legacy.rs b/tests/statements/query_legacy.rs index aaaf58aa..e990b435 100644 --- a/tests/statements/query_legacy.rs +++ b/tests/statements/query_legacy.rs @@ -70,18 +70,18 @@ async fn test_legacy_tables() { "+---------------------+------------+------------------+-----------------+----------------+", "| some_time | some_value | some_other_value | some_bool_value | some_int_value |", "+---------------------+------------+------------------+-----------------+----------------+", - "| | 40 | | | |", - "| | 41 | | | |", - "| 2022-01-01T20:01:01 | 42 | | | 1111 |", - "| | 42 | | | |", - "| 2022-01-01T20:02:02 | 43 | | | 2222 |", - "| 2022-01-01T20:03:03 | 44 | | | 3333 |", - "| | 45 | | | |", - "| | 46 | | | |", - "| | 46 | | | |", - "| | 47 | | | |", - "| | 47 | | | |", - "| | 48 | | | |", + "| | 40.0 | | | |", + "| | 41.0 | | | |", + "| 2022-01-01T20:01:01 | 42.0 | | | 1111 |", + "| | 42.0 | | | |", + "| 2022-01-01T20:02:02 | 43.0 | | | 2222 |", + "| 2022-01-01T20:03:03 | 44.0 | | | 3333 |", + "| | 45.0 | | | |", + "| | 46.0 | | | |", + "| | 46.0 | | | |", + "| | 47.0 | | | |", + "| | 47.0 | | | |", + "| | 48.0 | | | |", "+---------------------+------------+------------------+-----------------+----------------+", ]; @@ -207,7 +207,7 @@ async fn test_legacy_tables() { "+--------------------------------+--------------------------------------------+----------------------------+", "| MAX(test_table.some_int_value) | COUNT(DISTINCT test_table.some_bool_value) | MAX(test_table.some_value) |", "+--------------------------------+--------------------------------------------+----------------------------+", - "| 3333 | 0 | 48 |", + "| 3333 | 0 | 48.0 |", "+--------------------------------+--------------------------------------------+----------------------------+", ]; @@ -332,9 +332,9 @@ async fn test_legacy_tables() { "+-----------------+----------------+------------------+---------------------+------------+", "| some_bool_value | some_int_value | some_other_value | some_time | some_value |", "+-----------------+----------------+------------------+---------------------+------------+", - "| | 1111 | | 2022-01-01T20:01:01 | 42 |", - "| | 2222 | | 2022-01-01T20:02:02 | 43 |", - "| | 3333 | | 2022-01-01T20:03:03 | 44 |", + "| | 1111 | | 2022-01-01T20:01:01 | 42.0 |", + "| | 2222 | | 2022-01-01T20:02:02 | 43.0 |", + "| | 3333 | | 2022-01-01T20:03:03 | 44.0 |", "+-----------------+----------------+------------------+---------------------+------------+", ]; @@ -483,10 +483,10 @@ async fn test_vacuum_legacy_tables() { "+------------+", "| some_value |", "+------------+", - "| 40 |", - "| 41 |", - "| 42 |", - "| 42 |", + "| 40.0 |", + "| 41.0 |", + "| 42.0 |", + "| 42.0 |", "+------------+", ]; From 0e2543e1a5ebb384aa989fcc86aa0d243d7d937c Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Tue, 14 Mar 2023 11:49:51 +0100 Subject: [PATCH 20/21] Move copied delta-rs methods to a separate module --- ...0221081920_delta_lake_integration.down.sql | 2 +- src/context.rs | 253 +----------------- src/delta_rs/backports.rs | 248 +++++++++++++++++ src/delta_rs/mod.rs | 1 + src/lib.rs | 1 + src/object_store/wrapped.rs | 2 +- src/repository/default.rs | 2 +- src/system_tables.rs | 10 +- 8 files changed, 264 insertions(+), 255 deletions(-) create mode 100644 src/delta_rs/backports.rs create mode 100644 src/delta_rs/mod.rs diff --git a/migrations/postgres/20230221081920_delta_lake_integration.down.sql b/migrations/postgres/20230221081920_delta_lake_integration.down.sql index 7e3b0a47..0c59763d 100644 --- a/migrations/postgres/20230221081920_delta_lake_integration.down.sql +++ b/migrations/postgres/20230221081920_delta_lake_integration.down.sql @@ -2,4 +2,4 @@ ALTER TABLE "table" DROP COLUMN legacy; ALTER TABLE "table" DROP COLUMN uuid; -DROP TABLE dropped_table; \ No newline at end of file +DROP TABLE dropped_table; diff --git a/src/context.rs b/src/context.rs index 7f845348..53a479a5 100644 --- a/src/context.rs +++ b/src/context.rs @@ -16,7 +16,7 @@ use tokio::io::{AsyncReadExt, AsyncWriteExt, BufReader}; use std::fs::File; use datafusion::datasource::listing::{ - ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, PartitionedFile, + ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, }; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::execution::context::{default_session_builder, SessionState}; @@ -34,17 +34,16 @@ use futures::{StreamExt, TryStreamExt}; #[cfg(test)] use mockall::automock; -use object_store::{path::Path, ObjectMeta, ObjectStore}; +use object_store::{path::Path, ObjectStore}; use sqlparser::ast::{ AlterTableOperation, CreateFunctionBody, FunctionDefinition, Ident, ObjectName, ObjectType, SchemaName, Statement, TableFactor, TableWithJoins, }; -use arrow::compute::{cast_with_options, CastOptions}; use arrow_integration_test::field_to_json; -use arrow_schema::{ArrowError, DataType, TimeUnit}; -use chrono::{DateTime, FixedOffset, NaiveDateTime, Utc}; +use arrow_schema::DataType; +use chrono::{DateTime, FixedOffset, Utc}; use std::iter::zip; use std::ops::Deref; use std::str::FromStr; @@ -64,7 +63,6 @@ use datafusion::optimizer::{OptimizerContext, OptimizerRule}; use datafusion::physical_expr::create_physical_expr; use datafusion::physical_expr::execution_props::ExecutionProps; use datafusion::physical_optimizer::pruning::PruningPredicate; -use datafusion::physical_plan::file_format::{partition_type_wrap, FileScanConfig}; use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::prelude::SessionConfig; @@ -91,24 +89,20 @@ use datafusion_expr::logical_plan::{ CreateCatalog, CreateCatalogSchema, CreateExternalTable, CreateMemoryTable, DropTable, Extension, LogicalPlan, Projection, }; -use datafusion_expr::{DmlStatement, Expr, Filter, WriteOp}; +use datafusion_expr::{DmlStatement, Filter, WriteOp}; use deltalake::action::{Action, Add, Remove}; -use deltalake::delta_datafusion::DeltaDataChecker; -use deltalake::operations::writer::{DeltaWriter, WriterConfig}; use deltalake::operations::{create::CreateBuilder, write::WriteBuilder}; -use deltalake::storage::DeltaObjectStore; -use deltalake::{DeltaResult, DeltaTable, Schema as DeltaSchema}; +use deltalake::{DeltaTable, Schema as DeltaSchema}; use log::{debug, info, warn}; -use object_store::path::DELIMITER; use parking_lot::RwLock; use prost::Message; use tempfile::TempPath; use tokio::sync::Semaphore; -use url::Url; use uuid::Uuid; use crate::catalog::{PartitionCatalog, DEFAULT_SCHEMA, STAGING_SCHEMA}; use crate::datafusion::visit::VisitorMut; +use crate::delta_rs::backports::{parquet_scan_from_actions, write_execution_plan}; #[cfg(test)] use crate::frontend::http::tests::deterministic_uuid; use crate::provider::{ @@ -530,231 +524,6 @@ pub fn is_statement_read_only(statement: &DFStatement) -> bool { } } -// Appropriated from https://github.com/delta-io/delta-rs/pull/1176; once the DELETE and UPDATE ops -// are available through delta-rs this will be obsolete. -/// Write the provide ExecutionPlan to the underlying storage -/// The table's invariants are checked during this proccess -pub async fn write_execution_plan( - table: &DeltaTable, - state: SessionState, - plan: Arc, - partition_columns: Vec, - object_store: Arc, - target_file_size: Option, - write_batch_size: Option, -) -> Result> { - let invariants = table - .get_metadata() - .and_then(|meta| meta.schema.get_invariants()) - .unwrap_or_default(); - let checker = DeltaDataChecker::new(invariants); - - // Write data to disk - let mut tasks = vec![]; - for i in 0..plan.output_partitioning().partition_count() { - let inner_plan = plan.clone(); - let task_ctx = Arc::new(TaskContext::from(&state)); - - let config = WriterConfig::new( - inner_plan.schema(), - partition_columns.clone(), - None, - target_file_size, - write_batch_size, - ); - let mut writer = DeltaWriter::new(object_store.clone(), config); - let checker_stream = checker.clone(); - let mut stream = inner_plan.execute(i, task_ctx)?; - let handle: tokio::task::JoinHandle>> = - tokio::task::spawn(async move { - while let Some(maybe_batch) = stream.next().await { - let batch = maybe_batch?; - checker_stream.check_batch(&batch).await?; - writer.write(&batch).await?; - } - writer.close().await - }); - - tasks.push(handle); - } - - // Collect add actions to add to commit - Ok(futures::future::join_all(tasks) - .await - .into_iter() - .collect::, _>>() - .map_err(|err| { - DataFusionError::Execution(format!( - "Failed writing to delta table {table}: {err}" - )) - })? - .into_iter() - .collect::, _>>()? - .concat() - .into_iter() - .collect::>()) -} - -// Appropriated from https://github.com/delta-io/delta-rs/pull/1176 with minor changes. -// Once the DELETE and UPDATE ops are available through delta-rs this will be obsolete. -/// Create a Parquet scan limited to a set of files -pub async fn parquet_scan_from_actions( - table: &DeltaTable, - actions: &[Add], - schema: &Schema, - filters: &[Expr], - state: &SessionState, - projection: Option<&Vec>, - limit: Option, -) -> Result> { - // TODO we group files together by their partition values. If the table is partitioned - // and partitions are somewhat evenly distributed, probably not the worst choice ... - // However we may want to do some additional balancing in case we are far off from the above. - let mut file_groups: HashMap, Vec> = HashMap::new(); - for action in actions { - let part = partitioned_file_from_action(action, schema); - file_groups - .entry(part.partition_values.clone()) - .or_default() - .push(part); - } - - let table_partition_cols = table.get_metadata()?.partition_columns.clone(); - let file_schema = Arc::new(Schema::new( - schema - .fields() - .iter() - .filter(|f| !table_partition_cols.contains(f.name())) - .cloned() - .collect(), - )); - - let url = Url::parse(&table.table_uri()).unwrap(); - let host = format!( - "{}-{}{}", - url.scheme(), - url.host_str().unwrap_or_default(), - url.path().replace(DELIMITER, "-").replace(':', "-") - ); - state - .runtime_env() - .register_object_store("delta-rs", &host, table.object_store()); - let object_store_url = ObjectStoreUrl::parse(format!("delta-rs://{host}"))?; - - ParquetFormat::new() - .create_physical_plan( - state, - FileScanConfig { - object_store_url, - file_schema, - file_groups: file_groups.into_values().collect(), - statistics: table.datafusion_table_statistics(), - projection: projection.cloned(), - limit, - table_partition_cols: table_partition_cols - .iter() - .map(|c| { - Ok(( - c.to_owned(), - partition_type_wrap( - schema.field_with_name(c)?.data_type().clone(), - ), - )) - }) - .collect::, ArrowError>>()?, - output_ordering: None, - infinite_source: false, - }, - filters, - ) - .await -} - -// Copied from delta-rs as it's private there; once the DELETE and UPDATE ops -// are available through delta-rs this will be obsolete. -fn partitioned_file_from_action(action: &Add, schema: &Schema) -> PartitionedFile { - let partition_values = schema - .fields() - .iter() - .filter_map(|f| { - action.partition_values.get(f.name()).map(|val| match val { - Some(value) => to_correct_scalar_value( - &serde_json::Value::String(value.to_string()), - f.data_type(), - ) - .unwrap_or(ScalarValue::Null), - None => ScalarValue::Null, - }) - }) - .collect::>(); - - let ts_secs = action.modification_time / 1000; - let ts_ns = (action.modification_time % 1000) * 1_000_000; - let last_modified = DateTime::::from_utc( - NaiveDateTime::from_timestamp_opt(ts_secs, ts_ns as u32).unwrap(), - Utc, - ); - PartitionedFile { - object_meta: ObjectMeta { - location: Path::from(action.path.clone()), - last_modified, - size: action.size as usize, - }, - partition_values, - range: None, - extensions: None, - } -} - -// Copied from delta-rs as it's private there; once the DELETE and UPDATE ops -// are available through delta-rs this will be obsolete. -fn to_correct_scalar_value( - stat_val: &serde_json::Value, - field_dt: &DataType, -) -> Option { - match stat_val { - serde_json::Value::Array(_) => None, - serde_json::Value::Object(_) => None, - serde_json::Value::Null => None, - serde_json::Value::String(string_val) => match field_dt { - DataType::Timestamp(_, _) => { - let time_nanos = ScalarValue::try_from_string( - string_val.to_owned(), - &DataType::Timestamp(TimeUnit::Nanosecond, None), - ) - .ok()?; - let cast_arr = cast_with_options( - &time_nanos.to_array(), - field_dt, - &CastOptions { safe: false }, - ) - .ok()?; - Some(ScalarValue::try_from_array(&cast_arr, 0).ok()?) - } - _ => { - Some(ScalarValue::try_from_string(string_val.to_owned(), field_dt).ok()?) - } - }, - other => match field_dt { - DataType::Timestamp(_, _) => { - let time_nanos = ScalarValue::try_from_string( - other.to_string(), - &DataType::Timestamp(TimeUnit::Nanosecond, None), - ) - .ok()?; - let cast_arr = cast_with_options( - &time_nanos.to_array(), - field_dt, - &CastOptions { safe: false }, - ) - .ok()?; - Some(ScalarValue::try_from_array(&cast_arr, 0).ok()?) - } - _ => Some(ScalarValue::try_from_string(other.to_string(), field_dt).ok()?), - }, - } -} - // The only reason to keep this trait around (instead of migrating all the functions directly into // DefaultSeafowlContext), is that `create_physical_plan` would then be a recursive async function, // which works for traits, but not for structs: https://stackoverflow.com/a/74737853 @@ -1088,7 +857,7 @@ impl DefaultSeafowlContext { .await?; // TODO: if `DeltaTable::get_version_timestamp` was globally public we could also pass the - // exact version timestamp, instead of creating onw automatically in our own catalog (which + // exact version timestamp, instead of creating one automatically in our own catalog (which // could lead to minor timestamp differences). self.table_catalog .create_new_table_version(table_uuid, table.version()) @@ -1675,7 +1444,7 @@ impl SeafowlContext for DefaultSeafowlContext { // This is actually CREATE TABLE AS let physical = self.create_physical_plan(input).await?; - // First create the table and then insert the data from the subqeury + // First create the table and then insert the data from the subquery // TODO: this means we'll have 2 table versions at the end, 1st from the create // and 2nd from the insert, while it seems more reasonable that in this case we have // only one @@ -2070,10 +1839,6 @@ impl SeafowlContext for DefaultSeafowlContext { .get_collection_id_by_name(&self.database, name) .await? { - // TODO: as for DROP TABLE, we should really only flag all tables - // in the schema as DROPED, and then try to lazy delete them during - // subsequent `VACUUM`s. Only once all the underlying table files - // are deleted is it safe to drop the collection. self.table_catalog.drop_collection(collection_id).await? }; diff --git a/src/delta_rs/backports.rs b/src/delta_rs/backports.rs new file mode 100644 index 00000000..5da6a2ed --- /dev/null +++ b/src/delta_rs/backports.rs @@ -0,0 +1,248 @@ +use arrow::compute::{cast_with_options, CastOptions}; +use arrow_schema::{ArrowError, DataType, Schema, TimeUnit}; +use chrono::{DateTime, NaiveDateTime, Utc}; +use datafusion::datasource::file_format::parquet::ParquetFormat; +use datafusion::datasource::file_format::FileFormat; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::object_store::ObjectStoreUrl; +use datafusion::execution::context::{SessionState, TaskContext}; +use datafusion::physical_plan::file_format::{partition_type_wrap, FileScanConfig}; +use datafusion::physical_plan::ExecutionPlan; +use datafusion_common::{DataFusionError, Result, ScalarValue}; +use datafusion_expr::Expr; +use deltalake::action::Add; +use deltalake::delta_datafusion::DeltaDataChecker; +use deltalake::operations::writer::{DeltaWriter, WriterConfig}; +use deltalake::storage::DeltaObjectStore; +use deltalake::{DeltaResult, DeltaTable}; +use futures::StreamExt; +use object_store::path::{Path, DELIMITER}; +use object_store::ObjectMeta; +use std::collections::HashMap; +use std::sync::Arc; +use url::Url; + +// Appropriated from https://github.com/delta-io/delta-rs/pull/1176; once the DELETE and UPDATE ops +// are available through delta-rs this will be obsolete. +/// Write the provide ExecutionPlan to the underlying storage +/// The table's invariants are checked during this proccess +pub async fn write_execution_plan( + table: &DeltaTable, + state: SessionState, + plan: Arc, + partition_columns: Vec, + object_store: Arc, + target_file_size: Option, + write_batch_size: Option, +) -> Result> { + let invariants = table + .get_metadata() + .and_then(|meta| meta.schema.get_invariants()) + .unwrap_or_default(); + let checker = DeltaDataChecker::new(invariants); + + // Write data to disk + let mut tasks = vec![]; + for i in 0..plan.output_partitioning().partition_count() { + let inner_plan = plan.clone(); + let task_ctx = Arc::new(TaskContext::from(&state)); + + let config = WriterConfig::new( + inner_plan.schema(), + partition_columns.clone(), + None, + target_file_size, + write_batch_size, + ); + let mut writer = DeltaWriter::new(object_store.clone(), config); + let checker_stream = checker.clone(); + let mut stream = inner_plan.execute(i, task_ctx)?; + let handle: tokio::task::JoinHandle>> = + tokio::task::spawn(async move { + while let Some(maybe_batch) = stream.next().await { + let batch = maybe_batch?; + checker_stream.check_batch(&batch).await?; + writer.write(&batch).await?; + } + writer.close().await + }); + + tasks.push(handle); + } + + // Collect add actions to add to commit + Ok(futures::future::join_all(tasks) + .await + .into_iter() + .collect::, _>>() + .map_err(|err| { + DataFusionError::Execution(format!( + "Failed writing to delta table {table}: {err}" + )) + })? + .into_iter() + .collect::, _>>()? + .concat() + .into_iter() + .collect::>()) +} + +// Appropriated from https://github.com/delta-io/delta-rs/pull/1176 with minor changes. +// Once the DELETE and UPDATE ops are available through delta-rs this will be obsolete. +/// Create a Parquet scan limited to a set of files +pub async fn parquet_scan_from_actions( + table: &DeltaTable, + actions: &[Add], + schema: &Schema, + filters: &[Expr], + state: &SessionState, + projection: Option<&Vec>, + limit: Option, +) -> Result> { + // TODO we group files together by their partition values. If the table is partitioned + // and partitions are somewhat evenly distributed, probably not the worst choice ... + // However we may want to do some additional balancing in case we are far off from the above. + let mut file_groups: HashMap, Vec> = HashMap::new(); + for action in actions { + let part = partitioned_file_from_action(action, schema); + file_groups + .entry(part.partition_values.clone()) + .or_default() + .push(part); + } + + let table_partition_cols = table.get_metadata()?.partition_columns.clone(); + let file_schema = Arc::new(Schema::new( + schema + .fields() + .iter() + .filter(|f| !table_partition_cols.contains(f.name())) + .cloned() + .collect(), + )); + + let url = Url::parse(&table.table_uri()).unwrap(); + let host = format!( + "{}-{}{}", + url.scheme(), + url.host_str().unwrap_or_default(), + url.path().replace(DELIMITER, "-").replace(':', "-") + ); + state + .runtime_env() + .register_object_store("delta-rs", &host, table.object_store()); + let object_store_url = ObjectStoreUrl::parse(format!("delta-rs://{host}"))?; + + ParquetFormat::new() + .create_physical_plan( + state, + FileScanConfig { + object_store_url, + file_schema, + file_groups: file_groups.into_values().collect(), + statistics: table.datafusion_table_statistics(), + projection: projection.cloned(), + limit, + table_partition_cols: table_partition_cols + .iter() + .map(|c| { + Ok(( + c.to_owned(), + partition_type_wrap( + schema.field_with_name(c)?.data_type().clone(), + ), + )) + }) + .collect::, ArrowError>>()?, + output_ordering: None, + infinite_source: false, + }, + filters, + ) + .await +} + +// Copied from delta-rs as it's private there; once the DELETE and UPDATE ops +// are available through delta-rs this will be obsolete. +fn partitioned_file_from_action(action: &Add, schema: &Schema) -> PartitionedFile { + let partition_values = schema + .fields() + .iter() + .filter_map(|f| { + action.partition_values.get(f.name()).map(|val| match val { + Some(value) => to_correct_scalar_value( + &serde_json::Value::String(value.to_string()), + f.data_type(), + ) + .unwrap_or(ScalarValue::Null), + None => ScalarValue::Null, + }) + }) + .collect::>(); + + let ts_secs = action.modification_time / 1000; + let ts_ns = (action.modification_time % 1000) * 1_000_000; + let last_modified = DateTime::::from_utc( + NaiveDateTime::from_timestamp_opt(ts_secs, ts_ns as u32).unwrap(), + Utc, + ); + PartitionedFile { + object_meta: ObjectMeta { + location: Path::from(action.path.clone()), + last_modified, + size: action.size as usize, + }, + partition_values, + range: None, + extensions: None, + } +} + +// Copied from delta-rs as it's private there; once the DELETE and UPDATE ops +// are available through delta-rs this will be obsolete. +fn to_correct_scalar_value( + stat_val: &serde_json::Value, + field_dt: &DataType, +) -> Option { + match stat_val { + serde_json::Value::Array(_) => None, + serde_json::Value::Object(_) => None, + serde_json::Value::Null => None, + serde_json::Value::String(string_val) => match field_dt { + DataType::Timestamp(_, _) => { + let time_nanos = ScalarValue::try_from_string( + string_val.to_owned(), + &DataType::Timestamp(TimeUnit::Nanosecond, None), + ) + .ok()?; + let cast_arr = cast_with_options( + &time_nanos.to_array(), + field_dt, + &CastOptions { safe: false }, + ) + .ok()?; + Some(ScalarValue::try_from_array(&cast_arr, 0).ok()?) + } + _ => { + Some(ScalarValue::try_from_string(string_val.to_owned(), field_dt).ok()?) + } + }, + other => match field_dt { + DataType::Timestamp(_, _) => { + let time_nanos = ScalarValue::try_from_string( + other.to_string(), + &DataType::Timestamp(TimeUnit::Nanosecond, None), + ) + .ok()?; + let cast_arr = cast_with_options( + &time_nanos.to_array(), + field_dt, + &CastOptions { safe: false }, + ) + .ok()?; + Some(ScalarValue::try_from_array(&cast_arr, 0).ok()?) + } + _ => Some(ScalarValue::try_from_string(other.to_string(), field_dt).ok()?), + }, + } +} diff --git a/src/delta_rs/mod.rs b/src/delta_rs/mod.rs new file mode 100644 index 00000000..7dac374a --- /dev/null +++ b/src/delta_rs/mod.rs @@ -0,0 +1 @@ +pub mod backports; diff --git a/src/lib.rs b/src/lib.rs index 56b1dbe5..80417514 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -4,6 +4,7 @@ pub mod config; pub mod context; pub mod data_types; pub mod datafusion; +pub mod delta_rs; pub mod frontend; pub mod nodes; pub mod object_store; diff --git a/src/object_store/wrapped.rs b/src/object_store/wrapped.rs index 9681d901..e3e80213 100644 --- a/src/object_store/wrapped.rs +++ b/src/object_store/wrapped.rs @@ -133,7 +133,7 @@ impl InternalObjectStore { impl Display for InternalObjectStore { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!(f, "FileStorageBackend({})", self.root_uri) + write!(f, "InternalObjectStore({})", self.root_uri) } } diff --git a/src/repository/default.rs b/src/repository/default.rs index 4c98231e..9def75da 100644 --- a/src/repository/default.rs +++ b/src/repository/default.rs @@ -663,7 +663,7 @@ impl Repository for $repo { // them to a special table that is used for lazy cleanup of files via `VACUUM`. // TODO: We could do this via a trigger, but then we'd lose the ability to actually // perform hard deletes at the DB-level. - // NB: We rally only need the uuid for cleanup, but we also persist db/col name on the off + // NB: We really only need the uuid for cleanup, but we also persist db/col name on the off // chance that we want to add table restore/undrop at some point. let mut builder: QueryBuilder<_> = QueryBuilder::new( r#"INSERT INTO dropped_table(database_name, collection_name, table_name, uuid) diff --git a/src/system_tables.rs b/src/system_tables.rs index 3aa65037..5a0d6ad3 100644 --- a/src/system_tables.rs +++ b/src/system_tables.rs @@ -2,9 +2,7 @@ //! and datafusion's information_schema. use crate::catalog::TableCatalog; -use crate::repository::interface::{ - DroppedTablesResult, TablePartitionsResult, TableVersionsResult, -}; +use crate::repository::interface::{DroppedTablesResult, TablePartitionsResult}; use arrow::array::{ Int32Builder, Int64Builder, StringBuilder, StructBuilder, TimestampSecondBuilder, }; @@ -165,7 +163,6 @@ impl TableVersionsTable { Field::new("version", DataType::Int64, false), Field::new( "creation_time", - // TODO: should we be using a concrete timezone here? DataType::Timestamp(TimeUnit::Second, None), false, ), @@ -185,9 +182,7 @@ impl SeafowlSystemTable for TableVersionsTable { let table_versions = self .table_catalog .get_all_table_versions(&self.database, None) - .await? - .into_iter() - .collect::>(); + .await?; let mut builder = StructBuilder::from_fields( self.schema.fields().clone(), @@ -330,7 +325,6 @@ impl DroppedTablesTable { Field::new("deletion_status", DataType::Utf8, false), Field::new( "drop_time", - // TODO: should we be using a concrete timezone here? DataType::Timestamp(TimeUnit::Second, None), false, ), From 9f56b12c72527ae07a9f6f18e7027d38d736d63e Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Tue, 14 Mar 2023 15:14:12 +0100 Subject: [PATCH 21/21] Make table building logic more explicit --- .history | 4 ++++ src/catalog.rs | 26 +++++++++----------------- src/context.rs | 13 +++++++++++++ 3 files changed, 26 insertions(+), 17 deletions(-) create mode 100644 .history diff --git a/.history b/.history new file mode 100644 index 00000000..73d5eec3 --- /dev/null +++ b/.history @@ -0,0 +1,4 @@ +#V2 +create table test; +create table test (); +\\q diff --git a/src/catalog.rs b/src/catalog.rs index bcb57d27..887d0a24 100644 --- a/src/catalog.rs +++ b/src/catalog.rs @@ -364,7 +364,8 @@ impl DefaultCatalog { let table_columns_vec = table_columns.collect_vec(); // Recover the table ID and version ID (this is going to be the same for all columns). - // TODO: if the table has no columns, the result set will be empty, so we use a fake version ID. + // TODO: if the table has no columns then we wouldn't be in this function since we originally + // grouped by the table name before stepping down into here. let (table_id, table_version_id) = table_columns_vec .get(0) .map_or_else(|| (0, 0), |v| (v.table_id, v.table_version_id)); @@ -385,24 +386,13 @@ impl DefaultCatalog { (Arc::from(table_name.to_string()), Arc::new(table)) } - fn build_table<'a, I>( + fn build_table( &self, table_name: &str, - table_columns: I, - ) -> (Arc, Arc) - where - I: Iterator, - { - let table_columns_vec = table_columns.collect_vec(); - + table_uuid: Uuid, + ) -> (Arc, Arc) { // Build a delta table but don't load it yet; we'll do that only for tables that are // actually referenced in a statement, via the async `table` method of the schema provider. - - // TODO: if the table has no columns, the result set will be empty, so we use the default UUID (all zeros). - let table_uuid = table_columns_vec - .get(0) - .map_or_else(Uuid::default, |v| v.table_uuid); - let table_object_store = self.object_store.for_delta_table(table_uuid); let table = DeltaTable::new(table_object_store, Default::default()); @@ -431,9 +421,11 @@ impl DefaultCatalog { let tables = collection_columns_vec .into_iter() .filter(|c| !c.table_legacy) - .group_by(|col| &col.table_name) + .group_by(|col| (&col.table_name, &col.table_uuid)) .into_iter() - .map(|(tn, tc)| self.build_table(tn, tc)) + .map(|((table_name, table_uuid), _)| { + self.build_table(table_name, *table_uuid) + }) .collect::>(); ( diff --git a/src/context.rs b/src/context.rs index 53a479a5..1d258c88 100644 --- a/src/context.rs +++ b/src/context.rs @@ -2469,6 +2469,19 @@ mod tests { \n TableScan: testdb.testcol.some_table projection=[date, value]"); } + #[tokio::test] + async fn test_create_table_without_columns_fails() { + let context = Arc::new(in_memory_context().await); + let err = context + .plan_query("CREATE TABLE test_table") + .await + .unwrap_err(); + + assert!(err + .to_string() + .contains("At least one column must be defined to create a table.")); + } + #[tokio::test] async fn test_drop_table_pending_deletion() -> Result<()> { let context = Arc::new(in_memory_context().await);