From aec1add5f6105db0688f591410558e7033b8c73c Mon Sep 17 00:00:00 2001 From: Stephen Carman Date: Fri, 24 Nov 2023 12:00:59 -0500 Subject: [PATCH 01/13] feat: Add CHECK constraint functionality --- .../src/operations/constraints.rs | 116 ++++++++++++++++++ crates/deltalake-core/src/operations/mod.rs | 9 ++ crates/deltalake-core/src/protocol/mod.rs | 6 + ...-9805-5b9c1c20d5a4-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-8d57-f468582e5fc3-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-a6dc-697e58993bc6-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-9f16-1d38e6c74721-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-bb53-d4bea9526242-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-b1f2-7481029003d3-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-b2ca-81fc04c911f7-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-b110-bd15ea28efa4-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-915f-ed535933662f-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-abb9-0e8a53e1d863-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-a6aa-b8c0e810fce8-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-a27c-4c4e76a1c817-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-986e-e038051952f2-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-a5c5-6cf37b93f174-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-a3cc-58de2fec437d-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-b3c6-d796621d2cb4-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-89d8-baa3ab5348f0-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-ba76-9a4894572379-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-9f82-366415d2e69d-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-ae12-25fc7dbc7583-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-93e2-5470e8041767-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-b241-8d3ff6e3d5c5-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-b9f7-f269423daebd-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-969b-977133666c4d-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-8006-eb0db94111ee-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-acc3-92066e12b06a-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-81e6-8b745ff8a965-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-806f-ca720d517645-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-8490-ca4a17b22d06-c000.snappy.parquet.crc | Bin 0 -> 12 bytes ...-bb78-f217a28c8d5a-c000.snappy.parquet.crc | Bin 0 -> 16 bytes ...-b324-502f75238979-c000.snappy.parquet.crc | Bin 0 -> 16 bytes .../_delta_log/.00000000000000000000.json.crc | Bin 0 -> 36 bytes .../_delta_log/.00000000000000000001.json.crc | Bin 0 -> 16 bytes .../_delta_log/.00000000000000000002.json.crc | Bin 0 -> 36 bytes .../_delta_log/00000000000000000000.json | 13 ++ .../_delta_log/00000000000000000001.json | 3 + .../_delta_log/00000000000000000002.json | 11 ++ ...46a8-9805-5b9c1c20d5a4-c000.snappy.parquet | Bin 0 -> 527 bytes ...421b-8d57-f468582e5fc3-c000.snappy.parquet | Bin 0 -> 296 bytes ...479a-a6dc-697e58993bc6-c000.snappy.parquet | Bin 0 -> 296 bytes ...4401-9f16-1d38e6c74721-c000.snappy.parquet | Bin 0 -> 539 bytes ...4dde-bb53-d4bea9526242-c000.snappy.parquet | Bin 0 -> 296 bytes ...4b3e-b1f2-7481029003d3-c000.snappy.parquet | Bin 0 -> 527 bytes ...4fce-b2ca-81fc04c911f7-c000.snappy.parquet | Bin 0 -> 527 bytes ...4307-b110-bd15ea28efa4-c000.snappy.parquet | Bin 0 -> 296 bytes ...4d17-915f-ed535933662f-c000.snappy.parquet | Bin 0 -> 526 bytes ...470e-abb9-0e8a53e1d863-c000.snappy.parquet | Bin 0 -> 527 bytes ...4e0c-a6aa-b8c0e810fce8-c000.snappy.parquet | Bin 0 -> 527 bytes ...4491-a27c-4c4e76a1c817-c000.snappy.parquet | Bin 0 -> 296 bytes ...4676-986e-e038051952f2-c000.snappy.parquet | Bin 0 -> 527 bytes ...47ed-a5c5-6cf37b93f174-c000.snappy.parquet | Bin 0 -> 527 bytes ...45b4-a3cc-58de2fec437d-c000.snappy.parquet | Bin 0 -> 296 bytes ...4fac-b3c6-d796621d2cb4-c000.snappy.parquet | Bin 0 -> 527 bytes ...41b9-89d8-baa3ab5348f0-c000.snappy.parquet | Bin 0 -> 296 bytes ...4f1f-ba76-9a4894572379-c000.snappy.parquet | Bin 0 -> 527 bytes ...4a80-9f82-366415d2e69d-c000.snappy.parquet | Bin 0 -> 527 bytes ...4534-ae12-25fc7dbc7583-c000.snappy.parquet | Bin 0 -> 528 bytes ...4006-93e2-5470e8041767-c000.snappy.parquet | Bin 0 -> 527 bytes ...48ee-b241-8d3ff6e3d5c5-c000.snappy.parquet | Bin 0 -> 296 bytes ...4cce-b9f7-f269423daebd-c000.snappy.parquet | Bin 0 -> 296 bytes ...4c73-969b-977133666c4d-c000.snappy.parquet | Bin 0 -> 527 bytes ...41f6-8006-eb0db94111ee-c000.snappy.parquet | Bin 0 -> 527 bytes ...4f99-acc3-92066e12b06a-c000.snappy.parquet | Bin 0 -> 527 bytes ...4b80-81e6-8b745ff8a965-c000.snappy.parquet | Bin 0 -> 527 bytes ...4a17-806f-ca720d517645-c000.snappy.parquet | Bin 0 -> 296 bytes ...4160-8490-ca4a17b22d06-c000.snappy.parquet | Bin 0 -> 296 bytes ...4ab9-bb78-f217a28c8d5a-c000.snappy.parquet | Bin 0 -> 527 bytes ...4056-b324-502f75238979-c000.snappy.parquet | Bin 0 -> 527 bytes .../tests/read_delta_log_test.rs | 16 ++- 72 files changed, 173 insertions(+), 1 deletion(-) create mode 100644 crates/deltalake-core/src/operations/constraints.rs create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00000-9d9248d4-bd6e-46a8-9805-5b9c1c20d5a4-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00000-9ed8bdb4-315a-421b-8d57-f468582e5fc3-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00000-ca2db31d-ab94-479a-a6dc-697e58993bc6-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00000-cace3c04-e21c-4401-9f16-1d38e6c74721-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00001-53e441ee-68e6-4dde-bb53-d4bea9526242-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00001-6dea6ffc-5c2d-4b3e-b1f2-7481029003d3-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00001-a5330b91-bece-4fce-b2ca-81fc04c911f7-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00002-04434872-6b96-4307-b110-bd15ea28efa4-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00002-24d4d181-0877-4d17-915f-ed535933662f-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00002-4ca0c9db-02c8-470e-abb9-0e8a53e1d863-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00003-2bd9d2e4-27a6-4e0c-a6aa-b8c0e810fce8-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00003-7996975c-4199-4491-a27c-4c4e76a1c817-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00003-7b12c177-7cfd-4676-986e-e038051952f2-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00004-0aae8759-c282-47ed-a5c5-6cf37b93f174-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00004-4ca9cc40-3cd3-45b4-a3cc-58de2fec437d-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00004-f11e1f93-e35a-4fac-b3c6-d796621d2cb4-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00005-319662f8-ef0e-41b9-89d8-baa3ab5348f0-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00005-3ae98bc2-a3bd-4f1f-ba76-9a4894572379-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00005-d44a39cb-4364-4a80-9f82-366415d2e69d-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00006-05a437ee-7c40-4534-ae12-25fc7dbc7583-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00006-7acd5504-7d04-4006-93e2-5470e8041767-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00006-9d8b5796-2cc6-48ee-b241-8d3ff6e3d5c5-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00007-5a919d87-b7be-4cce-b9f7-f269423daebd-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00007-68dbff7e-17b6-4c73-969b-977133666c4d-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00007-9f374a9d-37ec-41f6-8006-eb0db94111ee-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00008-25115960-51e0-4f99-acc3-92066e12b06a-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00008-568f62b3-4b06-4b80-81e6-8b745ff8a965-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00008-df9a0d39-3663-4a17-806f-ca720d517645-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00009-658f63ed-06a4-4160-8490-ca4a17b22d06-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00009-93e49bd4-3350-4ab9-bb78-f217a28c8d5a-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00009-da77b766-6ecf-4056-b324-502f75238979-c000.snappy.parquet.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000000.json.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000001.json.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000002.json.crc create mode 100644 crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000000.json create mode 100644 crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000001.json create mode 100644 crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000002.json create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00000-9d9248d4-bd6e-46a8-9805-5b9c1c20d5a4-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00000-9ed8bdb4-315a-421b-8d57-f468582e5fc3-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00000-ca2db31d-ab94-479a-a6dc-697e58993bc6-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00000-cace3c04-e21c-4401-9f16-1d38e6c74721-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00001-53e441ee-68e6-4dde-bb53-d4bea9526242-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00001-6dea6ffc-5c2d-4b3e-b1f2-7481029003d3-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00001-a5330b91-bece-4fce-b2ca-81fc04c911f7-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00002-04434872-6b96-4307-b110-bd15ea28efa4-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00002-24d4d181-0877-4d17-915f-ed535933662f-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00002-4ca0c9db-02c8-470e-abb9-0e8a53e1d863-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00003-2bd9d2e4-27a6-4e0c-a6aa-b8c0e810fce8-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00003-7996975c-4199-4491-a27c-4c4e76a1c817-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00003-7b12c177-7cfd-4676-986e-e038051952f2-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00004-0aae8759-c282-47ed-a5c5-6cf37b93f174-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00004-4ca9cc40-3cd3-45b4-a3cc-58de2fec437d-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00004-f11e1f93-e35a-4fac-b3c6-d796621d2cb4-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00005-319662f8-ef0e-41b9-89d8-baa3ab5348f0-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00005-3ae98bc2-a3bd-4f1f-ba76-9a4894572379-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00005-d44a39cb-4364-4a80-9f82-366415d2e69d-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00006-05a437ee-7c40-4534-ae12-25fc7dbc7583-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00006-7acd5504-7d04-4006-93e2-5470e8041767-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00006-9d8b5796-2cc6-48ee-b241-8d3ff6e3d5c5-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00007-5a919d87-b7be-4cce-b9f7-f269423daebd-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00007-68dbff7e-17b6-4c73-969b-977133666c4d-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00007-9f374a9d-37ec-41f6-8006-eb0db94111ee-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00008-25115960-51e0-4f99-acc3-92066e12b06a-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00008-568f62b3-4b06-4b80-81e6-8b745ff8a965-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00008-df9a0d39-3663-4a17-806f-ca720d517645-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00009-658f63ed-06a4-4160-8490-ca4a17b22d06-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00009-93e49bd4-3350-4ab9-bb78-f217a28c8d5a-c000.snappy.parquet create mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00009-da77b766-6ecf-4056-b324-502f75238979-c000.snappy.parquet diff --git a/crates/deltalake-core/src/operations/constraints.rs b/crates/deltalake-core/src/operations/constraints.rs new file mode 100644 index 0000000000..39822bdf0f --- /dev/null +++ b/crates/deltalake-core/src/operations/constraints.rs @@ -0,0 +1,116 @@ +use std::collections::HashMap; +use chrono::Utc; +use datafusion::common::Column; +use datafusion::execution::context::SessionState; +use futures::future::BoxFuture; + +use crate::{DeltaResult, DeltaTableError}; +use crate::DeltaTable; +use crate::kernel::{Action, CommitInfo, Metadata, Protocol}; +use crate::logstore::LogStoreRef; +use crate::operations::datafusion_utils::Expression; +use crate::operations::transaction::commit; +use crate::protocol::DeltaOperation; +use crate::table::state::DeltaTableState; + +pub struct ConstraintBuilder { + snapshot: DeltaTableState, + name: Option, + expr: Option, + log_store: LogStoreRef, + state: Option, +} + +impl ConstraintBuilder { + pub fn new(log_store: LogStoreRef, snapshot: DeltaTableState) -> Self { + Self { + name: None, + expr: None, + snapshot, + log_store, + state: None, + } + } + + pub fn with_constraint, E: Into>(mut self, column: S, expression: E) -> Self { + self.name = Some(column.into()); + self.expr = Some(expression.into()); + self + } + + pub fn with_session_state(mut self, state: SessionState) -> Self { + self.state = Some(state); + self + } +} + +impl std::future::IntoFuture for ConstraintBuilder { + type Output = DeltaResult; + + type IntoFuture = BoxFuture<'static, Self::Output>; + + fn into_future(self) -> Self::IntoFuture { + let mut this = self; + + let fut = async move { + + if this.name.is_none() { + return Err(DeltaTableError::Generic("No name provided".to_string())) + } else if this.expr.is_none() { + return Err(DeltaTableError::Generic("No expression provided".to_string())) + } + + let name = this.name.unwrap().name; + let expr = match this.expr.unwrap() { + Expression::String(s) => s, + Expression::DataFusion(e) => e.to_string() + }; + let mut metadata = this.snapshot.current_metadata().ok_or(DeltaTableError::NoMetadata)?.clone(); + + metadata.configuration.insert(format!("delta.constraints.{}", name), Some(expr.into())); + + let protocol = Protocol { + min_reader_version: if this.snapshot.min_reader_version() > 1 { this.snapshot.min_reader_version() } else { 1 }, + min_writer_version: if this.snapshot.min_reader_version() > 3 { this.snapshot.min_reader_version() } else { 3 }, + reader_features: None, + writer_features: None, + }; + + let operational_parameters = HashMap::from_iter(&[(name, expr)]); + let commit_info = CommitInfo { + timestamp: Some(Utc::now().timestamp_millis()), + user_id: None, + user_name: None, + operation: Some("ADD CONSTRAINT".to_string()), + operation_parameters: Some(operational_parameters), + read_version: None, + isolation_level: None, + is_blind_append: None, + engine_info: None, + info: Default::default(), + }; + + let actions = vec![ + Action::CommitInfo(commit_info), + Action::Metadata(Metadata::try_from(metadata)?), + Action::Protocol(protocol) + ]; + + let operations = DeltaOperation::AddConstraint { + name: name.clone(), + expr: expr.clone() + }; + + let version = commit( + this.log_store.as_ref(), + &actions, + operations, + &this.snapshot, + None + ).await?; + + Ok(DeltaTable::new_with_state(this.log_store, this.snapshot)) + }; + Box::pin(fut) + } +} \ No newline at end of file diff --git a/crates/deltalake-core/src/operations/mod.rs b/crates/deltalake-core/src/operations/mod.rs index a81e16578f..b051c0ed88 100644 --- a/crates/deltalake-core/src/operations/mod.rs +++ b/crates/deltalake-core/src/operations/mod.rs @@ -36,7 +36,10 @@ use arrow::record_batch::RecordBatch; #[cfg(all(feature = "arrow", feature = "parquet"))] use optimize::OptimizeBuilder; use restore::RestoreBuilder; +use crate::operations::constraints::ConstraintBuilder; +#[cfg(feature = "datafusion")] +pub mod constraints; #[cfg(feature = "datafusion")] pub mod delete; #[cfg(feature = "datafusion")] @@ -170,6 +173,12 @@ impl DeltaOps { ) -> MergeBuilder { MergeBuilder::new(self.0.log_store, self.0.state, predicate.into(), source) } + + #[cfg(feature = "datafusion")] + #[must_use] + pub fn add_constraint(self) -> ConstraintBuilder { + ConstraintBuilder::new(self.0.log_store, self.0.state) + } } impl From for DeltaOps { diff --git a/crates/deltalake-core/src/protocol/mod.rs b/crates/deltalake-core/src/protocol/mod.rs index e2add9b529..c7879f41cc 100644 --- a/crates/deltalake-core/src/protocol/mod.rs +++ b/crates/deltalake-core/src/protocol/mod.rs @@ -410,6 +410,12 @@ pub enum DeltaOperation { /// The update predicate predicate: Option, }, + /// Add constraints to a table + AddConstraint { + /// Actual constraints + name: String, + expr: String + }, /// Merge data with a source data with the following predicate #[serde(rename_all = "camelCase")] diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00000-9d9248d4-bd6e-46a8-9805-5b9c1c20d5a4-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00000-9d9248d4-bd6e-46a8-9805-5b9c1c20d5a4-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..4c1142b3631050669ee2f805d75da508f37c80ec GIT binary patch literal 16 XcmYc;N@ieSU}Bgmv-4ln0=5+ZB_IV$ literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00000-9ed8bdb4-315a-421b-8d57-f468582e5fc3-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00000-9ed8bdb4-315a-421b-8d57-f468582e5fc3-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..05122662073f102329fc607217e1f52dfda5223a GIT binary patch literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00000-ca2db31d-ab94-479a-a6dc-697e58993bc6-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00000-ca2db31d-ab94-479a-a6dc-697e58993bc6-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..05122662073f102329fc607217e1f52dfda5223a GIT binary patch literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00000-cace3c04-e21c-4401-9f16-1d38e6c74721-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00000-cace3c04-e21c-4401-9f16-1d38e6c74721-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..27af40b4c433bf5f3bd92890753ad4917f301e4c GIT binary patch literal 16 XcmYc;N@ieSU}D%XCG6f|S*`y7CkO?+ literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00001-53e441ee-68e6-4dde-bb53-d4bea9526242-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00001-53e441ee-68e6-4dde-bb53-d4bea9526242-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..05122662073f102329fc607217e1f52dfda5223a GIT binary patch literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00001-6dea6ffc-5c2d-4b3e-b1f2-7481029003d3-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00001-6dea6ffc-5c2d-4b3e-b1f2-7481029003d3-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..7f46f251fcc586268596a6e4e176bbb338d923a3 GIT binary patch literal 16 XcmYc;N@ieSU}7-dtNtr$0ow`yBKHLJ literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00001-a5330b91-bece-4fce-b2ca-81fc04c911f7-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00001-a5330b91-bece-4fce-b2ca-81fc04c911f7-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..27392c680da38fc3da7ac56cd97e1464f7f3a9f8 GIT binary patch literal 16 XcmYc;N@ieSU}A{BWPUSh0ow`yB&7u; literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00002-04434872-6b96-4307-b110-bd15ea28efa4-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00002-04434872-6b96-4307-b110-bd15ea28efa4-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..05122662073f102329fc607217e1f52dfda5223a GIT binary patch literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00002-24d4d181-0877-4d17-915f-ed535933662f-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00002-24d4d181-0877-4d17-915f-ed535933662f-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..c8834849406476f3bfcff334bad3a999575a5666 GIT binary patch literal 16 XcmYc;N@ieSU}E@jQ-F0*eP|^BCAj?fNcc;A|?dU literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00005-319662f8-ef0e-41b9-89d8-baa3ab5348f0-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00005-319662f8-ef0e-41b9-89d8-baa3ab5348f0-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..05122662073f102329fc607217e1f52dfda5223a GIT binary patch literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00005-3ae98bc2-a3bd-4f1f-ba76-9a4894572379-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00005-3ae98bc2-a3bd-4f1f-ba76-9a4894572379-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..d1848d96d4ee8c03642054b7e95b81dae3dacf3b GIT binary patch literal 16 XcmYc;N@ieSU}7kHSCJRBfNcc;C6EOz literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00005-d44a39cb-4364-4a80-9f82-366415d2e69d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00005-d44a39cb-4364-4a80-9f82-366415d2e69d-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..704cd3fc0035eda27dd726874c4fc7ce16173a66 GIT binary patch literal 16 XcmYc;N@ieSU}E^~`0iQM0=5+ZDj5ay literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00006-05a437ee-7c40-4534-ae12-25fc7dbc7583-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00006-05a437ee-7c40-4534-ae12-25fc7dbc7583-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..b1d07c79a34c18788645af60238c38c758aa5c5d GIT binary patch literal 16 XcmYc;N@ieSU}9kBsao;vVGs`hAV&nH literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00006-7acd5504-7d04-4006-93e2-5470e8041767-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00006-7acd5504-7d04-4006-93e2-5470e8041767-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..6e09f75f6871409fdde7ce5e51c788e9662d6a7e GIT binary patch literal 16 XcmYc;N@ieSU}CtUA(j`lfNcc;Aj$+| literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00006-9d8b5796-2cc6-48ee-b241-8d3ff6e3d5c5-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00006-9d8b5796-2cc6-48ee-b241-8d3ff6e3d5c5-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..05122662073f102329fc607217e1f52dfda5223a GIT binary patch literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00007-5a919d87-b7be-4cce-b9f7-f269423daebd-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00007-5a919d87-b7be-4cce-b9f7-f269423daebd-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..05122662073f102329fc607217e1f52dfda5223a GIT binary patch literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00007-68dbff7e-17b6-4c73-969b-977133666c4d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00007-68dbff7e-17b6-4c73-969b-977133666c4d-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..b69b6ede7d062310b75c24e6b8414ebb2b94e6ad GIT binary patch literal 16 XcmYc;N@ieSU}E^Ipt&Y$0ow`yBZ36g literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00007-9f374a9d-37ec-41f6-8006-eb0db94111ee-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00007-9f374a9d-37ec-41f6-8006-eb0db94111ee-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..752fca483df976fee2175565da07ceeb6bca543c GIT binary patch literal 16 XcmYc;N@ieSU}BhXN@qsY0=5+ZByI%( literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00008-25115960-51e0-4f99-acc3-92066e12b06a-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00008-25115960-51e0-4f99-acc3-92066e12b06a-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..f322948501a793370d7f48a77ba4be2c6375e983 GIT binary patch literal 16 XcmYc;N@ieSU}D&E;m4n-1#Bw-EOZ82 literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00008-568f62b3-4b06-4b80-81e6-8b745ff8a965-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00008-568f62b3-4b06-4b80-81e6-8b745ff8a965-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..d3580b086dec5180856d8627a585b300ad09f66b GIT binary patch literal 16 XcmYc;N@ieSU}D&G+2BXi0=5+ZC$$BT literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00008-df9a0d39-3663-4a17-806f-ca720d517645-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00008-df9a0d39-3663-4a17-806f-ca720d517645-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..05122662073f102329fc607217e1f52dfda5223a GIT binary patch literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00009-658f63ed-06a4-4160-8490-ca4a17b22d06-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00009-658f63ed-06a4-4160-8490-ca4a17b22d06-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..05122662073f102329fc607217e1f52dfda5223a GIT binary patch literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00009-93e49bd4-3350-4ab9-bb78-f217a28c8d5a-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00009-93e49bd4-3350-4ab9-bb78-f217a28c8d5a-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..a5ec277a825f6f75afc6ce738b863959cbb620d1 GIT binary patch literal 16 XcmYc;N@ieSU}A_@`*JL50ow`yBme~> literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00009-da77b766-6ecf-4056-b324-502f75238979-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00009-da77b766-6ecf-4056-b324-502f75238979-c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..67d82325b997a2b6ec29fce93dcbc23089d4cc03 GIT binary patch literal 16 XcmYc;N@ieSU}6ZCkq(bqz_tPa92o<| literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000000.json.crc b/crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000000.json.crc new file mode 100644 index 0000000000000000000000000000000000000000..77f82b91e3347746997f7b2a3e293ea62c354ad6 GIT binary patch literal 36 scmYc;N@ieSU}AVCJAH}#;TY4~(P_(_&Lzb?C|tCg=h{5~qf^+}0QguA;Q#;t literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000001.json.crc b/crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000001.json.crc new file mode 100644 index 0000000000000000000000000000000000000000..8c18cbf62c53dad4c0715ead105d689682c82865 GIT binary patch literal 16 XcmYc;N@ieSU}D(&*v8VSy;27NB*_H6 literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000002.json.crc b/crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000002.json.crc new file mode 100644 index 0000000000000000000000000000000000000000..29f5b4b28efe439716b4eee6110ef256bec498c6 GIT binary patch literal 36 scmYc;N@ieSU}AXQrE<)eDOB>vsxuxX$>&xUomnIQ*ZrE>Kb;v|0P|T79smFU literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000000.json new file mode 100644 index 0000000000..3fa9ceef08 --- /dev/null +++ b/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000000.json @@ -0,0 +1,13 @@ +{"commitInfo":{"timestamp":1700616581733,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"10","numOutputRows":"100","numOutputBytes":"5282"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.0.0","txnId":"9fc06c03-2630-4160-9baf-a9f9f8547fb4"}} +{"metaData":{"id":"2d9d5c40-6358-4aa3-a9eb-94ba012bdef4","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1700616580070}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"add":{"path":"part-00000-cace3c04-e21c-4401-9f16-1d38e6c74721-c000.snappy.parquet","partitionValues":{},"size":539,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":0},\"maxValues\":{\"id\":9},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00001-6dea6ffc-5c2d-4b3e-b1f2-7481029003d3-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":10},\"maxValues\":{\"id\":19},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-4ca0c9db-02c8-470e-abb9-0e8a53e1d863-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":20},\"maxValues\":{\"id\":29},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-2bd9d2e4-27a6-4e0c-a6aa-b8c0e810fce8-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":30},\"maxValues\":{\"id\":39},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-f11e1f93-e35a-4fac-b3c6-d796621d2cb4-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":40},\"maxValues\":{\"id\":49},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-d44a39cb-4364-4a80-9f82-366415d2e69d-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":50},\"maxValues\":{\"id\":59},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00006-7acd5504-7d04-4006-93e2-5470e8041767-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":60},\"maxValues\":{\"id\":69},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-68dbff7e-17b6-4c73-969b-977133666c4d-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":70},\"maxValues\":{\"id\":79},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-568f62b3-4b06-4b80-81e6-8b745ff8a965-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581638,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":80},\"maxValues\":{\"id\":89},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-93e49bd4-3350-4ab9-bb78-f217a28c8d5a-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":90},\"maxValues\":{\"id\":99},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000001.json new file mode 100644 index 0000000000..1eba75269f --- /dev/null +++ b/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000001.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1700618848815,"operation":"ADD CONSTRAINT","operationParameters":{"name":"id","expr":"id > - 1"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.0.0","txnId":"43d64f90-fed8-42a6-a62e-45c640c76007"}} +{"metaData":{"id":"2d9d5c40-6358-4aa3-a9eb-94ba012bdef4","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.constraints.id":"id > - 1"},"createdTime":1700616580070}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":3}} diff --git a/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000002.json b/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000002.json new file mode 100644 index 0000000000..4eaa6d3494 --- /dev/null +++ b/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000002.json @@ -0,0 +1,11 @@ +{"commitInfo":{"timestamp":1700619387829,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"10","numOutputRows":"100","numOutputBytes":"5270"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.0.0","txnId":"10b320e3-6283-4d4a-8e86-fb6548ed5a76"}} +{"add":{"path":"part-00000-9d9248d4-bd6e-46a8-9805-5b9c1c20d5a4-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":100},\"maxValues\":{\"id\":109},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00001-a5330b91-bece-4fce-b2ca-81fc04c911f7-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":110},\"maxValues\":{\"id\":119},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00002-24d4d181-0877-4d17-915f-ed535933662f-c000.snappy.parquet","partitionValues":{},"size":526,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":120},\"maxValues\":{\"id\":129},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00003-7b12c177-7cfd-4676-986e-e038051952f2-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":130},\"maxValues\":{\"id\":139},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00004-0aae8759-c282-47ed-a5c5-6cf37b93f174-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":140},\"maxValues\":{\"id\":149},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00005-3ae98bc2-a3bd-4f1f-ba76-9a4894572379-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":150},\"maxValues\":{\"id\":159},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00006-05a437ee-7c40-4534-ae12-25fc7dbc7583-c000.snappy.parquet","partitionValues":{},"size":528,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":160},\"maxValues\":{\"id\":169},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00007-9f374a9d-37ec-41f6-8006-eb0db94111ee-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":170},\"maxValues\":{\"id\":179},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00008-25115960-51e0-4f99-acc3-92066e12b06a-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":180},\"maxValues\":{\"id\":189},\"nullCount\":{\"id\":0}}"}} +{"add":{"path":"part-00009-da77b766-6ecf-4056-b324-502f75238979-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":190},\"maxValues\":{\"id\":199},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00000-9d9248d4-bd6e-46a8-9805-5b9c1c20d5a4-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00000-9d9248d4-bd6e-46a8-9805-5b9c1c20d5a4-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..827910fefe8cc307cdaa4178bdddee911484b409 GIT binary patch literal 527 zcmZWnO-lnY5S?ABM$m(>2?^|>1S|?Q?6R%aB7!&ZBp$qoNVbV}b+;e8TSZFGUi<|f zJcvhslE1)T;Z$3DFz`s`&1BvilGCHJkOKX1Xh~n+UY?0hJ&czr0XX+v0Pq|Gs}9)J zl?q(16X+st5DDTIakowmQmmV<>~{mP;R;U7JmO@Ypsrv5!e@lOJ9q*l>c?xvK<9T? zMd!w5nz}$Mn=W?~1I!b?@M~zQBAP&!#gwHk^Vp}u-klmbShQZ~G1&SOT(GE$gP|Y* z`*J8_IP)P)8+kQ?%{=r~=e(bHl^$rcqJCVddz%+YzsHv%)l>oS?SB=;FjY9faAZrB z_a9VdOKo(i`T1Cuj{_^)vMBRWr<8Sho!HbAvVAd@J)KQci9vc+miBvHjC!Lq)p3eh zyk_fi-cYv3lgZ@-_{mIeC(Eldf7MWhkqWC@@hC}6)x?dk6@|gRX~ug|!*-hcQD9no bLC0tt?uNDzw2X=Mw(xy?h9?K$6u-hZ$bfNi literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00000-9ed8bdb4-315a-421b-8d57-f468582e5fc3-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00000-9ed8bdb4-315a-421b-8d57-f468582e5fc3-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..364e9aaa2c2e5d971890eaf96978e93286b38dd7 GIT binary patch literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00000-ca2db31d-ab94-479a-a6dc-697e58993bc6-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00000-ca2db31d-ab94-479a-a6dc-697e58993bc6-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..364e9aaa2c2e5d971890eaf96978e93286b38dd7 GIT binary patch literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00000-cace3c04-e21c-4401-9f16-1d38e6c74721-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00000-cace3c04-e21c-4401-9f16-1d38e6c74721-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d0b59376d6570030bbc4657a3c364738121d4a9e GIT binary patch literal 539 zcmZWn%}OId5Uw6G*%7h_VY=ytIfRA{5n?mLB<6=j@FshbJ$MmWdeW^9PJW!3?6QPh zyvXib>{0L$Jo*B@fX}iOlbC~jsp{{ms_!ehPmjMx6zC(Ne+hl~_wvULsEuT zMRA)cT;O(QOO;Q4sgW(U(WT~Rb6NhLS~-wKna_r$Y{|>iW~PvX^SK=B(UM9H!t1hZ zGQPswc$Q^4$&kf1+mf4saxh;kE*8MQt>q3zd427#9;ql+QN5YW(#+IN(vEs@6!uLg l>Ben4?DXT%^t$2DXd4|ywjK72N%X!50(^#h0&t4o;y+B=Yj^+v literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00001-53e441ee-68e6-4dde-bb53-d4bea9526242-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00001-53e441ee-68e6-4dde-bb53-d4bea9526242-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..364e9aaa2c2e5d971890eaf96978e93286b38dd7 GIT binary patch literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00001-6dea6ffc-5c2d-4b3e-b1f2-7481029003d3-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00001-6dea6ffc-5c2d-4b3e-b1f2-7481029003d3-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..bf4e92f1916414c296fe007cf4d71c6ca15f39f9 GIT binary patch literal 527 zcmZWnO-m~=6umQ4jURn}O+o^*C;^K?4KqxuwTR$OT!{-ekB4NMJm28-!}z`Pi<)MXxfb=Z?rC5JQVnID7g|AsRb zU2!lp1YqwDWejINg0YZSB`D@$Up3Bq*_G0LjjgD!uGFo~@}$@0bCGJO0Qk<2iu@*3 zIKa)o7AouAsg5nQ(S_z0Ls{JSt!&A>$Oi2~*5u#BrY4uI-$U8eopCBLNKeYrUiS*4 z?jTKdoMINQ*_xaR%GPi+`ZEH4IMe3S!Mgdn|J*=qyMmK!8F8{qP**Sj;WI*C96SLM?bDy- zNEdfk<>1y1HFbeDc3kcz2AC&&;kSrS8=?zjUCdePGLOAE?8Rx3wHL*pR<5b}Q zT?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00002-24d4d181-0877-4d17-915f-ed535933662f-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00002-24d4d181-0877-4d17-915f-ed535933662f-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..12940c44e228f5c5e4d90fec3d7d19e967c36c1b GIT binary patch literal 526 zcmZWn&r2gQ6n+`18$l1kBqT7060j)LFvGNKEh2bZ_9Py>h)AZ1I_&h<%yd`nR^8M7 zA0EZy{x$m-_%C?#Ra<*7@R7Xly?oz$B!>q_5d-!_SjnFLz1#%M$8ecZfaAae0H0jJ z3W4jDFBQ01qcBGNMobX5h`TkqcaL?`_2+#9x$a3pujY}U^91uG2ao}$?1A73kf?uK zEe0n4aaMFLUDq%VSY^W#UTT5+GLS)y2CO2RK$qo|WghqWf5IO~jqWX^7k(TzKMWTv zy5eAH2*95SM*FxZNI>q`H0d6D*eVky>59fH{UR8b5w zjROovuGD$|m+rdKSW_BtJXYoPz^S$>%6!x*Rb8E@F0+MdpNv({bf>A*B0Ve1`n?N` zdZR2eNrqXx=IUxLsM_Pn36Gi9cx_lHFAxxWo=@%WDwvp{3QT~_!C|MlM!xN)_ z&bglu$T6Rk4@mMAnN98Z5Wz#5H=}uPkj}n*3n|b$hnDo=-_0ZOsfTf$5`c5x1pv?S zpy2>*ysH7*Bk%#i5d!fMvA;)-eXOh2!T$}!dslGcZ6Z!K3F-<4AbdvXt%D~(qWg8X znCtw{&tm-BPBnFb);_u1O$;zk_`+|Ysha2$sEdlFF7w#E!)~1xIo`J3>M>})3T|0c z#lcVzfIT^sF`W4XszzRmU^8?px}4AQiPCe8R@93t_1ES_GRyc*r1~lV-g#3|T%-yI zxLDXy<+DpQwWT(?)ckxY%d5GSBUzOBVqD6u{FT_$6ms-^DKkB-Qi(x&U6#(W2}ao> zO?8}N7O&Z^+%%M<cAARa!yiN%~*LMNH zbGBjAft}5b3ItmOJ|iUJ3*swccZ(cVv2MP;{o6o%as?;b%ZQU@g1UkM2%ix;bMOR6 z)Q&$CBb{HJ7rjf{*VF}C*>$;_8epFAg;$Ub9z+N267|#3y=0aYTpqPh!)i@vK7fO#bwxZs;Qh#h-q{9JUi&aB~!1v!( z6xW%;0j|fkRQd2m^=+w*E;a8?WO+NXvMq};ANNXGlfP4&nL@UIP2@oL=c&{ny(r6u zgA0rX<1EujhFQF3YjPtWNFMDGjV$7i^606O>;UI8}pZEpYo literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00003-7996975c-4199-4491-a27c-4c4e76a1c817-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00003-7996975c-4199-4491-a27c-4c4e76a1c817-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..364e9aaa2c2e5d971890eaf96978e93286b38dd7 GIT binary patch literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00003-7b12c177-7cfd-4676-986e-e038051952f2-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00003-7b12c177-7cfd-4676-986e-e038051952f2-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..dffb701f25eeb8869d0e45e53852f1b2782282f4 GIT binary patch literal 527 zcmZWnO-m~=6umRlz6iQ_FbN6Fq691oePM=awH6WF$yi?~DFBOVsX#v|5^*VFd~;-@P(xtvCvOcT@<3_$pd&}RovfJAM3 zA@A$#&tcv^u^mlaprs|3yNLnj319eC;?t6704a+xOI_x%H-|ktRkAUYp7}9Y{%1I2 z(G>?nLjd;SP{wfPO&AM#Rf1w3_EqD&mmMqJ*Vu~s>Pns2EKhn}J{PHm3V^TtSCOBl z3I{kH*g|E!bJekhHoDM!Zzzk4zLhPR7uld)$eKJ#Y-)1Z+8@fU?u=83L3&b__PWOy zbq8sx;}o-a&DP{pP_~An(ZLA#?}^>7PF9)pR|!=Zsj#vf50cbWOk59}Q5dY7M!Xi) jZM(4^1*W+cw2ijmYG~_0)0kMV2;awNxN`t@@GE=*>`imh literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00004-0aae8759-c282-47ed-a5c5-6cf37b93f174-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00004-0aae8759-c282-47ed-a5c5-6cf37b93f174-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..71b259cb03b741326327eff6dbaa8b24fe7130d4 GIT binary patch literal 527 zcmZWn%}OId5Uv@b+oB$X>82az5E?cp#Ab#`GzmfQCZ1#uUY2F)Nw+%e!$1B{{~{i6`V}w5hwEmbp-JI{WrD@0{7LrY_LZN0+;a0p zW3c&VxM0y02SY;u_Q#=&;mmt574j+s#XRh*%6UIKReGSY74@$x_1$K9((m!5NYzyU zeCu6Bex52E;CyHcmGyt9t}V3Dh33a2SzHXPY{|UHhMhuIcZyMOn5H^TF^ku1Rn7%vYcw8z83R9@*=={T@~ppdsKQ8v<;{4Qq^4}*TG)uf pplRyyZd9|KdNT@4V>jp+ZNr_=)`EsHvECBCkI!)F036|0_zTT5bX@=d literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00004-4ca9cc40-3cd3-45b4-a3cc-58de2fec437d-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00004-4ca9cc40-3cd3-45b4-a3cc-58de2fec437d-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..364e9aaa2c2e5d971890eaf96978e93286b38dd7 GIT binary patch literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00004-f11e1f93-e35a-4fac-b3c6-d796621d2cb4-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00004-f11e1f93-e35a-4fac-b3c6-d796621d2cb4-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..c1c2ba1ed6b1c68074e21d4c272f28e34bba8e16 GIT binary patch literal 527 zcmZWn%}yIJ5T0EE7^xhP)|M^pA+ppcX&1BBCJ-W2skf>p#DPnNkT*84V)M7VX@#Pk zc>>;m0|#D(K0%*=r(i;p=1@zY?D=N=eKWF8zn*Cd^x2~oeR%o(v=Pz(<1I=6&O;vn z09<+A0Yw&58z-L5+*heVD!7e$8v2L;S-wni`FF0wgB2HEb>I()SLPqGLhbKTH z`u3q5o8tDe?BBS7p+3;sN1ywd1r~@idtGlt2Jy}-8v|q_cUS%$~rR@Ef$)Oo6Qkg}1S(cB6 z*BA|_d2Z4ivv|!#awRBxv-$jD4*dJlZhuhJm;KeX(uvabW;)GsTeoRjcM}~S+fI6v jv|Yb*oW!T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00005-3ae98bc2-a3bd-4f1f-ba76-9a4894572379-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00005-3ae98bc2-a3bd-4f1f-ba76-9a4894572379-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..ddcddb6fb10a7abbde9e5fd0189d43556762e716 GIT binary patch literal 527 zcmZWnO-my|5Um-awx|bTy6J{Fga(5`wwYlPjX@T?i6`;kMP%to_v+x}Yi6P>A?N)E z>hAh+@#s&IWzYE)Rx~k(ZFqFotLpcv=swy%4k^$VhZgkjc>Hh0ryjx5F1ARaXJt3t2Ews^v<|jj0{ODQPlzEX2T7|62?}<%KE}N%A+0pGuDlte;%hGP= z9HY)4O?8}N7O&Z=oC(V2a5Op_0Y9ADeQ9UqX@BKVg^>!&oADq?P1(e?un~p9uBpd6 kQO&mMyHQ{oJ3-568*YcT7Bq~B^_K8`e1@M6z!83hH($MVzyJUM literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00005-d44a39cb-4364-4a80-9f82-366415d2e69d-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00005-d44a39cb-4364-4a80-9f82-366415d2e69d-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..540bb969cf704bc6290ba3dbe6084b1e117f3fe2 GIT binary patch literal 527 zcmZWnO-my|5Um-qNeg;dm~OgZ4xwR#LTqL-8S|kmcw0P)2QMN^Pr8kRlMiPmE=$N& zPoDe>f){_1zhHlb6-~@x8y?;Ds`|Yux=)YKLJIWNp(TBMx__zp)Wf(+3BbAU0)XeV zV9SAMt5$*b4uKePfaoB)h~5r4>SNt<9e!>gzPo~x);i*3ouIB@0K#X4K09~c0UeiwslJJQqzTKVB}H#NXK;S0Y(d|DAPfvQ-t)MXxfbJ(-fAV(YNjUR*EFT)Ls zt~eMP0cbK~T)YzM7m*@}bgGjjgDUuGF2)i*z#P-y#*O0Qla26~%3) zaDdyHEmc1GqeiyWMwgnO&t>^{YGq#*Wj-5}vMH}po0&rPFXnQrN6S=dkY1H#lkpIv z@hr=9l3^CF*`{0z%Km(@xLg1~S=sH6^7^X3dZ@xkh4tNJmS(1Il6KgQ!k}m3q!YF6 hAnrwh>2`vF(Kb8?Z9C{1ljuF+`}hnG4!|jXg?C)BZZ`k` literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00006-05a437ee-7c40-4534-ae12-25fc7dbc7583-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00006-05a437ee-7c40-4534-ae12-25fc7dbc7583-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..6aa24c5c22c5d4a3b77d6449eed29fe851f6f66e GIT binary patch literal 528 zcmZWnO=}x55FIUUqCit9tb_!6FoKu_T(QgAiDN<`x6)I{A(s+@)>6|Y>#yB)N-@3_ z`Ztoabs7l$Jw4=_TWA zJOJ>?1*{RcTwATc)jEam7(Xz6VocX*+&UsyO>H%}YL*PaygVvz~DAebjPfDAZg zQ-UWzp#JVaH_3}G9PtHRaYNVm)SzKKa5q+bZ1m* z5uTT2{oW~Ny-}8#BtsUjxw={is`hv?Ii7%cKev0+&8zeNs*#Rj9aT4zQJUGRO&U=v qj>28rOm^ai>oj-c(6)BMjx{dYj$9*bS(}(G83gzY-w40~{)NB9a&`&; literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00006-7acd5504-7d04-4006-93e2-5470e8041767-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00006-7acd5504-7d04-4006-93e2-5470e8041767-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..3d88c4aa21420ae94ea6e8e877e942b9f41e4fc7 GIT binary patch literal 527 zcmZWn!D<^Z5S?9|R3Xiwuo4pN!3bhvaK$cb$BuE+CTXDb6mrNVl%lm%cj>Oz-d(2@ z<8yx?pO9nzp+C|u$f5KTGEVIJP{Bi*H>3AvkS$W$0qi*NX6Ho>j`s+>MZ80tAWjiydt~q)>z3>Mc?0pv6`Ty#5hv>ebp-kE!ih0;qi}Sn~DLv8HihAlw-Poc`@-g3vR7VBC zoBvgm(^TOA)0wSQk$+d2t+dgV=2vrB{g_zUmt|GVhLvo|FNsY}Df`!RIo8=Sl^CQ~ zWobShVKknlsg6_3;x*fnYeCtcFBYE`z(20+US~yp)n7eSVWh(PK|D)RQ#Wxt>_uU4 oWIAy-YTIGwC<;uk8w`!M;bCaoLC=_2H-+!xGyHS_F7PWn0yyGtzW@LL literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00006-9d8b5796-2cc6-48ee-b241-8d3ff6e3d5c5-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00006-9d8b5796-2cc6-48ee-b241-8d3ff6e3d5c5-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..364e9aaa2c2e5d971890eaf96978e93286b38dd7 GIT binary patch literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00007-5a919d87-b7be-4cce-b9f7-f269423daebd-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00007-5a919d87-b7be-4cce-b9f7-f269423daebd-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..364e9aaa2c2e5d971890eaf96978e93286b38dd7 GIT binary patch literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00007-68dbff7e-17b6-4c73-969b-977133666c4d-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00007-68dbff7e-17b6-4c73-969b-977133666c4d-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..5ac21aaf0422e64976f45689ec73271e93f0fbf2 GIT binary patch literal 527 zcmZWn&1xGl5T0EeM=r^suo4pN!3bg!aK$cbH?}c_Ufle(4L$f$LeN@DvSj_YyKX7Q z$I=JrEBM$)$`gb>LJu7`c6_MdLz-_!-#3GF`Sw#tf&Ox6L4W`G{_}-TJ&eng0Q7wq z06eD$6$jo`c1v)wPv8`BhB!xDAl~nj?gy+}t&dL|h?lP5q_>GU*(9hd7=Z8@p-Tr( zfJFUdFQ4e_{yHDr*`cN`(9(g+-NXR%gfILW@o7mk36#Z(r7rW>gTt0ijdZutTR#Se z&xTtTU2!lp1YnO2WejJ211lk~Mo`SdzUrKhvs8?kmM7y8--%RH1;CH~ ztH`IR!U3i;Tc~XORSj*SjV?6r&t>s#Vr56>MK&80vMz5Do0?p9KF{Sy4_B$gAiXY2 z$D><}Mzb{4af(^IX6tezC_D4T;%Wi>Y;E^ym{r&PRYMgifU|Pq)z-Sx34s9c7858Rx;rsXuKOBHd{0e^o;Ms63 literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00007-9f374a9d-37ec-41f6-8006-eb0db94111ee-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00007-9f374a9d-37ec-41f6-8006-eb0db94111ee-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..50479539bee2c237b928cebc57a9c051102ff0c9 GIT binary patch literal 527 zcmZWnO-m~=6umQ4jqqFulaRnHO5hcR8fKVQYZ1YnxRSSUBl5^JdA`Bvhci<}O4t4o zSAKu&{2{LV1uh@{0B?P*T^Kkd_uQLv?j_mX`V~^3*A6Y{%fILMl21L1XDI>L^IZV& zoJ*K@;A(!Z1lJ1$ZV&Rc(NTT>Tk>8H!x!~pYzFZ>GeX-U+9%!(09UFNY@hg~=ovN4sO`Y~Ag zGMuvLii4pc0DE&NV>t6BjD)-jK`{^es&d}V4wdd{Y(;%`rA};?C*2O8iBw$$z?Z*O zwQZq|E;Qd8$l|PLWmD!w)^8QED*q%lHMwl=4`fHTN2$ahJuXYT zokNT|{WR5aidnp7t8yYJn}gx-U1;GLoOi{t)-?*)?d5pgkXHm zAIULC=c7&fPx1?T@3|c(c6^B7A281GnY%65MSNxJNu7ejzVNHWrzOze)gYU1)wZk;UcE%9hNFY}_qmO@2>oYI4~+p2)uL%~Od%dQp}R z`llH6$7!nL6tj5E*5p!9wkFf*$rSjPh23s1t1SAfger_wSlN!pNop!4u7}Mi3=T{q m-jC|G+c<~<)7%fbM%!>NwDq8AOssc=@8dIEIRG8}3jYBB4R>Gw literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00008-568f62b3-4b06-4b80-81e6-8b745ff8a965-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00008-568f62b3-4b06-4b80-81e6-8b745ff8a965-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d1f5d89f7ff0a56ff3d1e594527d2e6ee9e36296 GIT binary patch literal 527 zcmZWnO=}x55S?9|M27lMSP2RCU<5G4jh-aN^nvkaEdrXd_nXP=M~buz`Diy`nG}i=n77HtB8|Tg1UkM2%i!9k$_KZ0sHqFIwC!>?F~B_G3%^EuS`tkHn_|IImwD{jVNXttbl1{rKL$JR zhHDmGaWFIlV1FFS7|whE3n8yYP|U-=>YR_W8>J^2TT%bIQr~TsC*u*{h*VPrz<2+v z$nR5y1Kdw-p|bH0HME5`y3o8olf}ct%8ty7Y&s}pU0x?PHM#73o5_(LE>ej>dRdl^ zM>iOarfI6<6tj5E*5yi2c4qVWT?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00009-658f63ed-06a4-4160-8490-ca4a17b22d06-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00009-658f63ed-06a4-4160-8490-ca4a17b22d06-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..364e9aaa2c2e5d971890eaf96978e93286b38dd7 GIT binary patch literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00009-93e49bd4-3350-4ab9-bb78-f217a28c8d5a-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00009-93e49bd4-3350-4ab9-bb78-f217a28c8d5a-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..2ea76d747fda72d4a4947ff22639e02b6042191c GIT binary patch literal 527 zcmZWn%W4}j6umPzsRH$)FcK2X!U$q)aKsE_*AGG=yU>;CB8xnlXhw=Nbo|K7xTP4c zv&^<1;AK7|KT+rx^eb}R+VLWSgLKcmI_F-b;mMbf0)295NuPiHeysS^!+4((fHU6( z0M9vx0|zb+-c{hTO5h6d6>*KYLFg(uje7HO;A@b0O2!2R}P*4 ziPq`+Vy5$N*F|z?$C|o8D~B$3Qv=KszVPeBrxnp5urJmub(zQhIBezA$?;Bl=f|M& zYPe(36$e8@0QT3RjN#0mU@hd;35t2xSBvv$K2mz7u@&{NEA`#xMLM1Ey-0Ob0KEC8 zqPWi#4sbuWrOKy2)Yz8V=u-2ug)D#0tQ^Rq%;!ldTk5^gZNrbDZ3jJLV%-$JkI(Sn01WXfyZ~|`aVr1- literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00009-da77b766-6ecf-4056-b324-502f75238979-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00009-da77b766-6ecf-4056-b324-502f75238979-c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..df84f4dd708da2fb5db60c7992e0341d83de64b2 GIT binary patch literal 527 zcmZWn&1xGl5T0F}REGEvSP2RCun1yeaKtWaH?}c_-bzoVhg?c1T1!cmu79$-ZYjp+ zzKD-~gyR0UPtaSRpbrr`o7(lEf)8oF8GYXj(&+KChys1*(31YVdw+M}QxD@qN&qf= z7XUnG19b;J)F0H~V}rma#An18#8D+f3mAw+maUeD+#RY^jYdHNRNO^3B}Jfh@{=F)n3WUZplOg&aIz%Bh}Isnj67 zEz4%pYmBCgEYnGbS-fW3awjMU%hl>~1^nsO?rf3=+x~(`MX`#4(`1omCNN1S>cvsm oH{Iks?$~j+ABU!Q9*&K+(Zk4g!k#gSZVBJVXL#!XjPNV`0d@X(H2?qr literal 0 HcmV?d00001 diff --git a/crates/deltalake-core/tests/read_delta_log_test.rs b/crates/deltalake-core/tests/read_delta_log_test.rs index 248f25c876..22d09c246e 100644 --- a/crates/deltalake-core/tests/read_delta_log_test.rs +++ b/crates/deltalake-core/tests/read_delta_log_test.rs @@ -2,6 +2,9 @@ use deltalake_core::{DeltaResult, DeltaTableBuilder}; use pretty_assertions::assert_eq; use std::collections::HashMap; use std::time::SystemTime; +use arrow_array::RecordBatch; +use datafusion_physical_expr::expressions::Column; +use deltalake_core::operations::collect_sendable_stream; #[allow(dead_code)] mod fs_common; @@ -27,7 +30,7 @@ async fn test_log_buffering() { location.clone(), deltalake_core::storage::config::StorageOptions::from(HashMap::new()), ) - .unwrap(), + .unwrap(), ); let mut seq_version = 0; @@ -172,3 +175,14 @@ async fn read_delta_table_from_dlt() { assert_eq!(table.version(), 1); assert!(table.schema().is_some()); } + +#[tokio::test] +async fn read_delta_table_with_check_constraints() -> DeltaResult<()> { + let table = deltalake_core::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; + // + // let (_table, stream) = table.load().await?; + // let data: Vec = collect_sendable_stream(stream).await?; + let constraint = table.add_constraint().with_constraint(Column::new("id", 0), ""); + constraint.await?; + Ok(()) +} \ No newline at end of file From 34a3180144859cfc3920c296d4bdda8e4e76ca27 Mon Sep 17 00:00:00 2001 From: Stephen Carman Date: Sun, 26 Nov 2023 11:58:09 -0500 Subject: [PATCH 02/13] feat: check constraints --- .../src/operations/constraints.rs | 68 +++++++++++-------- crates/deltalake-core/src/operations/mod.rs | 5 +- crates/deltalake-core/src/protocol/mod.rs | 8 ++- crates/deltalake-core/src/table/mod.rs | 52 ++++++++++++++ .../tests/read_delta_log_test.rs | 14 ++-- 5 files changed, 109 insertions(+), 38 deletions(-) diff --git a/crates/deltalake-core/src/operations/constraints.rs b/crates/deltalake-core/src/operations/constraints.rs index 39822bdf0f..3f5a7318e6 100644 --- a/crates/deltalake-core/src/operations/constraints.rs +++ b/crates/deltalake-core/src/operations/constraints.rs @@ -1,21 +1,21 @@ use std::collections::HashMap; + use chrono::Utc; -use datafusion::common::Column; use datafusion::execution::context::SessionState; use futures::future::BoxFuture; -use crate::{DeltaResult, DeltaTableError}; -use crate::DeltaTable; use crate::kernel::{Action, CommitInfo, Metadata, Protocol}; use crate::logstore::LogStoreRef; use crate::operations::datafusion_utils::Expression; use crate::operations::transaction::commit; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; +use crate::DeltaTable; +use crate::{DeltaResult, DeltaTableError}; pub struct ConstraintBuilder { snapshot: DeltaTableState, - name: Option, + name: Option, expr: Option, log_store: LogStoreRef, state: Option, @@ -32,7 +32,11 @@ impl ConstraintBuilder { } } - pub fn with_constraint, E: Into>(mut self, column: S, expression: E) -> Self { + pub fn with_constraint, E: Into>( + mut self, + column: S, + expression: E, + ) -> Self { self.name = Some(column.into()); self.expr = Some(expression.into()); self @@ -53,25 +57,40 @@ impl std::future::IntoFuture for ConstraintBuilder { let mut this = self; let fut = async move { - if this.name.is_none() { - return Err(DeltaTableError::Generic("No name provided".to_string())) + return Err(DeltaTableError::Generic("No name provided".to_string())); } else if this.expr.is_none() { - return Err(DeltaTableError::Generic("No expression provided".to_string())) + return Err(DeltaTableError::Generic( + "No expression provided".to_string(), + )); } - let name = this.name.unwrap().name; + let name = this.name.unwrap(); let expr = match this.expr.unwrap() { Expression::String(s) => s, - Expression::DataFusion(e) => e.to_string() + Expression::DataFusion(e) => e.to_string(), }; - let mut metadata = this.snapshot.current_metadata().ok_or(DeltaTableError::NoMetadata)?.clone(); + let mut metadata = this + .snapshot + .current_metadata() + .ok_or(DeltaTableError::NoMetadata)? + .clone(); - metadata.configuration.insert(format!("delta.constraints.{}", name), Some(expr.into())); + metadata + .configuration + .insert(format!("delta.constraints.{}", name), Some(expr.into())); let protocol = Protocol { - min_reader_version: if this.snapshot.min_reader_version() > 1 { this.snapshot.min_reader_version() } else { 1 }, - min_writer_version: if this.snapshot.min_reader_version() > 3 { this.snapshot.min_reader_version() } else { 3 }, + min_reader_version: if this.snapshot.min_reader_version() > 1 { + this.snapshot.min_reader_version() + } else { + 1 + }, + min_writer_version: if this.snapshot.min_reader_version() > 3 { + this.snapshot.min_reader_version() + } else { + 3 + }, reader_features: None, writer_features: None, }; @@ -79,26 +98,20 @@ impl std::future::IntoFuture for ConstraintBuilder { let operational_parameters = HashMap::from_iter(&[(name, expr)]); let commit_info = CommitInfo { timestamp: Some(Utc::now().timestamp_millis()), - user_id: None, - user_name: None, operation: Some("ADD CONSTRAINT".to_string()), operation_parameters: Some(operational_parameters), - read_version: None, - isolation_level: None, - is_blind_append: None, - engine_info: None, - info: Default::default(), + ..Default::default() }; let actions = vec![ Action::CommitInfo(commit_info), Action::Metadata(Metadata::try_from(metadata)?), - Action::Protocol(protocol) + Action::Protocol(protocol), ]; let operations = DeltaOperation::AddConstraint { name: name.clone(), - expr: expr.clone() + expr: expr.clone(), }; let version = commit( @@ -106,11 +119,12 @@ impl std::future::IntoFuture for ConstraintBuilder { &actions, operations, &this.snapshot, - None - ).await?; - + None, + ) + .await?; + Ok(DeltaTable::new_with_state(this.log_store, this.snapshot)) }; Box::pin(fut) } -} \ No newline at end of file +} diff --git a/crates/deltalake-core/src/operations/mod.rs b/crates/deltalake-core/src/operations/mod.rs index b051c0ed88..99dc11d5b2 100644 --- a/crates/deltalake-core/src/operations/mod.rs +++ b/crates/deltalake-core/src/operations/mod.rs @@ -26,8 +26,8 @@ pub mod vacuum; #[cfg(feature = "datafusion")] use self::{ - datafusion_utils::Expression, delete::DeleteBuilder, load::LoadBuilder, merge::MergeBuilder, - update::UpdateBuilder, write::WriteBuilder, + constraints::ConstraintBuilder, datafusion_utils::Expression, delete::DeleteBuilder, + load::LoadBuilder, merge::MergeBuilder, update::UpdateBuilder, write::WriteBuilder, }; #[cfg(feature = "datafusion")] pub use ::datafusion::physical_plan::common::collect as collect_sendable_stream; @@ -36,7 +36,6 @@ use arrow::record_batch::RecordBatch; #[cfg(all(feature = "arrow", feature = "parquet"))] use optimize::OptimizeBuilder; use restore::RestoreBuilder; -use crate::operations::constraints::ConstraintBuilder; #[cfg(feature = "datafusion")] pub mod constraints; diff --git a/crates/deltalake-core/src/protocol/mod.rs b/crates/deltalake-core/src/protocol/mod.rs index c7879f41cc..897946c123 100644 --- a/crates/deltalake-core/src/protocol/mod.rs +++ b/crates/deltalake-core/src/protocol/mod.rs @@ -414,7 +414,7 @@ pub enum DeltaOperation { AddConstraint { /// Actual constraints name: String, - expr: String + expr: String, }, /// Merge data with a source data with the following predicate @@ -503,6 +503,7 @@ impl DeltaOperation { DeltaOperation::Restore { .. } => "RESTORE", DeltaOperation::VacuumStart { .. } => "VACUUM START", DeltaOperation::VacuumEnd { .. } => "VACUUM END", + DeltaOperation::AddConstraint { .. } => "ADD CONSTRAINT", } } @@ -538,7 +539,10 @@ impl DeltaOperation { /// Denotes if the operation changes the data contained in the table pub fn changes_data(&self) -> bool { match self { - Self::Optimize { .. } | Self::VacuumStart { .. } | Self::VacuumEnd { .. } => false, + Self::Optimize { .. } + | Self::VacuumStart { .. } + | Self::VacuumEnd { .. } + | Self::AddConstraint { .. } => false, Self::Create { .. } | Self::FileSystemCheck {} | Self::StreamingUpdate { .. } diff --git a/crates/deltalake-core/src/table/mod.rs b/crates/deltalake-core/src/table/mod.rs index 33fdea2ad0..297946d755 100644 --- a/crates/deltalake-core/src/table/mod.rs +++ b/crates/deltalake-core/src/table/mod.rs @@ -137,6 +137,24 @@ impl PartialEq for CheckPoint { impl Eq for CheckPoint {} +#[derive(Eq, PartialEq, Debug, Default, Clone)] +pub struct Constraint { + /// The full path to the field. + pub name: String, + /// The SQL string that must always evaluate to true. + pub expr: String, +} + +impl Constraint { + /// Create a new invariant + pub fn new(field_name: &str, invariant_sql: &str) -> Self { + Self { + name: field_name.to_string(), + expr: invariant_sql.to_string(), + } + } +} + /// Delta table metadata #[derive(Serialize, Deserialize, Clone, Debug, PartialEq)] pub struct DeltaTableMetaData { @@ -188,6 +206,19 @@ impl DeltaTableMetaData { &self.configuration } + pub fn get_constraints(&self) -> Vec { + self.configuration + .iter() + .filter_map(|(field, value)| { + if field.starts_with("delta.constraints") { + value.as_ref().map(|f| Constraint::new(field, f)) + } else { + None + } + }) + .collect() + } + /// Return partition fields along with their data type from the current schema. pub fn get_partition_col_data_types(&self) -> Vec<(&String, &DataType)> { // JSON add actions contain a `partitionValues` field which is a map. @@ -958,6 +989,27 @@ mod tests { } } + #[test] + fn get_table_constraints() { + let state = DeltaTableMetaData::new( + None, + None, + None, + StructType::new(vec![]), + vec![], + HashMap::from_iter(vec![ + ( + "delta.constraints.id".to_string(), + Some("id > 0".to_string()), + ), + ("delta.blahblah".to_string(), None), + ]), + ); + + let constraints = state.get_constraints(); + assert_eq!(constraints.len(), 1) + } + async fn create_test_table() -> (DeltaTable, TempDir) { let tmp_dir = TempDir::new("create_table_test").unwrap(); let table_dir = tmp_dir.path().join("test_create"); diff --git a/crates/deltalake-core/tests/read_delta_log_test.rs b/crates/deltalake-core/tests/read_delta_log_test.rs index 22d09c246e..620736b23e 100644 --- a/crates/deltalake-core/tests/read_delta_log_test.rs +++ b/crates/deltalake-core/tests/read_delta_log_test.rs @@ -1,10 +1,10 @@ +use arrow_array::RecordBatch; +use datafusion_physical_expr::expressions::Column; +use deltalake_core::operations::collect_sendable_stream; use deltalake_core::{DeltaResult, DeltaTableBuilder}; use pretty_assertions::assert_eq; use std::collections::HashMap; use std::time::SystemTime; -use arrow_array::RecordBatch; -use datafusion_physical_expr::expressions::Column; -use deltalake_core::operations::collect_sendable_stream; #[allow(dead_code)] mod fs_common; @@ -30,7 +30,7 @@ async fn test_log_buffering() { location.clone(), deltalake_core::storage::config::StorageOptions::from(HashMap::new()), ) - .unwrap(), + .unwrap(), ); let mut seq_version = 0; @@ -182,7 +182,9 @@ async fn read_delta_table_with_check_constraints() -> DeltaResult<()> { // // let (_table, stream) = table.load().await?; // let data: Vec = collect_sendable_stream(stream).await?; - let constraint = table.add_constraint().with_constraint(Column::new("id", 0), ""); + let constraint = table + .add_constraint() + .with_constraint(Column::new("id", 0), ""); constraint.await?; Ok(()) -} \ No newline at end of file +} From 3192887e7121b7363e3c3aac10d6b77dbee7dfa6 Mon Sep 17 00:00:00 2001 From: Stephen Carman Date: Sun, 26 Nov 2023 16:43:38 -0500 Subject: [PATCH 03/13] feat: CHECK constraints --- .../src/kernel/actions/types.rs | 2 + .../src/operations/constraints.rs | 48 +++++++++++++++---- crates/deltalake-core/src/operations/mod.rs | 1 + .../src/operations/transaction/protocol.rs | 2 +- crates/deltalake-core/src/protocol/mod.rs | 3 +- crates/deltalake-core/src/table/mod.rs | 2 + .../tests/read_delta_log_test.rs | 7 +-- 7 files changed, 47 insertions(+), 18 deletions(-) diff --git a/crates/deltalake-core/src/kernel/actions/types.rs b/crates/deltalake-core/src/kernel/actions/types.rs index aa60823e4a..cf3716fad1 100644 --- a/crates/deltalake-core/src/kernel/actions/types.rs +++ b/crates/deltalake-core/src/kernel/actions/types.rs @@ -130,9 +130,11 @@ pub struct Protocol { pub min_writer_version: i32, /// A collection of features that a client must implement in order to correctly /// read this table (exist only when minReaderVersion is set to 3) + #[serde(skip_serializing_if = "Option::is_none")] pub reader_features: Option>, /// A collection of features that a client must implement in order to correctly /// write this table (exist only when minWriterVersion is set to 7) + #[serde(skip_serializing_if = "Option::is_none")] pub writer_features: Option>, } diff --git a/crates/deltalake-core/src/operations/constraints.rs b/crates/deltalake-core/src/operations/constraints.rs index 3f5a7318e6..3e35c437c8 100644 --- a/crates/deltalake-core/src/operations/constraints.rs +++ b/crates/deltalake-core/src/operations/constraints.rs @@ -1,10 +1,13 @@ +//! Add a check constraint to a table + use std::collections::HashMap; use chrono::Utc; use datafusion::execution::context::SessionState; use futures::future::BoxFuture; +use serde_json::json; -use crate::kernel::{Action, CommitInfo, Metadata, Protocol}; +use crate::kernel::{Action, CommitInfo, IsolationLevel, Metadata, Protocol}; use crate::logstore::LogStoreRef; use crate::operations::datafusion_utils::Expression; use crate::operations::transaction::commit; @@ -12,7 +15,9 @@ use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; use crate::DeltaTable; use crate::{DeltaResult, DeltaTableError}; +use crate::delta_datafusion::DeltaDataChecker; +/// Build a constraint to add to a table pub struct ConstraintBuilder { snapshot: DeltaTableState, name: Option, @@ -22,6 +27,7 @@ pub struct ConstraintBuilder { } impl ConstraintBuilder { + /// Create a new builder pub fn new(log_store: LogStoreRef, snapshot: DeltaTableState) -> Self { Self { name: None, @@ -32,6 +38,7 @@ impl ConstraintBuilder { } } + /// Specify the constraint to be added pub fn with_constraint, E: Into>( mut self, column: S, @@ -42,6 +49,7 @@ impl ConstraintBuilder { self } + /// Specify the datafusion session context pub fn with_session_state(mut self, state: SessionState) -> Self { self.state = Some(state); self @@ -54,9 +62,9 @@ impl std::future::IntoFuture for ConstraintBuilder { type IntoFuture = BoxFuture<'static, Self::Output>; fn into_future(self) -> Self::IntoFuture { - let mut this = self; + let this = self; - let fut = async move { + Box::pin(async move { if this.name.is_none() { return Err(DeltaTableError::Generic("No name provided".to_string())); } else if this.expr.is_none() { @@ -70,15 +78,27 @@ impl std::future::IntoFuture for ConstraintBuilder { Expression::String(s) => s, Expression::DataFusion(e) => e.to_string(), }; + + // let checker = DeltaDataChecker::new(this.snapshot); + + dbg!(&name, &expr); let mut metadata = this .snapshot .current_metadata() .ok_or(DeltaTableError::NoMetadata)? .clone(); + let configuration_key = format!("delta.constraints.{}", name); + + if metadata.configuration.contains_key(&configuration_key) { + return Err(DeltaTableError::Generic(format!( + "Constraint with name: {} already exists, expr: {}", + name, expr + ))); + } metadata .configuration - .insert(format!("delta.constraints.{}", name), Some(expr.into())); + .insert(format!("delta.constraints.{}", name), Some(expr.clone())); let protocol = Protocol { min_reader_version: if this.snapshot.min_reader_version() > 1 { @@ -91,15 +111,21 @@ impl std::future::IntoFuture for ConstraintBuilder { } else { 3 }, - reader_features: None, - writer_features: None, + reader_features: this.snapshot.reader_features().cloned(), + writer_features: this.snapshot.writer_features().cloned(), }; - let operational_parameters = HashMap::from_iter(&[(name, expr)]); + let operational_parameters = HashMap::from_iter([ + ("name".to_string(), json!(&name)), + ("expr".to_string(), json!(&expr)), + ]); let commit_info = CommitInfo { timestamp: Some(Utc::now().timestamp_millis()), operation: Some("ADD CONSTRAINT".to_string()), operation_parameters: Some(operational_parameters), + read_version: Some(this.snapshot.version()), + isolation_level: Some(IsolationLevel::Serializable), + is_blind_append: Some(false), ..Default::default() }; @@ -114,7 +140,9 @@ impl std::future::IntoFuture for ConstraintBuilder { expr: expr.clone(), }; - let version = commit( + dbg!(&actions); + + let _version = commit( this.log_store.as_ref(), &actions, operations, @@ -122,9 +150,9 @@ impl std::future::IntoFuture for ConstraintBuilder { None, ) .await?; + dbg!(_version); Ok(DeltaTable::new_with_state(this.log_store, this.snapshot)) - }; - Box::pin(fut) + }) } } diff --git a/crates/deltalake-core/src/operations/mod.rs b/crates/deltalake-core/src/operations/mod.rs index 99dc11d5b2..b038de027c 100644 --- a/crates/deltalake-core/src/operations/mod.rs +++ b/crates/deltalake-core/src/operations/mod.rs @@ -173,6 +173,7 @@ impl DeltaOps { MergeBuilder::new(self.0.log_store, self.0.state, predicate.into(), source) } + /// Add a check constraint to a table #[cfg(feature = "datafusion")] #[must_use] pub fn add_constraint(self) -> ConstraintBuilder { diff --git a/crates/deltalake-core/src/operations/transaction/protocol.rs b/crates/deltalake-core/src/operations/transaction/protocol.rs index 47e4d0a41a..6fa5a5fa8c 100644 --- a/crates/deltalake-core/src/operations/transaction/protocol.rs +++ b/crates/deltalake-core/src/operations/transaction/protocol.rs @@ -160,7 +160,7 @@ pub static INSTANCE: Lazy = Lazy::new(|| { let mut writer_features = HashSet::new(); writer_features.insert(WriterFeatures::AppendOnly); writer_features.insert(WriterFeatures::Invariants); - // writer_features.insert(WriterFeatures::CheckConstraints); + writer_features.insert(WriterFeatures::CheckConstraints); // writer_features.insert(WriterFeatures::ChangeDataFeed); // writer_features.insert(WriterFeatures::GeneratedColumns); // writer_features.insert(WriterFeatures::ColumnMapping); diff --git a/crates/deltalake-core/src/protocol/mod.rs b/crates/deltalake-core/src/protocol/mod.rs index 48e5b23f1b..e1e02e382d 100644 --- a/crates/deltalake-core/src/protocol/mod.rs +++ b/crates/deltalake-core/src/protocol/mod.rs @@ -412,8 +412,9 @@ pub enum DeltaOperation { }, /// Add constraints to a table AddConstraint { - /// Actual constraints + /// Constraints name name: String, + /// Expression to check against expr: String, }, diff --git a/crates/deltalake-core/src/table/mod.rs b/crates/deltalake-core/src/table/mod.rs index 297946d755..d9aedb10b0 100644 --- a/crates/deltalake-core/src/table/mod.rs +++ b/crates/deltalake-core/src/table/mod.rs @@ -137,6 +137,7 @@ impl PartialEq for CheckPoint { impl Eq for CheckPoint {} +/// A constraint in a check constraint #[derive(Eq, PartialEq, Debug, Default, Clone)] pub struct Constraint { /// The full path to the field. @@ -206,6 +207,7 @@ impl DeltaTableMetaData { &self.configuration } + /// Return the check constraints on the current table pub fn get_constraints(&self) -> Vec { self.configuration .iter() diff --git a/crates/deltalake-core/tests/read_delta_log_test.rs b/crates/deltalake-core/tests/read_delta_log_test.rs index 620736b23e..f8e7bed3eb 100644 --- a/crates/deltalake-core/tests/read_delta_log_test.rs +++ b/crates/deltalake-core/tests/read_delta_log_test.rs @@ -1,6 +1,3 @@ -use arrow_array::RecordBatch; -use datafusion_physical_expr::expressions::Column; -use deltalake_core::operations::collect_sendable_stream; use deltalake_core::{DeltaResult, DeltaTableBuilder}; use pretty_assertions::assert_eq; use std::collections::HashMap; @@ -182,9 +179,7 @@ async fn read_delta_table_with_check_constraints() -> DeltaResult<()> { // // let (_table, stream) = table.load().await?; // let data: Vec = collect_sendable_stream(stream).await?; - let constraint = table - .add_constraint() - .with_constraint(Column::new("id", 0), ""); + let constraint = table.add_constraint().with_constraint("id2", "id < 100"); constraint.await?; Ok(()) } From d6020e84e4ee8d7c57d9e3d85a924a95953c955e Mon Sep 17 00:00:00 2001 From: Stephen Carman Date: Sat, 2 Dec 2023 12:28:56 -0500 Subject: [PATCH 04/13] feat: CHECK constraints, added check on table for initial add of constraint --- .../src/delta_datafusion/mod.rs | 85 ++++++++++++++----- crates/deltalake-core/src/kernel/schema.rs | 18 ++++ .../src/operations/constraints.rs | 34 +++++++- crates/deltalake-core/src/operations/write.rs | 7 +- crates/deltalake-core/src/table/mod.rs | 14 ++- .../tests/read_delta_log_test.rs | 14 +-- crates/deltalake-sql/src/planner.rs | 2 - python/src/lib.rs | 2 +- 8 files changed, 136 insertions(+), 40 deletions(-) diff --git a/crates/deltalake-core/src/delta_datafusion/mod.rs b/crates/deltalake-core/src/delta_datafusion/mod.rs index 8dea811383..e4292ecd04 100644 --- a/crates/deltalake-core/src/delta_datafusion/mod.rs +++ b/crates/deltalake-core/src/delta_datafusion/mod.rs @@ -71,11 +71,12 @@ use serde::{Deserialize, Serialize}; use url::Url; use crate::errors::{DeltaResult, DeltaTableError}; -use crate::kernel::{Add, DataType as DeltaDataType, Invariant, PrimitiveType}; +use crate::kernel::{Add, DataCheck, DataType as DeltaDataType, Invariant, PrimitiveType}; use crate::logstore::LogStoreRef; use crate::protocol::{ColumnCountStat, ColumnValueStat}; use crate::table::builder::ensure_table_uri; use crate::table::state::DeltaTableState; +use crate::table::Constraint; use crate::{open_table, open_table_with_storage_options, DeltaTable}; const PATH_COLUMN: &str = "__delta_rs_path"; @@ -1016,15 +1017,41 @@ pub(crate) fn logical_expr_to_physical_expr( /// Responsible for checking batches of data conform to table's invariants. #[derive(Clone)] pub struct DeltaDataChecker { + constraints: Vec, invariants: Vec, ctx: SessionContext, } impl DeltaDataChecker { + /// Create a new DeltaDataChecker with a specified set of invariants + pub fn with_invariants(invariants: Vec) -> Self { + Self { + invariants, + constraints: vec![], + ctx: SessionContext::new(), + } + } + + /// Create a new DeltaDataChecker with a specified set of constraints + pub fn with_constraints(constraints: Vec) -> Self { + Self { + constraints, + invariants: vec![], + ctx: SessionContext::new(), + } + } + /// Create a new DeltaDataChecker - pub fn new(invariants: Vec) -> Self { + pub fn new(snapshot: &DeltaTableState) -> Self { + let invariants = snapshot + .current_metadata() + .and_then(|meta| meta.schema.get_invariants().ok()) + .unwrap_or_default(); + + let constraints = vec![]; Self { invariants, + constraints, ctx: SessionContext::new(), } } @@ -1034,45 +1061,53 @@ impl DeltaDataChecker { /// If it does not, it will return [DeltaTableError::InvalidData] with a list /// of values that violated each invariant. pub async fn check_batch(&self, record_batch: &RecordBatch) -> Result<(), DeltaTableError> { - self.enforce_invariants(record_batch).await - // TODO: for support for Protocol V3, check constraints + self.enforce_checks(record_batch, &self.invariants).await?; + self.enforce_checks(record_batch, &self.constraints).await } - async fn enforce_invariants(&self, record_batch: &RecordBatch) -> Result<(), DeltaTableError> { - // Invariants are deprecated, so let's not pay the overhead for any of this - // if we can avoid it. - if self.invariants.is_empty() { + async fn enforce_checks( + &self, + record_batch: &RecordBatch, + checks: &[C], + ) -> Result<(), DeltaTableError> { + if checks.is_empty() { return Ok(()); } - let table = MemTable::try_new(record_batch.schema(), vec![vec![record_batch.clone()]])?; - self.ctx.register_table("data", Arc::new(table))?; + if !self.ctx.table_exist("data")? { + let table = MemTable::try_new(record_batch.schema(), vec![vec![record_batch.clone()]])?; + self.ctx.register_table("data", Arc::new(table))?; + } let mut violations: Vec = Vec::new(); - for invariant in self.invariants.iter() { - if invariant.field_name.contains('.') { + for check in checks { + if check.get_name().contains('.') { return Err(DeltaTableError::Generic( - "Support for column invariants on nested columns is not supported.".to_string(), + "Support for nested columns is not supported.".to_string(), )); } let sql = format!( - "SELECT {} FROM data WHERE not ({}) LIMIT 1", - invariant.field_name, invariant.invariant_sql + "SELECT {} FROM data WHERE NOT ({}) LIMIT 1", + check.get_name(), + check.get_expression() ); + dbg!(&sql); let dfs: Vec = self.ctx.sql(&sql).await?.collect().await?; if !dfs.is_empty() && dfs[0].num_rows() > 0 { let value = format!("{:?}", dfs[0].column(0)); let msg = format!( - "Invariant ({}) violated by value {}", - invariant.invariant_sql, value + "Check or Invariant ({}) violated by value {}", + check.get_expression(), + value ); violations.push(msg); } } + self.ctx.deregister_table("data")?; if !violations.is_empty() { Err(DeltaTableError::InvalidData { violations }) } else { @@ -1642,7 +1677,7 @@ mod tests { .unwrap(); // Empty invariants is okay let invariants: Vec = vec![]; - assert!(DeltaDataChecker::new(invariants) + assert!(DeltaDataChecker::with_invariants(invariants) .check_batch(&batch) .await .is_ok()); @@ -1652,7 +1687,7 @@ mod tests { Invariant::new("a", "a is not null"), Invariant::new("b", "b < 1000"), ]; - assert!(DeltaDataChecker::new(invariants) + assert!(DeltaDataChecker::with_invariants(invariants) .check_batch(&batch) .await .is_ok()); @@ -1662,7 +1697,9 @@ mod tests { Invariant::new("a", "a is null"), Invariant::new("b", "b < 100"), ]; - let result = DeltaDataChecker::new(invariants).check_batch(&batch).await; + let result = DeltaDataChecker::with_invariants(invariants) + .check_batch(&batch) + .await; assert!(result.is_err()); assert!(matches!(result, Err(DeltaTableError::InvalidData { .. }))); if let Err(DeltaTableError::InvalidData { violations }) = result { @@ -1671,7 +1708,9 @@ mod tests { // Irrelevant invariants return a different error let invariants = vec![Invariant::new("c", "c > 2000")]; - let result = DeltaDataChecker::new(invariants).check_batch(&batch).await; + let result = DeltaDataChecker::with_invariants(invariants) + .check_batch(&batch) + .await; assert!(result.is_err()); // Nested invariants are unsupported @@ -1685,7 +1724,9 @@ mod tests { let batch = RecordBatch::try_new(schema, vec![inner]).unwrap(); let invariants = vec![Invariant::new("x.b", "x.b < 1000")]; - let result = DeltaDataChecker::new(invariants).check_batch(&batch).await; + let result = DeltaDataChecker::with_invariants(invariants) + .check_batch(&batch) + .await; assert!(result.is_err()); assert!(matches!(result, Err(DeltaTableError::Generic { .. }))); } diff --git a/crates/deltalake-core/src/kernel/schema.rs b/crates/deltalake-core/src/kernel/schema.rs index bc83c05070..f491341c9e 100644 --- a/crates/deltalake-core/src/kernel/schema.rs +++ b/crates/deltalake-core/src/kernel/schema.rs @@ -78,6 +78,14 @@ impl AsRef for ColumnMetadataKey { } } +/// A trait for all kernel types that are used as part of data checking +pub trait DataCheck { + /// The name of the specific check + fn get_name(&self) -> &str; + /// The SQL expression to use for the check + fn get_expression(&self) -> &str; +} + /// An invariant for a column that is enforced on all writes to a Delta table. #[derive(Eq, PartialEq, Debug, Default, Clone)] pub struct Invariant { @@ -97,6 +105,16 @@ impl Invariant { } } +impl DataCheck for Invariant { + fn get_name(&self) -> &str { + &self.field_name + } + + fn get_expression(&self) -> &str { + &self.invariant_sql + } +} + /// Represents a struct field defined in the Delta table schema. // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#Schema-Serialization-Format #[derive(Debug, Serialize, Deserialize, PartialEq, Clone)] diff --git a/crates/deltalake-core/src/operations/constraints.rs b/crates/deltalake-core/src/operations/constraints.rs index 3e35c437c8..453251c08c 100644 --- a/crates/deltalake-core/src/operations/constraints.rs +++ b/crates/deltalake-core/src/operations/constraints.rs @@ -1,21 +1,27 @@ //! Add a check constraint to a table use std::collections::HashMap; +use std::sync::Arc; use chrono::Utc; use datafusion::execution::context::SessionState; +use datafusion::execution::{SendableRecordBatchStream, TaskContext}; +use datafusion::physical_plan::ExecutionPlan; +use datafusion::prelude::SessionContext; use futures::future::BoxFuture; use serde_json::json; +use crate::delta_datafusion::{find_files, register_store, DeltaDataChecker, DeltaScanBuilder}; use crate::kernel::{Action, CommitInfo, IsolationLevel, Metadata, Protocol}; use crate::logstore::LogStoreRef; use crate::operations::datafusion_utils::Expression; use crate::operations::transaction::commit; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; +use crate::table::Constraint; use crate::DeltaTable; use crate::{DeltaResult, DeltaTableError}; -use crate::delta_datafusion::DeltaDataChecker; +use crate::operations::collect_sendable_stream; /// Build a constraint to add to a table pub struct ConstraintBuilder { @@ -79,7 +85,31 @@ impl std::future::IntoFuture for ConstraintBuilder { Expression::DataFusion(e) => e.to_string(), }; - // let checker = DeltaDataChecker::new(this.snapshot); + let state = this.state.unwrap_or_else(|| { + let session = SessionContext::new(); + register_store(this.log_store.clone(), session.runtime_env()); + session.state() + }); + dbg!(&state); + + let checker = DeltaDataChecker::with_constraints(vec![Constraint::new("*", &expr)]); + + let files_to_check = + find_files(&this.snapshot, this.log_store.clone(), &state, None).await?; + dbg!(&files_to_check.candidates); + let scan = DeltaScanBuilder::new(&this.snapshot, this.log_store.clone(), &state) + .with_files(&files_to_check.candidates) + .build() + .await?; + let scan = Arc::new(scan); + + let task_ctx = Arc::new(TaskContext::from(&state)); + + let record_stream: SendableRecordBatchStream = scan.execute(0, task_ctx)?; + let records = collect_sendable_stream(record_stream).await?; + for batch in records { + checker.check_batch(&batch).await?; + } dbg!(&name, &expr); let mut metadata = this diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index cb68b72bb2..ff502b070b 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -252,15 +252,10 @@ pub(crate) async fn write_execution_plan( writer_properties: Option, safe_cast: bool, ) -> DeltaResult> { - let invariants = snapshot - .current_metadata() - .and_then(|meta| meta.schema.get_invariants().ok()) - .unwrap_or_default(); - // Use input schema to prevent wrapping partitions columns into a dictionary. let schema = snapshot.input_schema().unwrap_or(plan.schema()); - let checker = DeltaDataChecker::new(invariants); + let checker = DeltaDataChecker::new(snapshot); // Write data to disk let mut tasks = vec![]; diff --git a/crates/deltalake-core/src/table/mod.rs b/crates/deltalake-core/src/table/mod.rs index d9aedb10b0..7165bb8665 100644 --- a/crates/deltalake-core/src/table/mod.rs +++ b/crates/deltalake-core/src/table/mod.rs @@ -22,8 +22,8 @@ use self::builder::DeltaTableConfig; use self::state::DeltaTableState; use crate::errors::DeltaTableError; use crate::kernel::{ - Action, Add, CommitInfo, DataType, Format, Metadata, ReaderFeatures, Remove, StructType, - WriterFeatures, + Action, Add, CommitInfo, DataCheck, DataType, Format, Metadata, ReaderFeatures, Remove, + StructType, WriterFeatures, }; use crate::logstore::LogStoreConfig; use crate::logstore::LogStoreRef; @@ -156,6 +156,16 @@ impl Constraint { } } +impl DataCheck for Constraint { + fn get_name(&self) -> &str { + &self.name + } + + fn get_expression(&self) -> &str { + &self.expr + } +} + /// Delta table metadata #[derive(Serialize, Deserialize, Clone, Debug, PartialEq)] pub struct DeltaTableMetaData { diff --git a/crates/deltalake-core/tests/read_delta_log_test.rs b/crates/deltalake-core/tests/read_delta_log_test.rs index f8e7bed3eb..a668bad39c 100644 --- a/crates/deltalake-core/tests/read_delta_log_test.rs +++ b/crates/deltalake-core/tests/read_delta_log_test.rs @@ -2,6 +2,7 @@ use deltalake_core::{DeltaResult, DeltaTableBuilder}; use pretty_assertions::assert_eq; use std::collections::HashMap; use std::time::SystemTime; +use deltalake_core::DeltaTableError::InvalidData; #[allow(dead_code)] mod fs_common; @@ -176,10 +177,13 @@ async fn read_delta_table_from_dlt() { #[tokio::test] async fn read_delta_table_with_check_constraints() -> DeltaResult<()> { let table = deltalake_core::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; - // - // let (_table, stream) = table.load().await?; - // let data: Vec = collect_sendable_stream(stream).await?; - let constraint = table.add_constraint().with_constraint("id2", "id < 100"); - constraint.await?; + + let constraint = table.add_constraint().with_constraint("id3", "id < 60"); + + if let Err(InvalidData {violations} ) = constraint.await { + for v in violations { + println!("{}", v); + } + } Ok(()) } diff --git a/crates/deltalake-sql/src/planner.rs b/crates/deltalake-sql/src/planner.rs index bf07825d4b..099f97087d 100644 --- a/crates/deltalake-sql/src/planner.rs +++ b/crates/deltalake-sql/src/planner.rs @@ -48,7 +48,6 @@ impl<'a, S: ContextProvider> DeltaSqlToRel<'a, S> { } Statement::Describe(describe) => self.describe_to_plan(describe), Statement::Vacuum(vacuum) => self.vacuum_to_plan(vacuum), - _ => todo!(), } } @@ -92,7 +91,6 @@ mod tests { use arrow_schema::{DataType, Field, Schema}; use datafusion_common::config::ConfigOptions; use datafusion_common::DataFusionError; - use datafusion_common::Result as DataFusionResult; use datafusion_expr::logical_plan::builder::LogicalTableSource; use datafusion_expr::{AggregateUDF, ScalarUDF, TableSource}; use datafusion_sql::TableReference; diff --git a/python/src/lib.rs b/python/src/lib.rs index 69195e866d..f4fca8a5cb 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1242,7 +1242,7 @@ impl PyDeltaDataChecker { }) .collect(); Self { - inner: DeltaDataChecker::new(invariants), + inner: DeltaDataChecker::with_invariants(invariants), rt: tokio::runtime::Runtime::new().unwrap(), } } From 916e1e75923020645118a931a6d0deee34cee68b Mon Sep 17 00:00:00 2001 From: Stephen Carman Date: Sun, 3 Dec 2023 18:01:03 -0500 Subject: [PATCH 05/13] feat: CHECK constraints, added check on table for initial add of constraint --- .../deltalake-core/src/delta_datafusion/mod.rs | 16 +++++++++------- crates/deltalake-core/src/kernel/schema.rs | 2 +- .../deltalake-core/src/operations/constraints.rs | 13 ++++++------- crates/deltalake-core/src/operations/write.rs | 2 +- crates/deltalake-core/src/table/mod.rs | 2 +- .../deltalake-core/tests/read_delta_log_test.rs | 4 ++-- 6 files changed, 20 insertions(+), 19 deletions(-) diff --git a/crates/deltalake-core/src/delta_datafusion/mod.rs b/crates/deltalake-core/src/delta_datafusion/mod.rs index e4292ecd04..61ac4fd6c6 100644 --- a/crates/deltalake-core/src/delta_datafusion/mod.rs +++ b/crates/deltalake-core/src/delta_datafusion/mod.rs @@ -1042,18 +1042,20 @@ impl DeltaDataChecker { } /// Create a new DeltaDataChecker - pub fn new(snapshot: &DeltaTableState) -> Self { - let invariants = snapshot - .current_metadata() + pub fn new(snapshot: &DeltaTableState) -> Result { + let metadata = snapshot.current_metadata(); + + let invariants = metadata .and_then(|meta| meta.schema.get_invariants().ok()) .unwrap_or_default(); - - let constraints = vec![]; - Self { + let constraints = metadata + .and_then(|meta| Some(meta.get_constraints())) + .unwrap_or_default(); + Ok(Self { invariants, constraints, ctx: SessionContext::new(), - } + }) } /// Check that a record batch conforms to table's invariants. diff --git a/crates/deltalake-core/src/kernel/schema.rs b/crates/deltalake-core/src/kernel/schema.rs index f491341c9e..169eab4b6e 100644 --- a/crates/deltalake-core/src/kernel/schema.rs +++ b/crates/deltalake-core/src/kernel/schema.rs @@ -111,7 +111,7 @@ impl DataCheck for Invariant { } fn get_expression(&self) -> &str { - &self.invariant_sql + &self.invariant_sql } } diff --git a/crates/deltalake-core/src/operations/constraints.rs b/crates/deltalake-core/src/operations/constraints.rs index 453251c08c..b1ff621731 100644 --- a/crates/deltalake-core/src/operations/constraints.rs +++ b/crates/deltalake-core/src/operations/constraints.rs @@ -14,14 +14,13 @@ use serde_json::json; use crate::delta_datafusion::{find_files, register_store, DeltaDataChecker, DeltaScanBuilder}; use crate::kernel::{Action, CommitInfo, IsolationLevel, Metadata, Protocol}; use crate::logstore::LogStoreRef; +use crate::operations::collect_sendable_stream; use crate::operations::datafusion_utils::Expression; use crate::operations::transaction::commit; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; -use crate::table::Constraint; use crate::DeltaTable; use crate::{DeltaResult, DeltaTableError}; -use crate::operations::collect_sendable_stream; /// Build a constraint to add to a table pub struct ConstraintBuilder { @@ -80,9 +79,10 @@ impl std::future::IntoFuture for ConstraintBuilder { } let name = this.name.unwrap(); - let expr = match this.expr.unwrap() { - Expression::String(s) => s, - Expression::DataFusion(e) => e.to_string(), + let expr = match this.expr { + Some(Expression::String(s)) => s, + Some(Expression::DataFusion(e)) => e.to_string(), + None => unreachable!(), }; let state = this.state.unwrap_or_else(|| { @@ -92,11 +92,10 @@ impl std::future::IntoFuture for ConstraintBuilder { }); dbg!(&state); - let checker = DeltaDataChecker::with_constraints(vec![Constraint::new("*", &expr)]); + let checker = DeltaDataChecker::new(&this.snapshot)?; let files_to_check = find_files(&this.snapshot, this.log_store.clone(), &state, None).await?; - dbg!(&files_to_check.candidates); let scan = DeltaScanBuilder::new(&this.snapshot, this.log_store.clone(), &state) .with_files(&files_to_check.candidates) .build() diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index ff502b070b..9f064e6ff4 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -255,7 +255,7 @@ pub(crate) async fn write_execution_plan( // Use input schema to prevent wrapping partitions columns into a dictionary. let schema = snapshot.input_schema().unwrap_or(plan.schema()); - let checker = DeltaDataChecker::new(snapshot); + let checker = DeltaDataChecker::new(snapshot)?; // Write data to disk let mut tasks = vec![]; diff --git a/crates/deltalake-core/src/table/mod.rs b/crates/deltalake-core/src/table/mod.rs index 7165bb8665..7ff58c61cf 100644 --- a/crates/deltalake-core/src/table/mod.rs +++ b/crates/deltalake-core/src/table/mod.rs @@ -223,7 +223,7 @@ impl DeltaTableMetaData { .iter() .filter_map(|(field, value)| { if field.starts_with("delta.constraints") { - value.as_ref().map(|f| Constraint::new(field, f)) + value.as_ref().map(|f| Constraint::new("*", f)) } else { None } diff --git a/crates/deltalake-core/tests/read_delta_log_test.rs b/crates/deltalake-core/tests/read_delta_log_test.rs index a668bad39c..bb3c412703 100644 --- a/crates/deltalake-core/tests/read_delta_log_test.rs +++ b/crates/deltalake-core/tests/read_delta_log_test.rs @@ -1,8 +1,8 @@ +use deltalake_core::DeltaTableError::InvalidData; use deltalake_core::{DeltaResult, DeltaTableBuilder}; use pretty_assertions::assert_eq; use std::collections::HashMap; use std::time::SystemTime; -use deltalake_core::DeltaTableError::InvalidData; #[allow(dead_code)] mod fs_common; @@ -180,7 +180,7 @@ async fn read_delta_table_with_check_constraints() -> DeltaResult<()> { let constraint = table.add_constraint().with_constraint("id3", "id < 60"); - if let Err(InvalidData {violations} ) = constraint.await { + if let Err(InvalidData { violations }) = constraint.await { for v in violations { println!("{}", v); } From 305bae9193f9e1f5c95d1bb21820a4ae1f808f0c Mon Sep 17 00:00:00 2001 From: Stephen Carman Date: Sun, 3 Dec 2023 19:30:10 -0500 Subject: [PATCH 06/13] feat: check constraints --- .../src/delta_datafusion/mod.rs | 17 ++++--- .../src/operations/constraints.rs | 44 +++++++++++++------ crates/deltalake-core/src/operations/write.rs | 2 +- .../tests/read_delta_log_test.rs | 17 +------ 4 files changed, 40 insertions(+), 40 deletions(-) diff --git a/crates/deltalake-core/src/delta_datafusion/mod.rs b/crates/deltalake-core/src/delta_datafusion/mod.rs index fa10973dee..c45bbfcb01 100644 --- a/crates/deltalake-core/src/delta_datafusion/mod.rs +++ b/crates/deltalake-core/src/delta_datafusion/mod.rs @@ -1042,8 +1042,8 @@ impl DeltaDataChecker { } /// Create a new DeltaDataChecker - pub fn new(snapshot: &DeltaTableState) -> Result { - let metadata = snapshot.current_metadata(); + pub fn new(snapshot: &DeltaTableState) -> Self { + let metadata = snapshot.metadata(); let invariants = metadata .and_then(|meta| meta.schema.get_invariants().ok()) @@ -1051,11 +1051,11 @@ impl DeltaDataChecker { let constraints = metadata .and_then(|meta| Some(meta.get_constraints())) .unwrap_or_default(); - Ok(Self { + Self { invariants, constraints, ctx: SessionContext::new(), - }) + } } /// Check that a record batch conforms to table's invariants. @@ -1076,10 +1076,10 @@ impl DeltaDataChecker { return Ok(()); } - if !self.ctx.table_exist("data")? { - let table = MemTable::try_new(record_batch.schema(), vec![vec![record_batch.clone()]])?; - self.ctx.register_table("data", Arc::new(table))?; - } + // if !self.ctx.table_exist("data")? { + let table = MemTable::try_new(record_batch.schema(), vec![vec![record_batch.clone()]])?; + self.ctx.register_table("data", Arc::new(table))?; + // } let mut violations: Vec = Vec::new(); @@ -1095,7 +1095,6 @@ impl DeltaDataChecker { check.get_name(), check.get_expression() ); - dbg!(&sql); let dfs: Vec = self.ctx.sql(&sql).await?.collect().await?; if !dfs.is_empty() && dfs[0].num_rows() > 0 { diff --git a/crates/deltalake-core/src/operations/constraints.rs b/crates/deltalake-core/src/operations/constraints.rs index b1ff621731..5f71deaaca 100644 --- a/crates/deltalake-core/src/operations/constraints.rs +++ b/crates/deltalake-core/src/operations/constraints.rs @@ -90,9 +90,8 @@ impl std::future::IntoFuture for ConstraintBuilder { register_store(this.log_store.clone(), session.runtime_env()); session.state() }); - dbg!(&state); - let checker = DeltaDataChecker::new(&this.snapshot)?; + let checker = DeltaDataChecker::new(&this.snapshot); let files_to_check = find_files(&this.snapshot, this.log_store.clone(), &state, None).await?; @@ -100,7 +99,6 @@ impl std::future::IntoFuture for ConstraintBuilder { .with_files(&files_to_check.candidates) .build() .await?; - let scan = Arc::new(scan); let task_ctx = Arc::new(TaskContext::from(&state)); @@ -110,10 +108,9 @@ impl std::future::IntoFuture for ConstraintBuilder { checker.check_batch(&batch).await?; } - dbg!(&name, &expr); let mut metadata = this .snapshot - .current_metadata() + .metadata() .ok_or(DeltaTableError::NoMetadata)? .clone(); let configuration_key = format!("delta.constraints.{}", name); @@ -129,19 +126,20 @@ impl std::future::IntoFuture for ConstraintBuilder { .configuration .insert(format!("delta.constraints.{}", name), Some(expr.clone())); + let old_protocol = this.snapshot.protocol(); let protocol = Protocol { - min_reader_version: if this.snapshot.min_reader_version() > 1 { - this.snapshot.min_reader_version() + min_reader_version: if old_protocol.min_reader_version > 1 { + old_protocol.min_reader_version } else { 1 }, - min_writer_version: if this.snapshot.min_reader_version() > 3 { - this.snapshot.min_reader_version() + min_writer_version: if old_protocol.min_reader_version > 3 { + old_protocol.min_reader_version } else { 3 }, - reader_features: this.snapshot.reader_features().cloned(), - writer_features: this.snapshot.writer_features().cloned(), + reader_features: old_protocol.reader_features.clone(), + writer_features: old_protocol.writer_features.clone(), }; let operational_parameters = HashMap::from_iter([ @@ -169,8 +167,6 @@ impl std::future::IntoFuture for ConstraintBuilder { expr: expr.clone(), }; - dbg!(&actions); - let _version = commit( this.log_store.as_ref(), &actions, @@ -179,9 +175,29 @@ impl std::future::IntoFuture for ConstraintBuilder { None, ) .await?; - dbg!(_version); Ok(DeltaTable::new_with_state(this.log_store, this.snapshot)) }) } } + +#[cfg(test)] +mod tests { + use crate::DeltaResult; + use crate::DeltaTableError::InvalidData; + + #[cfg(feature = "datafusion")] + #[tokio::test] + async fn read_delta_table_with_check_constraints() -> DeltaResult<()> { + let table = crate::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; + + let constraint = table.add_constraint().with_constraint("id3", "id < 60"); + + if let Err(InvalidData { violations }) = constraint.await { + for v in violations { + println!("{}", v); + } + } + Ok(()) + } +} \ No newline at end of file diff --git a/crates/deltalake-core/src/operations/write.rs b/crates/deltalake-core/src/operations/write.rs index 19889c1057..cdb01ab737 100644 --- a/crates/deltalake-core/src/operations/write.rs +++ b/crates/deltalake-core/src/operations/write.rs @@ -313,7 +313,7 @@ pub(crate) async fn write_execution_plan( snapshot.input_schema().unwrap_or(plan.schema()) }; - let checker = DeltaDataChecker::new(snapshot)?; + let checker = DeltaDataChecker::new(snapshot); // Write data to disk let mut tasks = vec![]; diff --git a/crates/deltalake-core/tests/read_delta_log_test.rs b/crates/deltalake-core/tests/read_delta_log_test.rs index bb3c412703..e70a3ee1a5 100644 --- a/crates/deltalake-core/tests/read_delta_log_test.rs +++ b/crates/deltalake-core/tests/read_delta_log_test.rs @@ -1,4 +1,3 @@ -use deltalake_core::DeltaTableError::InvalidData; use deltalake_core::{DeltaResult, DeltaTableBuilder}; use pretty_assertions::assert_eq; use std::collections::HashMap; @@ -172,18 +171,4 @@ async fn read_delta_table_from_dlt() { .unwrap(); assert_eq!(table.version(), 1); assert!(table.schema().is_some()); -} - -#[tokio::test] -async fn read_delta_table_with_check_constraints() -> DeltaResult<()> { - let table = deltalake_core::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; - - let constraint = table.add_constraint().with_constraint("id3", "id < 60"); - - if let Err(InvalidData { violations }) = constraint.await { - for v in violations { - println!("{}", v); - } - } - Ok(()) -} +} \ No newline at end of file From 8f913e7d5a00162797a536013cc176e89caef996 Mon Sep 17 00:00:00 2001 From: Stephen Carman Date: Tue, 5 Dec 2023 22:09:14 -0500 Subject: [PATCH 07/13] feat: CHECK constraints, added check on table for initial add of constraint --- .../src/delta_datafusion/mod.rs | 2 +- .../src/operations/constraints.rs | 110 +++++++++++++----- .../tests/read_delta_log_test.rs | 2 +- 3 files changed, 86 insertions(+), 28 deletions(-) diff --git a/crates/deltalake-core/src/delta_datafusion/mod.rs b/crates/deltalake-core/src/delta_datafusion/mod.rs index c45bbfcb01..2ad61f6720 100644 --- a/crates/deltalake-core/src/delta_datafusion/mod.rs +++ b/crates/deltalake-core/src/delta_datafusion/mod.rs @@ -1079,7 +1079,7 @@ impl DeltaDataChecker { // if !self.ctx.table_exist("data")? { let table = MemTable::try_new(record_batch.schema(), vec![vec![record_batch.clone()]])?; self.ctx.register_table("data", Arc::new(table))?; - // } + // } let mut violations: Vec = Vec::new(); diff --git a/crates/deltalake-core/src/operations/constraints.rs b/crates/deltalake-core/src/operations/constraints.rs index 5f71deaaca..24ced69589 100644 --- a/crates/deltalake-core/src/operations/constraints.rs +++ b/crates/deltalake-core/src/operations/constraints.rs @@ -19,6 +19,7 @@ use crate::operations::datafusion_utils::Expression; use crate::operations::transaction::commit; use crate::protocol::DeltaOperation; use crate::table::state::DeltaTableState; +use crate::table::Constraint; use crate::DeltaTable; use crate::{DeltaResult, DeltaTableError}; @@ -77,7 +78,6 @@ impl std::future::IntoFuture for ConstraintBuilder { "No expression provided".to_string(), )); } - let name = this.name.unwrap(); let expr = match this.expr { Some(Expression::String(s)) => s, @@ -85,13 +85,28 @@ impl std::future::IntoFuture for ConstraintBuilder { None => unreachable!(), }; + let mut metadata = this + .snapshot + .metadata() + .ok_or(DeltaTableError::NoMetadata)? + .clone(); + let configuration_key = format!("delta.constraints.{}", name); + + if metadata.configuration.contains_key(&configuration_key) { + return Err(DeltaTableError::Generic(format!( + "Constraint with name: {} already exists, expr: {}", + name, expr + ))); + } + let state = this.state.unwrap_or_else(|| { let session = SessionContext::new(); register_store(this.log_store.clone(), session.runtime_env()); session.state() }); - let checker = DeltaDataChecker::new(&this.snapshot); + // Checker built here with the one time constraint to check. + let checker = DeltaDataChecker::with_constraints(vec![Constraint::new("*", &expr)]); let files_to_check = find_files(&this.snapshot, this.log_store.clone(), &state, None).await?; @@ -101,26 +116,15 @@ impl std::future::IntoFuture for ConstraintBuilder { .await?; let task_ctx = Arc::new(TaskContext::from(&state)); - let record_stream: SendableRecordBatchStream = scan.execute(0, task_ctx)?; let records = collect_sendable_stream(record_stream).await?; + for batch in records { checker.check_batch(&batch).await?; } - let mut metadata = this - .snapshot - .metadata() - .ok_or(DeltaTableError::NoMetadata)? - .clone(); - let configuration_key = format!("delta.constraints.{}", name); - - if metadata.configuration.contains_key(&configuration_key) { - return Err(DeltaTableError::Generic(format!( - "Constraint with name: {} already exists, expr: {}", - name, expr - ))); - } + // We have validated the table passes it's constraints, now to add the constraint to + // the table. metadata .configuration @@ -181,23 +185,77 @@ impl std::future::IntoFuture for ConstraintBuilder { } } +#[cfg(feature = "datafusion")] #[cfg(test)] mod tests { + use std::sync::Arc; + + use arrow_array::{Array, Int64Array, RecordBatch}; + + use crate::kernel::StructType; use crate::DeltaResult; - use crate::DeltaTableError::InvalidData; - #[cfg(feature = "datafusion")] #[tokio::test] - async fn read_delta_table_with_check_constraints() -> DeltaResult<()> { + async fn add_constraint_with_invalid_data() -> DeltaResult<()> { let table = crate::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; + let constraint = table + .add_constraint() + .with_constraint("id3", "id < 60") + .await; + assert!(constraint.is_err()); + Ok(()) + } - let constraint = table.add_constraint().with_constraint("id3", "id < 60"); + #[tokio::test] + async fn add_valid_constraint() -> DeltaResult<()> { + let table = crate::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; + let constraint = table + .add_constraint() + .with_constraint("id2", "id < 1000") + .await; + assert!(constraint.is_ok()); + let version = constraint?.version(); + assert_eq!(version, 2); + Ok(()) + } - if let Err(InvalidData { violations }) = constraint.await { - for v in violations { - println!("{}", v); - } - } + #[tokio::test] + async fn add_conflicting_named_constraint() -> DeltaResult<()> { + let table = crate::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; + let constraint = table + .add_constraint() + .with_constraint("id", "id < 60") + .await; + + assert!(constraint.is_err()); + Ok(()) + } + + #[tokio::test] + async fn write_data_that_violates_constraint() -> DeltaResult<()> { + let table = crate::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; + let metadata = table.0.get_metadata()?; + let arrow_schema = + >::try_from(&metadata.schema.clone())?; + let invalid_values: Vec> = vec![Arc::new(Int64Array::from(vec![-10]))]; + let batch = RecordBatch::try_new(Arc::new(arrow_schema), invalid_values)?; + let err = table.write(vec![batch]).await; + + assert!(err.is_err()); Ok(()) } -} \ No newline at end of file + + #[tokio::test] + async fn write_data_that_does_not_violate_constraint() -> DeltaResult<()> { + let table = crate::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; + let metadata = table.0.get_metadata()?; + let arrow_schema = + >::try_from(&metadata.schema.clone())?; + let invalid_values: Vec> = vec![Arc::new(Int64Array::from(vec![160]))]; + let batch = RecordBatch::try_new(Arc::new(arrow_schema), invalid_values)?; + let err = table.write(vec![batch]).await; + + assert!(err.is_ok()); + Ok(()) + } +} diff --git a/crates/deltalake-core/tests/read_delta_log_test.rs b/crates/deltalake-core/tests/read_delta_log_test.rs index e70a3ee1a5..248f25c876 100644 --- a/crates/deltalake-core/tests/read_delta_log_test.rs +++ b/crates/deltalake-core/tests/read_delta_log_test.rs @@ -171,4 +171,4 @@ async fn read_delta_table_from_dlt() { .unwrap(); assert_eq!(table.version(), 1); assert!(table.schema().is_some()); -} \ No newline at end of file +} From fba0b50fcfbee0410f762a33cbc7a67f2553cef7 Mon Sep 17 00:00:00 2001 From: Stephen Carman Date: Tue, 5 Dec 2023 22:32:04 -0500 Subject: [PATCH 08/13] feat: CHECK constraints, added check on table for initial add of constraint --- crates/deltalake-core/src/kernel/mod.rs | 8 ++++++++ crates/deltalake-core/src/kernel/schema.rs | 9 +-------- .../deltalake-core/src/operations/constraints.rs | 14 ++++++++------ crates/deltalake-core/src/table/mod.rs | 4 ++-- 4 files changed, 19 insertions(+), 16 deletions(-) diff --git a/crates/deltalake-core/src/kernel/mod.rs b/crates/deltalake-core/src/kernel/mod.rs index 54f742c3fb..4dd8b57d4c 100644 --- a/crates/deltalake-core/src/kernel/mod.rs +++ b/crates/deltalake-core/src/kernel/mod.rs @@ -9,3 +9,11 @@ pub mod schema; pub use actions::*; pub use error::*; pub use schema::*; + +/// A trait for all kernel types that are used as part of data checking +pub trait DataCheck { + /// The name of the specific check + fn get_name(&self) -> &str; + /// The SQL expression to use for the check + fn get_expression(&self) -> &str; +} diff --git a/crates/deltalake-core/src/kernel/schema.rs b/crates/deltalake-core/src/kernel/schema.rs index 169eab4b6e..08cf991dd5 100644 --- a/crates/deltalake-core/src/kernel/schema.rs +++ b/crates/deltalake-core/src/kernel/schema.rs @@ -6,6 +6,7 @@ use std::hash::{Hash, Hasher}; use std::sync::Arc; use std::{collections::HashMap, fmt::Display}; +use crate::kernel::DataCheck; use serde::{Deserialize, Serialize}; use serde_json::Value; @@ -78,14 +79,6 @@ impl AsRef for ColumnMetadataKey { } } -/// A trait for all kernel types that are used as part of data checking -pub trait DataCheck { - /// The name of the specific check - fn get_name(&self) -> &str; - /// The SQL expression to use for the check - fn get_expression(&self) -> &str; -} - /// An invariant for a column that is enforced on all writes to a Delta table. #[derive(Eq, PartialEq, Debug, Default, Clone)] pub struct Invariant { diff --git a/crates/deltalake-core/src/operations/constraints.rs b/crates/deltalake-core/src/operations/constraints.rs index 24ced69589..12f3884de3 100644 --- a/crates/deltalake-core/src/operations/constraints.rs +++ b/crates/deltalake-core/src/operations/constraints.rs @@ -234,9 +234,10 @@ mod tests { #[tokio::test] async fn write_data_that_violates_constraint() -> DeltaResult<()> { let table = crate::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; - let metadata = table.0.get_metadata()?; - let arrow_schema = - >::try_from(&metadata.schema.clone())?; + let metadata = table.0.metadata()?; + let arrow_schema = >::try_from( + &metadata.schema()?.clone(), + )?; let invalid_values: Vec> = vec![Arc::new(Int64Array::from(vec![-10]))]; let batch = RecordBatch::try_new(Arc::new(arrow_schema), invalid_values)?; let err = table.write(vec![batch]).await; @@ -248,9 +249,10 @@ mod tests { #[tokio::test] async fn write_data_that_does_not_violate_constraint() -> DeltaResult<()> { let table = crate::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; - let metadata = table.0.get_metadata()?; - let arrow_schema = - >::try_from(&metadata.schema.clone())?; + let metadata = table.0.metadata()?; + let arrow_schema = >::try_from( + &metadata.schema()?.clone(), + )?; let invalid_values: Vec> = vec![Arc::new(Int64Array::from(vec![160]))]; let batch = RecordBatch::try_new(Arc::new(arrow_schema), invalid_values)?; let err = table.write(vec![batch]).await; diff --git a/crates/deltalake-core/src/table/mod.rs b/crates/deltalake-core/src/table/mod.rs index f67b567ff5..a879174055 100644 --- a/crates/deltalake-core/src/table/mod.rs +++ b/crates/deltalake-core/src/table/mod.rs @@ -22,8 +22,8 @@ use self::builder::DeltaTableConfig; use self::state::DeltaTableState; use crate::errors::DeltaTableError; use crate::kernel::{ - Action, Add, CommitInfo, DataCheck, DataType, Format, Metadata, ReaderFeatures, Remove, - StructType, WriterFeatures, + Action, Add, CommitInfo, DataCheck, DataType, Format, Metadata, Protocol, ReaderFeatures, + Remove, StructType, WriterFeatures, }; use crate::logstore::LogStoreConfig; use crate::logstore::LogStoreRef; From 811a6d413b779f2cd40d9e03a74ff699cb6dabc6 Mon Sep 17 00:00:00 2001 From: Stephen Carman Date: Wed, 6 Dec 2023 17:40:35 -0500 Subject: [PATCH 09/13] feat: CHECK constraints addressing a lint --- crates/deltalake-core/src/delta_datafusion/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/deltalake-core/src/delta_datafusion/mod.rs b/crates/deltalake-core/src/delta_datafusion/mod.rs index ce2485c845..07e307a94c 100644 --- a/crates/deltalake-core/src/delta_datafusion/mod.rs +++ b/crates/deltalake-core/src/delta_datafusion/mod.rs @@ -1049,7 +1049,7 @@ impl DeltaDataChecker { .and_then(|meta| meta.schema.get_invariants().ok()) .unwrap_or_default(); let constraints = metadata - .and_then(|meta| Some(meta.get_constraints())) + .map(|meta| meta.get_constraints()) .unwrap_or_default(); Self { invariants, From e906f3c3c19ac66b3f0594619d9fb36812093a9d Mon Sep 17 00:00:00 2001 From: Stephen Carman Date: Fri, 8 Dec 2023 17:49:23 -0500 Subject: [PATCH 10/13] feat: check constraints, removed physical test table, rewrote tests to not need a physical table, addressed most (but not all) PR comments --- .../src/delta_datafusion/mod.rs | 32 ++-- .../src/operations/constraints.rs | 173 ++++++++++++------ ...-9805-5b9c1c20d5a4-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-8d57-f468582e5fc3-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-a6dc-697e58993bc6-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-9f16-1d38e6c74721-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-bb53-d4bea9526242-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-b1f2-7481029003d3-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-b2ca-81fc04c911f7-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-b110-bd15ea28efa4-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-915f-ed535933662f-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-abb9-0e8a53e1d863-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-a6aa-b8c0e810fce8-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-a27c-4c4e76a1c817-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-986e-e038051952f2-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-a5c5-6cf37b93f174-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-a3cc-58de2fec437d-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-b3c6-d796621d2cb4-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-89d8-baa3ab5348f0-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-ba76-9a4894572379-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-9f82-366415d2e69d-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-ae12-25fc7dbc7583-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-93e2-5470e8041767-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-b241-8d3ff6e3d5c5-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-b9f7-f269423daebd-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-969b-977133666c4d-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-8006-eb0db94111ee-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-acc3-92066e12b06a-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-81e6-8b745ff8a965-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-806f-ca720d517645-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-8490-ca4a17b22d06-c000.snappy.parquet.crc | Bin 12 -> 0 bytes ...-bb78-f217a28c8d5a-c000.snappy.parquet.crc | Bin 16 -> 0 bytes ...-b324-502f75238979-c000.snappy.parquet.crc | Bin 16 -> 0 bytes .../_delta_log/.00000000000000000000.json.crc | Bin 36 -> 0 bytes .../_delta_log/.00000000000000000001.json.crc | Bin 16 -> 0 bytes .../_delta_log/.00000000000000000002.json.crc | Bin 36 -> 0 bytes .../_delta_log/00000000000000000000.json | 13 -- .../_delta_log/00000000000000000001.json | 3 - .../_delta_log/00000000000000000002.json | 11 -- ...46a8-9805-5b9c1c20d5a4-c000.snappy.parquet | Bin 527 -> 0 bytes ...421b-8d57-f468582e5fc3-c000.snappy.parquet | Bin 296 -> 0 bytes ...479a-a6dc-697e58993bc6-c000.snappy.parquet | Bin 296 -> 0 bytes ...4401-9f16-1d38e6c74721-c000.snappy.parquet | Bin 539 -> 0 bytes ...4dde-bb53-d4bea9526242-c000.snappy.parquet | Bin 296 -> 0 bytes ...4b3e-b1f2-7481029003d3-c000.snappy.parquet | Bin 527 -> 0 bytes ...4fce-b2ca-81fc04c911f7-c000.snappy.parquet | Bin 527 -> 0 bytes ...4307-b110-bd15ea28efa4-c000.snappy.parquet | Bin 296 -> 0 bytes ...4d17-915f-ed535933662f-c000.snappy.parquet | Bin 526 -> 0 bytes ...470e-abb9-0e8a53e1d863-c000.snappy.parquet | Bin 527 -> 0 bytes ...4e0c-a6aa-b8c0e810fce8-c000.snappy.parquet | Bin 527 -> 0 bytes ...4491-a27c-4c4e76a1c817-c000.snappy.parquet | Bin 296 -> 0 bytes ...4676-986e-e038051952f2-c000.snappy.parquet | Bin 527 -> 0 bytes ...47ed-a5c5-6cf37b93f174-c000.snappy.parquet | Bin 527 -> 0 bytes ...45b4-a3cc-58de2fec437d-c000.snappy.parquet | Bin 296 -> 0 bytes ...4fac-b3c6-d796621d2cb4-c000.snappy.parquet | Bin 527 -> 0 bytes ...41b9-89d8-baa3ab5348f0-c000.snappy.parquet | Bin 296 -> 0 bytes ...4f1f-ba76-9a4894572379-c000.snappy.parquet | Bin 527 -> 0 bytes ...4a80-9f82-366415d2e69d-c000.snappy.parquet | Bin 527 -> 0 bytes ...4534-ae12-25fc7dbc7583-c000.snappy.parquet | Bin 528 -> 0 bytes ...4006-93e2-5470e8041767-c000.snappy.parquet | Bin 527 -> 0 bytes ...48ee-b241-8d3ff6e3d5c5-c000.snappy.parquet | Bin 296 -> 0 bytes ...4cce-b9f7-f269423daebd-c000.snappy.parquet | Bin 296 -> 0 bytes ...4c73-969b-977133666c4d-c000.snappy.parquet | Bin 527 -> 0 bytes ...41f6-8006-eb0db94111ee-c000.snappy.parquet | Bin 527 -> 0 bytes ...4f99-acc3-92066e12b06a-c000.snappy.parquet | Bin 527 -> 0 bytes ...4b80-81e6-8b745ff8a965-c000.snappy.parquet | Bin 527 -> 0 bytes ...4a17-806f-ca720d517645-c000.snappy.parquet | Bin 296 -> 0 bytes ...4160-8490-ca4a17b22d06-c000.snappy.parquet | Bin 296 -> 0 bytes ...4ab9-bb78-f217a28c8d5a-c000.snappy.parquet | Bin 527 -> 0 bytes ...4056-b324-502f75238979-c000.snappy.parquet | Bin 527 -> 0 bytes python/src/lib.rs | 2 +- 71 files changed, 134 insertions(+), 100 deletions(-) delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00000-9d9248d4-bd6e-46a8-9805-5b9c1c20d5a4-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00000-9ed8bdb4-315a-421b-8d57-f468582e5fc3-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00000-ca2db31d-ab94-479a-a6dc-697e58993bc6-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00000-cace3c04-e21c-4401-9f16-1d38e6c74721-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00001-53e441ee-68e6-4dde-bb53-d4bea9526242-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00001-6dea6ffc-5c2d-4b3e-b1f2-7481029003d3-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00001-a5330b91-bece-4fce-b2ca-81fc04c911f7-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00002-04434872-6b96-4307-b110-bd15ea28efa4-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00002-24d4d181-0877-4d17-915f-ed535933662f-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00002-4ca0c9db-02c8-470e-abb9-0e8a53e1d863-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00003-2bd9d2e4-27a6-4e0c-a6aa-b8c0e810fce8-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00003-7996975c-4199-4491-a27c-4c4e76a1c817-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00003-7b12c177-7cfd-4676-986e-e038051952f2-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00004-0aae8759-c282-47ed-a5c5-6cf37b93f174-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00004-4ca9cc40-3cd3-45b4-a3cc-58de2fec437d-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00004-f11e1f93-e35a-4fac-b3c6-d796621d2cb4-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00005-319662f8-ef0e-41b9-89d8-baa3ab5348f0-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00005-3ae98bc2-a3bd-4f1f-ba76-9a4894572379-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00005-d44a39cb-4364-4a80-9f82-366415d2e69d-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00006-05a437ee-7c40-4534-ae12-25fc7dbc7583-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00006-7acd5504-7d04-4006-93e2-5470e8041767-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00006-9d8b5796-2cc6-48ee-b241-8d3ff6e3d5c5-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00007-5a919d87-b7be-4cce-b9f7-f269423daebd-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00007-68dbff7e-17b6-4c73-969b-977133666c4d-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00007-9f374a9d-37ec-41f6-8006-eb0db94111ee-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00008-25115960-51e0-4f99-acc3-92066e12b06a-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00008-568f62b3-4b06-4b80-81e6-8b745ff8a965-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00008-df9a0d39-3663-4a17-806f-ca720d517645-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00009-658f63ed-06a4-4160-8490-ca4a17b22d06-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00009-93e49bd4-3350-4ab9-bb78-f217a28c8d5a-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/.part-00009-da77b766-6ecf-4056-b324-502f75238979-c000.snappy.parquet.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000000.json.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000001.json.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000002.json.crc delete mode 100644 crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000000.json delete mode 100644 crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000001.json delete mode 100644 crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000002.json delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00000-9d9248d4-bd6e-46a8-9805-5b9c1c20d5a4-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00000-9ed8bdb4-315a-421b-8d57-f468582e5fc3-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00000-ca2db31d-ab94-479a-a6dc-697e58993bc6-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00000-cace3c04-e21c-4401-9f16-1d38e6c74721-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00001-53e441ee-68e6-4dde-bb53-d4bea9526242-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00001-6dea6ffc-5c2d-4b3e-b1f2-7481029003d3-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00001-a5330b91-bece-4fce-b2ca-81fc04c911f7-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00002-04434872-6b96-4307-b110-bd15ea28efa4-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00002-24d4d181-0877-4d17-915f-ed535933662f-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00002-4ca0c9db-02c8-470e-abb9-0e8a53e1d863-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00003-2bd9d2e4-27a6-4e0c-a6aa-b8c0e810fce8-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00003-7996975c-4199-4491-a27c-4c4e76a1c817-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00003-7b12c177-7cfd-4676-986e-e038051952f2-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00004-0aae8759-c282-47ed-a5c5-6cf37b93f174-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00004-4ca9cc40-3cd3-45b4-a3cc-58de2fec437d-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00004-f11e1f93-e35a-4fac-b3c6-d796621d2cb4-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00005-319662f8-ef0e-41b9-89d8-baa3ab5348f0-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00005-3ae98bc2-a3bd-4f1f-ba76-9a4894572379-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00005-d44a39cb-4364-4a80-9f82-366415d2e69d-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00006-05a437ee-7c40-4534-ae12-25fc7dbc7583-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00006-7acd5504-7d04-4006-93e2-5470e8041767-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00006-9d8b5796-2cc6-48ee-b241-8d3ff6e3d5c5-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00007-5a919d87-b7be-4cce-b9f7-f269423daebd-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00007-68dbff7e-17b6-4c73-969b-977133666c4d-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00007-9f374a9d-37ec-41f6-8006-eb0db94111ee-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00008-25115960-51e0-4f99-acc3-92066e12b06a-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00008-568f62b3-4b06-4b80-81e6-8b745ff8a965-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00008-df9a0d39-3663-4a17-806f-ca720d517645-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00009-658f63ed-06a4-4160-8490-ca4a17b22d06-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00009-93e49bd4-3350-4ab9-bb78-f217a28c8d5a-c000.snappy.parquet delete mode 100644 crates/deltalake-core/tests/data/check-constraints/part-00009-da77b766-6ecf-4056-b324-502f75238979-c000.snappy.parquet diff --git a/crates/deltalake-core/src/delta_datafusion/mod.rs b/crates/deltalake-core/src/delta_datafusion/mod.rs index 07e307a94c..649fbf58d4 100644 --- a/crates/deltalake-core/src/delta_datafusion/mod.rs +++ b/crates/deltalake-core/src/delta_datafusion/mod.rs @@ -32,7 +32,9 @@ use arrow::datatypes::{DataType as ArrowDataType, Schema as ArrowSchema, SchemaR use arrow::error::ArrowError; use arrow::record_batch::RecordBatch; use arrow_array::types::UInt16Type; -use arrow_array::{DictionaryArray, StringArray}; +use arrow_array::{Array, DictionaryArray, StringArray}; +use arrow_cast::display::array_value_to_string; + use arrow_schema::Field; use async_trait::async_trait; use chrono::{NaiveDateTime, TimeZone, Utc}; @@ -65,6 +67,7 @@ use datafusion_physical_expr::execution_props::ExecutionProps; use datafusion_physical_expr::{create_physical_expr, PhysicalExpr}; use datafusion_proto::logical_plan::LogicalExtensionCodec; use datafusion_proto::physical_plan::PhysicalExtensionCodec; +use itertools::Itertools; use log::error; use object_store::ObjectMeta; use serde::{Deserialize, Serialize}; @@ -1024,7 +1027,7 @@ pub struct DeltaDataChecker { impl DeltaDataChecker { /// Create a new DeltaDataChecker with a specified set of invariants - pub fn with_invariants(invariants: Vec) -> Self { + pub fn new_with_invariants(invariants: Vec) -> Self { Self { invariants, constraints: vec![], @@ -1033,7 +1036,7 @@ impl DeltaDataChecker { } /// Create a new DeltaDataChecker with a specified set of constraints - pub fn with_constraints(constraints: Vec) -> Self { + pub fn new_with_constraints(constraints: Vec) -> Self { Self { constraints, invariants: vec![], @@ -1075,11 +1078,8 @@ impl DeltaDataChecker { if checks.is_empty() { return Ok(()); } - - // if !self.ctx.table_exist("data")? { let table = MemTable::try_new(record_batch.schema(), vec![vec![record_batch.clone()]])?; self.ctx.register_table("data", Arc::new(table))?; - // } let mut violations: Vec = Vec::new(); @@ -1098,9 +1098,15 @@ impl DeltaDataChecker { let dfs: Vec = self.ctx.sql(&sql).await?.collect().await?; if !dfs.is_empty() && dfs[0].num_rows() > 0 { - let value = format!("{:?}", dfs[0].column(0)); + let values: String = dfs[0] + .columns() + .iter() + .map(|c| array_value_to_string(c, 0).unwrap_or(String::from("null"))) + .join(", "); + + let value = format!("{}", values); let msg = format!( - "Check or Invariant ({}) violated by value {}", + "Check or Invariant ({}) violated by value in row: [{}]", check.get_expression(), value ); @@ -1677,7 +1683,7 @@ mod tests { .unwrap(); // Empty invariants is okay let invariants: Vec = vec![]; - assert!(DeltaDataChecker::with_invariants(invariants) + assert!(DeltaDataChecker::new_with_invariants(invariants) .check_batch(&batch) .await .is_ok()); @@ -1687,7 +1693,7 @@ mod tests { Invariant::new("a", "a is not null"), Invariant::new("b", "b < 1000"), ]; - assert!(DeltaDataChecker::with_invariants(invariants) + assert!(DeltaDataChecker::new_with_invariants(invariants) .check_batch(&batch) .await .is_ok()); @@ -1697,7 +1703,7 @@ mod tests { Invariant::new("a", "a is null"), Invariant::new("b", "b < 100"), ]; - let result = DeltaDataChecker::with_invariants(invariants) + let result = DeltaDataChecker::new_with_invariants(invariants) .check_batch(&batch) .await; assert!(result.is_err()); @@ -1708,7 +1714,7 @@ mod tests { // Irrelevant invariants return a different error let invariants = vec![Invariant::new("c", "c > 2000")]; - let result = DeltaDataChecker::with_invariants(invariants) + let result = DeltaDataChecker::new_with_invariants(invariants) .check_batch(&batch) .await; assert!(result.is_err()); @@ -1724,7 +1730,7 @@ mod tests { let batch = RecordBatch::try_new(schema, vec![inner]).unwrap(); let invariants = vec![Invariant::new("x.b", "x.b < 1000")]; - let result = DeltaDataChecker::with_invariants(invariants) + let result = DeltaDataChecker::new_with_invariants(invariants) .check_batch(&batch) .await; assert!(result.is_err()); diff --git a/crates/deltalake-core/src/operations/constraints.rs b/crates/deltalake-core/src/operations/constraints.rs index 12f3884de3..734c9a24a6 100644 --- a/crates/deltalake-core/src/operations/constraints.rs +++ b/crates/deltalake-core/src/operations/constraints.rs @@ -9,12 +9,12 @@ use datafusion::execution::{SendableRecordBatchStream, TaskContext}; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; use futures::future::BoxFuture; +use futures::StreamExt; use serde_json::json; -use crate::delta_datafusion::{find_files, register_store, DeltaDataChecker, DeltaScanBuilder}; +use crate::delta_datafusion::{register_store, DeltaDataChecker, DeltaScanBuilder}; use crate::kernel::{Action, CommitInfo, IsolationLevel, Metadata, Protocol}; use crate::logstore::LogStoreRef; -use crate::operations::collect_sendable_stream; use crate::operations::datafusion_utils::Expression; use crate::operations::transaction::commit; use crate::protocol::DeltaOperation; @@ -68,21 +68,21 @@ impl std::future::IntoFuture for ConstraintBuilder { type IntoFuture = BoxFuture<'static, Self::Output>; fn into_future(self) -> Self::IntoFuture { - let this = self; + let mut this = self; Box::pin(async move { - if this.name.is_none() { - return Err(DeltaTableError::Generic("No name provided".to_string())); - } else if this.expr.is_none() { - return Err(DeltaTableError::Generic( - "No expression provided".to_string(), - )); - } - let name = this.name.unwrap(); + let name = match this.name { + Some(v) => v, + None => return Err(DeltaTableError::Generic("No name provided".to_string())), + }; let expr = match this.expr { Some(Expression::String(s)) => s, Some(Expression::DataFusion(e)) => e.to_string(), - None => unreachable!(), + None => { + return Err(DeltaTableError::Generic( + "No expression provided".to_string(), + )) + } }; let mut metadata = this @@ -106,22 +106,36 @@ impl std::future::IntoFuture for ConstraintBuilder { }); // Checker built here with the one time constraint to check. - let checker = DeltaDataChecker::with_constraints(vec![Constraint::new("*", &expr)]); - - let files_to_check = - find_files(&this.snapshot, this.log_store.clone(), &state, None).await?; + let checker = DeltaDataChecker::new_with_constraints(vec![Constraint::new("*", &expr)]); let scan = DeltaScanBuilder::new(&this.snapshot, this.log_store.clone(), &state) - .with_files(&files_to_check.candidates) .build() .await?; - let task_ctx = Arc::new(TaskContext::from(&state)); - let record_stream: SendableRecordBatchStream = scan.execute(0, task_ctx)?; - let records = collect_sendable_stream(record_stream).await?; - - for batch in records { - checker.check_batch(&batch).await?; + let plan: Arc = Arc::new(scan); + let mut tasks = vec![]; + for p in 0..plan.output_partitioning().partition_count() { + let inner_plan = plan.clone(); + let inner_checker = checker.clone(); + let task_ctx = Arc::new(TaskContext::from(&state)); + let mut record_stream: SendableRecordBatchStream = + inner_plan.execute(p, task_ctx)?; + let handle: tokio::task::JoinHandle> = + tokio::task::spawn(async move { + while let Some(maybe_batch) = record_stream.next().await { + let batch = maybe_batch?; + inner_checker.check_batch(&batch).await?; + } + Ok(()) + }); + tasks.push(handle); } + futures::future::join_all(tasks) + .await + .into_iter() + .collect::, _>>() + .map_err(|err| DeltaTableError::Generic(err.to_string()))? + .into_iter() + .collect::, _>>()?; // We have validated the table passes it's constraints, now to add the constraint to // the table. @@ -137,8 +151,8 @@ impl std::future::IntoFuture for ConstraintBuilder { } else { 1 }, - min_writer_version: if old_protocol.min_reader_version > 3 { - old_protocol.min_reader_version + min_writer_version: if old_protocol.min_writer_version > 3 { + old_protocol.min_writer_version } else { 3 }, @@ -150,9 +164,15 @@ impl std::future::IntoFuture for ConstraintBuilder { ("name".to_string(), json!(&name)), ("expr".to_string(), json!(&expr)), ]); + + let operations = DeltaOperation::AddConstraint { + name: name.clone(), + expr: expr.clone(), + }; + let commit_info = CommitInfo { timestamp: Some(Utc::now().timestamp_millis()), - operation: Some("ADD CONSTRAINT".to_string()), + operation: Some(operations.name().to_string()), operation_parameters: Some(operational_parameters), read_version: Some(this.snapshot.version()), isolation_level: Some(IsolationLevel::Serializable), @@ -166,11 +186,6 @@ impl std::future::IntoFuture for ConstraintBuilder { Action::Protocol(protocol), ]; - let operations = DeltaOperation::AddConstraint { - name: name.clone(), - expr: expr.clone(), - }; - let _version = commit( this.log_store.as_ref(), &actions, @@ -180,6 +195,11 @@ impl std::future::IntoFuture for ConstraintBuilder { ) .await?; + this.snapshot.merge( + DeltaTableState::from_actions(actions, _version)?, + true, + true, + ); Ok(DeltaTable::new_with_state(this.log_store, this.snapshot)) }) } @@ -190,71 +210,106 @@ impl std::future::IntoFuture for ConstraintBuilder { mod tests { use std::sync::Arc; - use arrow_array::{Array, Int64Array, RecordBatch}; + use arrow_array::{Array, Int32Array, RecordBatch, StringArray}; - use crate::kernel::StructType; - use crate::DeltaResult; + use crate::writer::test_utils::{create_bare_table, get_arrow_schema, get_record_batch}; + use crate::{DeltaOps, DeltaResult}; + #[cfg(feature = "datafusion")] #[tokio::test] async fn add_constraint_with_invalid_data() -> DeltaResult<()> { - let table = crate::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; + let batch = get_record_batch(None, false); + let write = DeltaOps(create_bare_table()) + .write(vec![batch.clone()]) + .await?; + let table = DeltaOps(write); + let constraint = table .add_constraint() - .with_constraint("id3", "id < 60") + .with_constraint("id", "value > 5") .await; + dbg!(&constraint); assert!(constraint.is_err()); Ok(()) } + #[cfg(feature = "datafusion")] #[tokio::test] async fn add_valid_constraint() -> DeltaResult<()> { - let table = crate::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; + let batch = get_record_batch(None, false); + let write = DeltaOps(create_bare_table()) + .write(vec![batch.clone()]) + .await?; + let table = DeltaOps(write); + let constraint = table .add_constraint() - .with_constraint("id2", "id < 1000") + .with_constraint("id", "value < 1000") .await; + dbg!(&constraint); assert!(constraint.is_ok()); let version = constraint?.version(); - assert_eq!(version, 2); + assert_eq!(version, 1); Ok(()) } + #[cfg(feature = "datafusion")] #[tokio::test] async fn add_conflicting_named_constraint() -> DeltaResult<()> { - let table = crate::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; - let constraint = table + let batch = get_record_batch(None, false); + let write = DeltaOps(create_bare_table()) + .write(vec![batch.clone()]) + .await?; + let table = DeltaOps(write); + + let new_table = table .add_constraint() - .with_constraint("id", "id < 60") - .await; + .with_constraint("id", "value < 60") + .await?; - assert!(constraint.is_err()); + let new_table = DeltaOps(new_table); + let second_constraint = new_table + .add_constraint() + .with_constraint("id", "value < 10") + .await; + dbg!(&second_constraint); + assert!(second_constraint.is_err()); Ok(()) } + #[cfg(feature = "datafusion")] #[tokio::test] async fn write_data_that_violates_constraint() -> DeltaResult<()> { - let table = crate::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; - let metadata = table.0.metadata()?; - let arrow_schema = >::try_from( - &metadata.schema()?.clone(), - )?; - let invalid_values: Vec> = vec![Arc::new(Int64Array::from(vec![-10]))]; - let batch = RecordBatch::try_new(Arc::new(arrow_schema), invalid_values)?; - let err = table.write(vec![batch]).await; + let batch = get_record_batch(None, false); + let write = DeltaOps(create_bare_table()) + .write(vec![batch.clone()]) + .await?; + let table = DeltaOps(write) + .add_constraint() + .with_constraint("id", "value > 0") + .await?; + let table = DeltaOps(table); + let invalid_values: Vec> = vec![ + Arc::new(StringArray::from(vec!["A"])), + Arc::new(Int32Array::from(vec![-10])), + Arc::new(StringArray::from(vec!["2021-02-02"])), + ]; + let batch = RecordBatch::try_new(get_arrow_schema(&None), invalid_values)?; + let err = table.write(vec![batch]).await; + dbg!(&err); assert!(err.is_err()); Ok(()) } #[tokio::test] async fn write_data_that_does_not_violate_constraint() -> DeltaResult<()> { - let table = crate::DeltaOps::try_from_uri("./tests/data/check-constraints").await?; - let metadata = table.0.metadata()?; - let arrow_schema = >::try_from( - &metadata.schema()?.clone(), - )?; - let invalid_values: Vec> = vec![Arc::new(Int64Array::from(vec![160]))]; - let batch = RecordBatch::try_new(Arc::new(arrow_schema), invalid_values)?; + let batch = get_record_batch(None, false); + let write = DeltaOps(create_bare_table()) + .write(vec![batch.clone()]) + .await?; + let table = DeltaOps(write); + let err = table.write(vec![batch]).await; assert!(err.is_ok()); diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00000-9d9248d4-bd6e-46a8-9805-5b9c1c20d5a4-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00000-9d9248d4-bd6e-46a8-9805-5b9c1c20d5a4-c000.snappy.parquet.crc deleted file mode 100644 index 4c1142b3631050669ee2f805d75da508f37c80ec..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}Bgmv-4ln0=5+ZB_IV$ diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00000-9ed8bdb4-315a-421b-8d57-f468582e5fc3-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00000-9ed8bdb4-315a-421b-8d57-f468582e5fc3-c000.snappy.parquet.crc deleted file mode 100644 index 05122662073f102329fc607217e1f52dfda5223a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00000-ca2db31d-ab94-479a-a6dc-697e58993bc6-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00000-ca2db31d-ab94-479a-a6dc-697e58993bc6-c000.snappy.parquet.crc deleted file mode 100644 index 05122662073f102329fc607217e1f52dfda5223a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00000-cace3c04-e21c-4401-9f16-1d38e6c74721-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00000-cace3c04-e21c-4401-9f16-1d38e6c74721-c000.snappy.parquet.crc deleted file mode 100644 index 27af40b4c433bf5f3bd92890753ad4917f301e4c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}D%XCG6f|S*`y7CkO?+ diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00001-53e441ee-68e6-4dde-bb53-d4bea9526242-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00001-53e441ee-68e6-4dde-bb53-d4bea9526242-c000.snappy.parquet.crc deleted file mode 100644 index 05122662073f102329fc607217e1f52dfda5223a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00001-6dea6ffc-5c2d-4b3e-b1f2-7481029003d3-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00001-6dea6ffc-5c2d-4b3e-b1f2-7481029003d3-c000.snappy.parquet.crc deleted file mode 100644 index 7f46f251fcc586268596a6e4e176bbb338d923a3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}7-dtNtr$0ow`yBKHLJ diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00001-a5330b91-bece-4fce-b2ca-81fc04c911f7-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00001-a5330b91-bece-4fce-b2ca-81fc04c911f7-c000.snappy.parquet.crc deleted file mode 100644 index 27392c680da38fc3da7ac56cd97e1464f7f3a9f8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}A{BWPUSh0ow`yB&7u; diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00002-04434872-6b96-4307-b110-bd15ea28efa4-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00002-04434872-6b96-4307-b110-bd15ea28efa4-c000.snappy.parquet.crc deleted file mode 100644 index 05122662073f102329fc607217e1f52dfda5223a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00002-24d4d181-0877-4d17-915f-ed535933662f-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00002-24d4d181-0877-4d17-915f-ed535933662f-c000.snappy.parquet.crc deleted file mode 100644 index c8834849406476f3bfcff334bad3a999575a5666..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}E@jQ-F0*eP|^BCAj?fNcc;A|?dU diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00005-319662f8-ef0e-41b9-89d8-baa3ab5348f0-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00005-319662f8-ef0e-41b9-89d8-baa3ab5348f0-c000.snappy.parquet.crc deleted file mode 100644 index 05122662073f102329fc607217e1f52dfda5223a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00005-3ae98bc2-a3bd-4f1f-ba76-9a4894572379-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00005-3ae98bc2-a3bd-4f1f-ba76-9a4894572379-c000.snappy.parquet.crc deleted file mode 100644 index d1848d96d4ee8c03642054b7e95b81dae3dacf3b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}7kHSCJRBfNcc;C6EOz diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00005-d44a39cb-4364-4a80-9f82-366415d2e69d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00005-d44a39cb-4364-4a80-9f82-366415d2e69d-c000.snappy.parquet.crc deleted file mode 100644 index 704cd3fc0035eda27dd726874c4fc7ce16173a66..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}E^~`0iQM0=5+ZDj5ay diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00006-05a437ee-7c40-4534-ae12-25fc7dbc7583-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00006-05a437ee-7c40-4534-ae12-25fc7dbc7583-c000.snappy.parquet.crc deleted file mode 100644 index b1d07c79a34c18788645af60238c38c758aa5c5d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}9kBsao;vVGs`hAV&nH diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00006-7acd5504-7d04-4006-93e2-5470e8041767-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00006-7acd5504-7d04-4006-93e2-5470e8041767-c000.snappy.parquet.crc deleted file mode 100644 index 6e09f75f6871409fdde7ce5e51c788e9662d6a7e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}CtUA(j`lfNcc;Aj$+| diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00006-9d8b5796-2cc6-48ee-b241-8d3ff6e3d5c5-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00006-9d8b5796-2cc6-48ee-b241-8d3ff6e3d5c5-c000.snappy.parquet.crc deleted file mode 100644 index 05122662073f102329fc607217e1f52dfda5223a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00007-5a919d87-b7be-4cce-b9f7-f269423daebd-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00007-5a919d87-b7be-4cce-b9f7-f269423daebd-c000.snappy.parquet.crc deleted file mode 100644 index 05122662073f102329fc607217e1f52dfda5223a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00007-68dbff7e-17b6-4c73-969b-977133666c4d-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00007-68dbff7e-17b6-4c73-969b-977133666c4d-c000.snappy.parquet.crc deleted file mode 100644 index b69b6ede7d062310b75c24e6b8414ebb2b94e6ad..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}E^Ipt&Y$0ow`yBZ36g diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00007-9f374a9d-37ec-41f6-8006-eb0db94111ee-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00007-9f374a9d-37ec-41f6-8006-eb0db94111ee-c000.snappy.parquet.crc deleted file mode 100644 index 752fca483df976fee2175565da07ceeb6bca543c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}BhXN@qsY0=5+ZByI%( diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00008-25115960-51e0-4f99-acc3-92066e12b06a-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00008-25115960-51e0-4f99-acc3-92066e12b06a-c000.snappy.parquet.crc deleted file mode 100644 index f322948501a793370d7f48a77ba4be2c6375e983..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}D&E;m4n-1#Bw-EOZ82 diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00008-568f62b3-4b06-4b80-81e6-8b745ff8a965-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00008-568f62b3-4b06-4b80-81e6-8b745ff8a965-c000.snappy.parquet.crc deleted file mode 100644 index d3580b086dec5180856d8627a585b300ad09f66b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}D&G+2BXi0=5+ZC$$BT diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00008-df9a0d39-3663-4a17-806f-ca720d517645-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00008-df9a0d39-3663-4a17-806f-ca720d517645-c000.snappy.parquet.crc deleted file mode 100644 index 05122662073f102329fc607217e1f52dfda5223a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00009-658f63ed-06a4-4160-8490-ca4a17b22d06-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00009-658f63ed-06a4-4160-8490-ca4a17b22d06-c000.snappy.parquet.crc deleted file mode 100644 index 05122662073f102329fc607217e1f52dfda5223a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 12 TcmYc;N@ieSU}BhaX+94C6R-nR diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00009-93e49bd4-3350-4ab9-bb78-f217a28c8d5a-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00009-93e49bd4-3350-4ab9-bb78-f217a28c8d5a-c000.snappy.parquet.crc deleted file mode 100644 index a5ec277a825f6f75afc6ce738b863959cbb620d1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}A_@`*JL50ow`yBme~> diff --git a/crates/deltalake-core/tests/data/check-constraints/.part-00009-da77b766-6ecf-4056-b324-502f75238979-c000.snappy.parquet.crc b/crates/deltalake-core/tests/data/check-constraints/.part-00009-da77b766-6ecf-4056-b324-502f75238979-c000.snappy.parquet.crc deleted file mode 100644 index 67d82325b997a2b6ec29fce93dcbc23089d4cc03..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}6ZCkq(bqz_tPa92o<| diff --git a/crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000000.json.crc b/crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000000.json.crc deleted file mode 100644 index 77f82b91e3347746997f7b2a3e293ea62c354ad6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 36 scmYc;N@ieSU}AVCJAH}#;TY4~(P_(_&Lzb?C|tCg=h{5~qf^+}0QguA;Q#;t diff --git a/crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000001.json.crc b/crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000001.json.crc deleted file mode 100644 index 8c18cbf62c53dad4c0715ead105d689682c82865..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16 XcmYc;N@ieSU}D(&*v8VSy;27NB*_H6 diff --git a/crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000002.json.crc b/crates/deltalake-core/tests/data/check-constraints/_delta_log/.00000000000000000002.json.crc deleted file mode 100644 index 29f5b4b28efe439716b4eee6110ef256bec498c6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 36 scmYc;N@ieSU}AXQrE<)eDOB>vsxuxX$>&xUomnIQ*ZrE>Kb;v|0P|T79smFU diff --git a/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000000.json b/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000000.json deleted file mode 100644 index 3fa9ceef08..0000000000 --- a/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000000.json +++ /dev/null @@ -1,13 +0,0 @@ -{"commitInfo":{"timestamp":1700616581733,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"10","numOutputRows":"100","numOutputBytes":"5282"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.0.0","txnId":"9fc06c03-2630-4160-9baf-a9f9f8547fb4"}} -{"metaData":{"id":"2d9d5c40-6358-4aa3-a9eb-94ba012bdef4","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1700616580070}} -{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} -{"add":{"path":"part-00000-cace3c04-e21c-4401-9f16-1d38e6c74721-c000.snappy.parquet","partitionValues":{},"size":539,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":0},\"maxValues\":{\"id\":9},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00001-6dea6ffc-5c2d-4b3e-b1f2-7481029003d3-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":10},\"maxValues\":{\"id\":19},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00002-4ca0c9db-02c8-470e-abb9-0e8a53e1d863-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":20},\"maxValues\":{\"id\":29},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00003-2bd9d2e4-27a6-4e0c-a6aa-b8c0e810fce8-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":30},\"maxValues\":{\"id\":39},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00004-f11e1f93-e35a-4fac-b3c6-d796621d2cb4-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":40},\"maxValues\":{\"id\":49},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00005-d44a39cb-4364-4a80-9f82-366415d2e69d-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":50},\"maxValues\":{\"id\":59},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00006-7acd5504-7d04-4006-93e2-5470e8041767-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":60},\"maxValues\":{\"id\":69},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00007-68dbff7e-17b6-4c73-969b-977133666c4d-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":70},\"maxValues\":{\"id\":79},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00008-568f62b3-4b06-4b80-81e6-8b745ff8a965-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581638,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":80},\"maxValues\":{\"id\":89},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00009-93e49bd4-3350-4ab9-bb78-f217a28c8d5a-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700616581639,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":90},\"maxValues\":{\"id\":99},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000001.json b/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000001.json deleted file mode 100644 index 1eba75269f..0000000000 --- a/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000001.json +++ /dev/null @@ -1,3 +0,0 @@ -{"commitInfo":{"timestamp":1700618848815,"operation":"ADD CONSTRAINT","operationParameters":{"name":"id","expr":"id > - 1"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.0.0","txnId":"43d64f90-fed8-42a6-a62e-45c640c76007"}} -{"metaData":{"id":"2d9d5c40-6358-4aa3-a9eb-94ba012bdef4","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{"delta.constraints.id":"id > - 1"},"createdTime":1700616580070}} -{"protocol":{"minReaderVersion":1,"minWriterVersion":3}} diff --git a/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000002.json b/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000002.json deleted file mode 100644 index 4eaa6d3494..0000000000 --- a/crates/deltalake-core/tests/data/check-constraints/_delta_log/00000000000000000002.json +++ /dev/null @@ -1,11 +0,0 @@ -{"commitInfo":{"timestamp":1700619387829,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"10","numOutputRows":"100","numOutputBytes":"5270"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.0.0","txnId":"10b320e3-6283-4d4a-8e86-fb6548ed5a76"}} -{"add":{"path":"part-00000-9d9248d4-bd6e-46a8-9805-5b9c1c20d5a4-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":100},\"maxValues\":{\"id\":109},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00001-a5330b91-bece-4fce-b2ca-81fc04c911f7-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":110},\"maxValues\":{\"id\":119},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00002-24d4d181-0877-4d17-915f-ed535933662f-c000.snappy.parquet","partitionValues":{},"size":526,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":120},\"maxValues\":{\"id\":129},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00003-7b12c177-7cfd-4676-986e-e038051952f2-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":130},\"maxValues\":{\"id\":139},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00004-0aae8759-c282-47ed-a5c5-6cf37b93f174-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":140},\"maxValues\":{\"id\":149},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00005-3ae98bc2-a3bd-4f1f-ba76-9a4894572379-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":150},\"maxValues\":{\"id\":159},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00006-05a437ee-7c40-4534-ae12-25fc7dbc7583-c000.snappy.parquet","partitionValues":{},"size":528,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":160},\"maxValues\":{\"id\":169},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00007-9f374a9d-37ec-41f6-8006-eb0db94111ee-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":170},\"maxValues\":{\"id\":179},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00008-25115960-51e0-4f99-acc3-92066e12b06a-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":180},\"maxValues\":{\"id\":189},\"nullCount\":{\"id\":0}}"}} -{"add":{"path":"part-00009-da77b766-6ecf-4056-b324-502f75238979-c000.snappy.parquet","partitionValues":{},"size":527,"modificationTime":1700619385735,"dataChange":true,"stats":"{\"numRecords\":10,\"minValues\":{\"id\":190},\"maxValues\":{\"id\":199},\"nullCount\":{\"id\":0}}"}} diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00000-9d9248d4-bd6e-46a8-9805-5b9c1c20d5a4-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00000-9d9248d4-bd6e-46a8-9805-5b9c1c20d5a4-c000.snappy.parquet deleted file mode 100644 index 827910fefe8cc307cdaa4178bdddee911484b409..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 527 zcmZWnO-lnY5S?ABM$m(>2?^|>1S|?Q?6R%aB7!&ZBp$qoNVbV}b+;e8TSZFGUi<|f zJcvhslE1)T;Z$3DFz`s`&1BvilGCHJkOKX1Xh~n+UY?0hJ&czr0XX+v0Pq|Gs}9)J zl?q(16X+st5DDTIakowmQmmV<>~{mP;R;U7JmO@Ypsrv5!e@lOJ9q*l>c?xvK<9T? zMd!w5nz}$Mn=W?~1I!b?@M~zQBAP&!#gwHk^Vp}u-klmbShQZ~G1&SOT(GE$gP|Y* z`*J8_IP)P)8+kQ?%{=r~=e(bHl^$rcqJCVddz%+YzsHv%)l>oS?SB=;FjY9faAZrB z_a9VdOKo(i`T1Cuj{_^)vMBRWr<8Sho!HbAvVAd@J)KQci9vc+miBvHjC!Lq)p3eh zyk_fi-cYv3lgZ@-_{mIeC(Eldf7MWhkqWC@@hC}6)x?dk6@|gRX~ug|!*-hcQD9no bLC0tt?uNDzw2X=Mw(xy?h9?K$6u-hZ$bfNi diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00000-9ed8bdb4-315a-421b-8d57-f468582e5fc3-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00000-9ed8bdb4-315a-421b-8d57-f468582e5fc3-c000.snappy.parquet deleted file mode 100644 index 364e9aaa2c2e5d971890eaf96978e93286b38dd7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00000-ca2db31d-ab94-479a-a6dc-697e58993bc6-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00000-ca2db31d-ab94-479a-a6dc-697e58993bc6-c000.snappy.parquet deleted file mode 100644 index 364e9aaa2c2e5d971890eaf96978e93286b38dd7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00000-cace3c04-e21c-4401-9f16-1d38e6c74721-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00000-cace3c04-e21c-4401-9f16-1d38e6c74721-c000.snappy.parquet deleted file mode 100644 index d0b59376d6570030bbc4657a3c364738121d4a9e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 539 zcmZWn%}OId5Uw6G*%7h_VY=ytIfRA{5n?mLB<6=j@FshbJ$MmWdeW^9PJW!3?6QPh zyvXib>{0L$Jo*B@fX}iOlbC~jsp{{ms_!ehPmjMx6zC(Ne+hl~_wvULsEuT zMRA)cT;O(QOO;Q4sgW(U(WT~Rb6NhLS~-wKna_r$Y{|>iW~PvX^SK=B(UM9H!t1hZ zGQPswc$Q^4$&kf1+mf4saxh;kE*8MQt>q3zd427#9;ql+QN5YW(#+IN(vEs@6!uLg l>Ben4?DXT%^t$2DXd4|ywjK72N%X!50(^#h0&t4o;y+B=Yj^+v diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00001-53e441ee-68e6-4dde-bb53-d4bea9526242-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00001-53e441ee-68e6-4dde-bb53-d4bea9526242-c000.snappy.parquet deleted file mode 100644 index 364e9aaa2c2e5d971890eaf96978e93286b38dd7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00001-6dea6ffc-5c2d-4b3e-b1f2-7481029003d3-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00001-6dea6ffc-5c2d-4b3e-b1f2-7481029003d3-c000.snappy.parquet deleted file mode 100644 index bf4e92f1916414c296fe007cf4d71c6ca15f39f9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 527 zcmZWnO-m~=6umQ4jURn}O+o^*C;^K?4KqxuwTR$OT!{-ekB4NMJm28-!}z`Pi<)MXxfb=Z?rC5JQVnID7g|AsRb zU2!lp1YqwDWejINg0YZSB`D@$Up3Bq*_G0LjjgD!uGFo~@}$@0bCGJO0Qk<2iu@*3 zIKa)o7AouAsg5nQ(S_z0Ls{JSt!&A>$Oi2~*5u#BrY4uI-$U8eopCBLNKeYrUiS*4 z?jTKdoMINQ*_xaR%GPi+`ZEH4IMe3S!Mgdn|J*=qyMmK!8F8{qP**Sj;WI*C96SLM?bDy- zNEdfk<>1y1HFbeDc3kcz2AC&&;kSrS8=?zjUCdePGLOAE?8Rx3wHL*pR<5b}Q zT?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00002-24d4d181-0877-4d17-915f-ed535933662f-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00002-24d4d181-0877-4d17-915f-ed535933662f-c000.snappy.parquet deleted file mode 100644 index 12940c44e228f5c5e4d90fec3d7d19e967c36c1b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 526 zcmZWn&r2gQ6n+`18$l1kBqT7060j)LFvGNKEh2bZ_9Py>h)AZ1I_&h<%yd`nR^8M7 zA0EZy{x$m-_%C?#Ra<*7@R7Xly?oz$B!>q_5d-!_SjnFLz1#%M$8ecZfaAae0H0jJ z3W4jDFBQ01qcBGNMobX5h`TkqcaL?`_2+#9x$a3pujY}U^91uG2ao}$?1A73kf?uK zEe0n4aaMFLUDq%VSY^W#UTT5+GLS)y2CO2RK$qo|WghqWf5IO~jqWX^7k(TzKMWTv zy5eAH2*95SM*FxZNI>q`H0d6D*eVky>59fH{UR8b5w zjROovuGD$|m+rdKSW_BtJXYoPz^S$>%6!x*Rb8E@F0+MdpNv({bf>A*B0Ve1`n?N` zdZR2eNrqXx=IUxLsM_Pn36Gi9cx_lHFAxxWo=@%WDwvp{3QT~_!C|MlM!xN)_ z&bglu$T6Rk4@mMAnN98Z5Wz#5H=}uPkj}n*3n|b$hnDo=-_0ZOsfTf$5`c5x1pv?S zpy2>*ysH7*Bk%#i5d!fMvA;)-eXOh2!T$}!dslGcZ6Z!K3F-<4AbdvXt%D~(qWg8X znCtw{&tm-BPBnFb);_u1O$;zk_`+|Ysha2$sEdlFF7w#E!)~1xIo`J3>M>})3T|0c z#lcVzfIT^sF`W4XszzRmU^8?px}4AQiPCe8R@93t_1ES_GRyc*r1~lV-g#3|T%-yI zxLDXy<+DpQwWT(?)ckxY%d5GSBUzOBVqD6u{FT_$6ms-^DKkB-Qi(x&U6#(W2}ao> zO?8}N7O&Z^+%%M<cAARa!yiN%~*LMNH zbGBjAft}5b3ItmOJ|iUJ3*swccZ(cVv2MP;{o6o%as?;b%ZQU@g1UkM2%ix;bMOR6 z)Q&$CBb{HJ7rjf{*VF}C*>$;_8epFAg;$Ub9z+N267|#3y=0aYTpqPh!)i@vK7fO#bwxZs;Qh#h-q{9JUi&aB~!1v!( z6xW%;0j|fkRQd2m^=+w*E;a8?WO+NXvMq};ANNXGlfP4&nL@UIP2@oL=c&{ny(r6u zgA0rX<1EujhFQF3YjPtWNFMDGjV$7i^606O>;UI8}pZEpYo diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00003-7996975c-4199-4491-a27c-4c4e76a1c817-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00003-7996975c-4199-4491-a27c-4c4e76a1c817-c000.snappy.parquet deleted file mode 100644 index 364e9aaa2c2e5d971890eaf96978e93286b38dd7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00003-7b12c177-7cfd-4676-986e-e038051952f2-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00003-7b12c177-7cfd-4676-986e-e038051952f2-c000.snappy.parquet deleted file mode 100644 index dffb701f25eeb8869d0e45e53852f1b2782282f4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 527 zcmZWnO-m~=6umRlz6iQ_FbN6Fq691oePM=awH6WF$yi?~DFBOVsX#v|5^*VFd~;-@P(xtvCvOcT@<3_$pd&}RovfJAM3 zA@A$#&tcv^u^mlaprs|3yNLnj319eC;?t6704a+xOI_x%H-|ktRkAUYp7}9Y{%1I2 z(G>?nLjd;SP{wfPO&AM#Rf1w3_EqD&mmMqJ*Vu~s>Pns2EKhn}J{PHm3V^TtSCOBl z3I{kH*g|E!bJekhHoDM!Zzzk4zLhPR7uld)$eKJ#Y-)1Z+8@fU?u=83L3&b__PWOy zbq8sx;}o-a&DP{pP_~An(ZLA#?}^>7PF9)pR|!=Zsj#vf50cbWOk59}Q5dY7M!Xi) jZM(4^1*W+cw2ijmYG~_0)0kMV2;awNxN`t@@GE=*>`imh diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00004-0aae8759-c282-47ed-a5c5-6cf37b93f174-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00004-0aae8759-c282-47ed-a5c5-6cf37b93f174-c000.snappy.parquet deleted file mode 100644 index 71b259cb03b741326327eff6dbaa8b24fe7130d4..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 527 zcmZWn%}OId5Uv@b+oB$X>82az5E?cp#Ab#`GzmfQCZ1#uUY2F)Nw+%e!$1B{{~{i6`V}w5hwEmbp-JI{WrD@0{7LrY_LZN0+;a0p zW3c&VxM0y02SY;u_Q#=&;mmt574j+s#XRh*%6UIKReGSY74@$x_1$K9((m!5NYzyU zeCu6Bex52E;CyHcmGyt9t}V3Dh33a2SzHXPY{|UHhMhuIcZyMOn5H^TF^ku1Rn7%vYcw8z83R9@*=={T@~ppdsKQ8v<;{4Qq^4}*TG)uf pplRyyZd9|KdNT@4V>jp+ZNr_=)`EsHvECBCkI!)F036|0_zTT5bX@=d diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00004-4ca9cc40-3cd3-45b4-a3cc-58de2fec437d-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00004-4ca9cc40-3cd3-45b4-a3cc-58de2fec437d-c000.snappy.parquet deleted file mode 100644 index 364e9aaa2c2e5d971890eaf96978e93286b38dd7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00004-f11e1f93-e35a-4fac-b3c6-d796621d2cb4-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00004-f11e1f93-e35a-4fac-b3c6-d796621d2cb4-c000.snappy.parquet deleted file mode 100644 index c1c2ba1ed6b1c68074e21d4c272f28e34bba8e16..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 527 zcmZWn%}yIJ5T0EE7^xhP)|M^pA+ppcX&1BBCJ-W2skf>p#DPnNkT*84V)M7VX@#Pk zc>>;m0|#D(K0%*=r(i;p=1@zY?D=N=eKWF8zn*Cd^x2~oeR%o(v=Pz(<1I=6&O;vn z09<+A0Yw&58z-L5+*heVD!7e$8v2L;S-wni`FF0wgB2HEb>I()SLPqGLhbKTH z`u3q5o8tDe?BBS7p+3;sN1ywd1r~@idtGlt2Jy}-8v|q_cUS%$~rR@Ef$)Oo6Qkg}1S(cB6 z*BA|_d2Z4ivv|!#awRBxv-$jD4*dJlZhuhJm;KeX(uvabW;)GsTeoRjcM}~S+fI6v jv|Yb*oW!T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00005-3ae98bc2-a3bd-4f1f-ba76-9a4894572379-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00005-3ae98bc2-a3bd-4f1f-ba76-9a4894572379-c000.snappy.parquet deleted file mode 100644 index ddcddb6fb10a7abbde9e5fd0189d43556762e716..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 527 zcmZWnO-my|5Um-awx|bTy6J{Fga(5`wwYlPjX@T?i6`;kMP%to_v+x}Yi6P>A?N)E z>hAh+@#s&IWzYE)Rx~k(ZFqFotLpcv=swy%4k^$VhZgkjc>Hh0ryjx5F1ARaXJt3t2Ews^v<|jj0{ODQPlzEX2T7|62?}<%KE}N%A+0pGuDlte;%hGP= z9HY)4O?8}N7O&Z=oC(V2a5Op_0Y9ADeQ9UqX@BKVg^>!&oADq?P1(e?un~p9uBpd6 kQO&mMyHQ{oJ3-568*YcT7Bq~B^_K8`e1@M6z!83hH($MVzyJUM diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00005-d44a39cb-4364-4a80-9f82-366415d2e69d-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00005-d44a39cb-4364-4a80-9f82-366415d2e69d-c000.snappy.parquet deleted file mode 100644 index 540bb969cf704bc6290ba3dbe6084b1e117f3fe2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 527 zcmZWnO-my|5Um-qNeg;dm~OgZ4xwR#LTqL-8S|kmcw0P)2QMN^Pr8kRlMiPmE=$N& zPoDe>f){_1zhHlb6-~@x8y?;Ds`|Yux=)YKLJIWNp(TBMx__zp)Wf(+3BbAU0)XeV zV9SAMt5$*b4uKePfaoB)h~5r4>SNt<9e!>gzPo~x);i*3ouIB@0K#X4K09~c0UeiwslJJQqzTKVB}H#NXK;S0Y(d|DAPfvQ-t)MXxfbJ(-fAV(YNjUR*EFT)Ls zt~eMP0cbK~T)YzM7m*@}bgGjjgDUuGF2)i*z#P-y#*O0Qla26~%3) zaDdyHEmc1GqeiyWMwgnO&t>^{YGq#*Wj-5}vMH}po0&rPFXnQrN6S=dkY1H#lkpIv z@hr=9l3^CF*`{0z%Km(@xLg1~S=sH6^7^X3dZ@xkh4tNJmS(1Il6KgQ!k}m3q!YF6 hAnrwh>2`vF(Kb8?Z9C{1ljuF+`}hnG4!|jXg?C)BZZ`k` diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00006-05a437ee-7c40-4534-ae12-25fc7dbc7583-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00006-05a437ee-7c40-4534-ae12-25fc7dbc7583-c000.snappy.parquet deleted file mode 100644 index 6aa24c5c22c5d4a3b77d6449eed29fe851f6f66e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 528 zcmZWnO=}x55FIUUqCit9tb_!6FoKu_T(QgAiDN<`x6)I{A(s+@)>6|Y>#yB)N-@3_ z`Ztoabs7l$Jw4=_TWA zJOJ>?1*{RcTwATc)jEam7(Xz6VocX*+&UsyO>H%}YL*PaygVvz~DAebjPfDAZg zQ-UWzp#JVaH_3}G9PtHRaYNVm)SzKKa5q+bZ1m* z5uTT2{oW~Ny-}8#BtsUjxw={is`hv?Ii7%cKev0+&8zeNs*#Rj9aT4zQJUGRO&U=v qj>28rOm^ai>oj-c(6)BMjx{dYj$9*bS(}(G83gzY-w40~{)NB9a&`&; diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00006-7acd5504-7d04-4006-93e2-5470e8041767-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00006-7acd5504-7d04-4006-93e2-5470e8041767-c000.snappy.parquet deleted file mode 100644 index 3d88c4aa21420ae94ea6e8e877e942b9f41e4fc7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 527 zcmZWn!D<^Z5S?9|R3Xiwuo4pN!3bhvaK$cb$BuE+CTXDb6mrNVl%lm%cj>Oz-d(2@ z<8yx?pO9nzp+C|u$f5KTGEVIJP{Bi*H>3AvkS$W$0qi*NX6Ho>j`s+>MZ80tAWjiydt~q)>z3>Mc?0pv6`Ty#5hv>ebp-kE!ih0;qi}Sn~DLv8HihAlw-Poc`@-g3vR7VBC zoBvgm(^TOA)0wSQk$+d2t+dgV=2vrB{g_zUmt|GVhLvo|FNsY}Df`!RIo8=Sl^CQ~ zWobShVKknlsg6_3;x*fnYeCtcFBYE`z(20+US~yp)n7eSVWh(PK|D)RQ#Wxt>_uU4 oWIAy-YTIGwC<;uk8w`!M;bCaoLC=_2H-+!xGyHS_F7PWn0yyGtzW@LL diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00006-9d8b5796-2cc6-48ee-b241-8d3ff6e3d5c5-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00006-9d8b5796-2cc6-48ee-b241-8d3ff6e3d5c5-c000.snappy.parquet deleted file mode 100644 index 364e9aaa2c2e5d971890eaf96978e93286b38dd7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00007-5a919d87-b7be-4cce-b9f7-f269423daebd-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00007-5a919d87-b7be-4cce-b9f7-f269423daebd-c000.snappy.parquet deleted file mode 100644 index 364e9aaa2c2e5d971890eaf96978e93286b38dd7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00007-68dbff7e-17b6-4c73-969b-977133666c4d-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00007-68dbff7e-17b6-4c73-969b-977133666c4d-c000.snappy.parquet deleted file mode 100644 index 5ac21aaf0422e64976f45689ec73271e93f0fbf2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 527 zcmZWn&1xGl5T0EeM=r^suo4pN!3bg!aK$cbH?}c_Ufle(4L$f$LeN@DvSj_YyKX7Q z$I=JrEBM$)$`gb>LJu7`c6_MdLz-_!-#3GF`Sw#tf&Ox6L4W`G{_}-TJ&eng0Q7wq z06eD$6$jo`c1v)wPv8`BhB!xDAl~nj?gy+}t&dL|h?lP5q_>GU*(9hd7=Z8@p-Tr( zfJFUdFQ4e_{yHDr*`cN`(9(g+-NXR%gfILW@o7mk36#Z(r7rW>gTt0ijdZutTR#Se z&xTtTU2!lp1YnO2WejJ211lk~Mo`SdzUrKhvs8?kmM7y8--%RH1;CH~ ztH`IR!U3i;Tc~XORSj*SjV?6r&t>s#Vr56>MK&80vMz5Do0?p9KF{Sy4_B$gAiXY2 z$D><}Mzb{4af(^IX6tezC_D4T;%Wi>Y;E^ym{r&PRYMgifU|Pq)z-Sx34s9c7858Rx;rsXuKOBHd{0e^o;Ms63 diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00007-9f374a9d-37ec-41f6-8006-eb0db94111ee-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00007-9f374a9d-37ec-41f6-8006-eb0db94111ee-c000.snappy.parquet deleted file mode 100644 index 50479539bee2c237b928cebc57a9c051102ff0c9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 527 zcmZWnO-m~=6umQ4jqqFulaRnHO5hcR8fKVQYZ1YnxRSSUBl5^JdA`Bvhci<}O4t4o zSAKu&{2{LV1uh@{0B?P*T^Kkd_uQLv?j_mX`V~^3*A6Y{%fILMl21L1XDI>L^IZV& zoJ*K@;A(!Z1lJ1$ZV&Rc(NTT>Tk>8H!x!~pYzFZ>GeX-U+9%!(09UFNY@hg~=ovN4sO`Y~Ag zGMuvLii4pc0DE&NV>t6BjD)-jK`{^es&d}V4wdd{Y(;%`rA};?C*2O8iBw$$z?Z*O zwQZq|E;Qd8$l|PLWmD!w)^8QED*q%lHMwl=4`fHTN2$ahJuXYT zokNT|{WR5aidnp7t8yYJn}gx-U1;GLoOi{t)-?*)?d5pgkXHm zAIULC=c7&fPx1?T@3|c(c6^B7A281GnY%65MSNxJNu7ejzVNHWrzOze)gYU1)wZk;UcE%9hNFY}_qmO@2>oYI4~+p2)uL%~Od%dQp}R z`llH6$7!nL6tj5E*5p!9wkFf*$rSjPh23s1t1SAfger_wSlN!pNop!4u7}Mi3=T{q m-jC|G+c<~<)7%fbM%!>NwDq8AOssc=@8dIEIRG8}3jYBB4R>Gw diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00008-568f62b3-4b06-4b80-81e6-8b745ff8a965-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00008-568f62b3-4b06-4b80-81e6-8b745ff8a965-c000.snappy.parquet deleted file mode 100644 index d1f5d89f7ff0a56ff3d1e594527d2e6ee9e36296..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 527 zcmZWnO=}x55S?9|M27lMSP2RCU<5G4jh-aN^nvkaEdrXd_nXP=M~buz`Diy`nG}i=n77HtB8|Tg1UkM2%i!9k$_KZ0sHqFIwC!>?F~B_G3%^EuS`tkHn_|IImwD{jVNXttbl1{rKL$JR zhHDmGaWFIlV1FFS7|whE3n8yYP|U-=>YR_W8>J^2TT%bIQr~TsC*u*{h*VPrz<2+v z$nR5y1Kdw-p|bH0HME5`y3o8olf}ct%8ty7Y&s}pU0x?PHM#73o5_(LE>ej>dRdl^ zM>iOarfI6<6tj5E*5yi2c4qVWT?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00009-658f63ed-06a4-4160-8490-ca4a17b22d06-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00009-658f63ed-06a4-4160-8490-ca4a17b22d06-c000.snappy.parquet deleted file mode 100644 index 364e9aaa2c2e5d971890eaf96978e93286b38dd7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 296 zcmZXQO-sZu5QbyX+g`h*&_f9bF4WM_*6sG@P4={S*o%lvJ8>T?4)RV`V z=b3kgIov-A-wWc0l^H<4znJ<>w%~ijzr6OS*C1JwU=_ra_WHzO0LS^S%0HwwP2bV0 zl;=Dlbo0+JXT@FWd6bsx{*$-T0)hp4oS6L@BrBP*daNyr*=r*e8dg3{tb_IkHR#%l zruy#Ws%}&YD&?pflQH`XWaTu^&oiO>W!-J7qvgIxaFKEm-Be?vFv2PmMJkdV=G8XM hWS#HQ1dDA_BS>*4WR?_&6>NeqB!q+)gdE(Dz5x@AShxTH diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00009-93e49bd4-3350-4ab9-bb78-f217a28c8d5a-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00009-93e49bd4-3350-4ab9-bb78-f217a28c8d5a-c000.snappy.parquet deleted file mode 100644 index 2ea76d747fda72d4a4947ff22639e02b6042191c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 527 zcmZWn%W4}j6umPzsRH$)FcK2X!U$q)aKsE_*AGG=yU>;CB8xnlXhw=Nbo|K7xTP4c zv&^<1;AK7|KT+rx^eb}R+VLWSgLKcmI_F-b;mMbf0)295NuPiHeysS^!+4((fHU6( z0M9vx0|zb+-c{hTO5h6d6>*KYLFg(uje7HO;A@b0O2!2R}P*4 ziPq`+Vy5$N*F|z?$C|o8D~B$3Qv=KszVPeBrxnp5urJmub(zQhIBezA$?;Bl=f|M& zYPe(36$e8@0QT3RjN#0mU@hd;35t2xSBvv$K2mz7u@&{NEA`#xMLM1Ey-0Ob0KEC8 zqPWi#4sbuWrOKy2)Yz8V=u-2ug)D#0tQ^Rq%;!ldTk5^gZNrbDZ3jJLV%-$JkI(Sn01WXfyZ~|`aVr1- diff --git a/crates/deltalake-core/tests/data/check-constraints/part-00009-da77b766-6ecf-4056-b324-502f75238979-c000.snappy.parquet b/crates/deltalake-core/tests/data/check-constraints/part-00009-da77b766-6ecf-4056-b324-502f75238979-c000.snappy.parquet deleted file mode 100644 index df84f4dd708da2fb5db60c7992e0341d83de64b2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 527 zcmZWn&1xGl5T0F}REGEvSP2RCun1yeaKtWaH?}c_-bzoVhg?c1T1!cmu79$-ZYjp+ zzKD-~gyR0UPtaSRpbrr`o7(lEf)8oF8GYXj(&+KChys1*(31YVdw+M}QxD@qN&qf= z7XUnG19b;J)F0H~V}rma#An18#8D+f3mAw+maUeD+#RY^jYdHNRNO^3B}Jfh@{=F)n3WUZplOg&aIz%Bh}Isnj67 zEz4%pYmBCgEYnGbS-fW3awjMU%hl>~1^nsO?rf3=+x~(`MX`#4(`1omCNN1S>cvsm oH{Iks?$~j+ABU!Q9*&K+(Zk4g!k#gSZVBJVXL#!XjPNV`0d@X(H2?qr diff --git a/python/src/lib.rs b/python/src/lib.rs index 5094e64149..40ea6b00d5 100644 --- a/python/src/lib.rs +++ b/python/src/lib.rs @@ -1342,7 +1342,7 @@ impl PyDeltaDataChecker { }) .collect(); Self { - inner: DeltaDataChecker::with_invariants(invariants), + inner: DeltaDataChecker::new_with_invariants(invariants), rt: tokio::runtime::Runtime::new().unwrap(), } } From c5e23c1dad6a98d0fcdf3078015949fcf8334d83 Mon Sep 17 00:00:00 2001 From: Stephen Carman Date: Sat, 9 Dec 2023 18:15:32 -0500 Subject: [PATCH 11/13] feat: check constraints, fix a lint --- crates/deltalake-core/src/delta_datafusion/mod.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/crates/deltalake-core/src/delta_datafusion/mod.rs b/crates/deltalake-core/src/delta_datafusion/mod.rs index 649fbf58d4..e1079e8029 100644 --- a/crates/deltalake-core/src/delta_datafusion/mod.rs +++ b/crates/deltalake-core/src/delta_datafusion/mod.rs @@ -1098,13 +1098,12 @@ impl DeltaDataChecker { let dfs: Vec = self.ctx.sql(&sql).await?.collect().await?; if !dfs.is_empty() && dfs[0].num_rows() > 0 { - let values: String = dfs[0] + let value: String = dfs[0] .columns() .iter() .map(|c| array_value_to_string(c, 0).unwrap_or(String::from("null"))) .join(", "); - let value = format!("{}", values); let msg = format!( "Check or Invariant ({}) violated by value in row: [{}]", check.get_expression(), From 1efd1fe013276a8df582784d0ecf0286eaae1539 Mon Sep 17 00:00:00 2001 From: Stephen Carman Date: Wed, 13 Dec 2023 14:34:23 -0500 Subject: [PATCH 12/13] feat: CHECK constraints addressing a lint --- crates/deltalake-core/src/operations/constraints.rs | 9 +++------ .../src/operations/transaction/protocol.rs | 2 ++ 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/crates/deltalake-core/src/operations/constraints.rs b/crates/deltalake-core/src/operations/constraints.rs index 734c9a24a6..889e668b1a 100644 --- a/crates/deltalake-core/src/operations/constraints.rs +++ b/crates/deltalake-core/src/operations/constraints.rs @@ -186,7 +186,7 @@ impl std::future::IntoFuture for ConstraintBuilder { Action::Protocol(protocol), ]; - let _version = commit( + let version = commit( this.log_store.as_ref(), &actions, operations, @@ -195,11 +195,8 @@ impl std::future::IntoFuture for ConstraintBuilder { ) .await?; - this.snapshot.merge( - DeltaTableState::from_actions(actions, _version)?, - true, - true, - ); + this.snapshot + .merge(DeltaTableState::from_actions(actions, version)?, true, true); Ok(DeltaTable::new_with_state(this.log_store, this.snapshot)) }) } diff --git a/crates/deltalake-core/src/operations/transaction/protocol.rs b/crates/deltalake-core/src/operations/transaction/protocol.rs index 9c20755935..8d350f26dd 100644 --- a/crates/deltalake-core/src/operations/transaction/protocol.rs +++ b/crates/deltalake-core/src/operations/transaction/protocol.rs @@ -169,7 +169,9 @@ pub static INSTANCE: Lazy = Lazy::new(|| { let mut writer_features = HashSet::new(); writer_features.insert(WriterFeatures::AppendOnly); + #[cfg(feature = "datafusion")] writer_features.insert(WriterFeatures::Invariants); + #[cfg(feature = "datafusion")] writer_features.insert(WriterFeatures::CheckConstraints); // writer_features.insert(WriterFeatures::ChangeDataFeed); // writer_features.insert(WriterFeatures::GeneratedColumns); From e4f59a3d860092767c87f2fee1ca93a7a355ef04 Mon Sep 17 00:00:00 2001 From: Stephen Carman Date: Wed, 13 Dec 2023 19:21:02 -0500 Subject: [PATCH 13/13] feat: CHECK constraints, back out limiting features behind feature flag because it caused many tests to fail --- crates/deltalake-core/src/operations/transaction/protocol.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/crates/deltalake-core/src/operations/transaction/protocol.rs b/crates/deltalake-core/src/operations/transaction/protocol.rs index 8d350f26dd..9c20755935 100644 --- a/crates/deltalake-core/src/operations/transaction/protocol.rs +++ b/crates/deltalake-core/src/operations/transaction/protocol.rs @@ -169,9 +169,7 @@ pub static INSTANCE: Lazy = Lazy::new(|| { let mut writer_features = HashSet::new(); writer_features.insert(WriterFeatures::AppendOnly); - #[cfg(feature = "datafusion")] writer_features.insert(WriterFeatures::Invariants); - #[cfg(feature = "datafusion")] writer_features.insert(WriterFeatures::CheckConstraints); // writer_features.insert(WriterFeatures::ChangeDataFeed); // writer_features.insert(WriterFeatures::GeneratedColumns);